Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add gap-filling transform for timeseries #15786

Merged
merged 2 commits into from
Jul 13, 2022
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions .github/autolabeler.yml
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ extensions: ["sdks/java/extensions/**/*", "runners/extensions-java/**/*"]
"sketching": ["sdks/java/extensions/sketching/**/*"]
"sorter": ["sdks/java/extensions/sorter/**/*"]
"sql": ["sdks/java/extensions/sql/**/*"]
"timeseries": ["sdks/java/extensions/timeseries/*"]
"zetasketch": ["sdks/java/extensions/zetasketch/**/*"]

# IO
Expand Down
1 change: 1 addition & 0 deletions build.gradle.kts
Original file line number Diff line number Diff line change
Expand Up @@ -195,6 +195,7 @@ tasks.register("javaPreCommitPortabilityApi") {

tasks.register("javaPostCommit") {
dependsOn(":sdks:java:extensions:google-cloud-platform-core:postCommit")
dependsOn(":sdks:java:extensions:timeseries:postCommit")
dependsOn(":sdks:java:extensions:zetasketch:postCommit")
dependsOn(":sdks:java:extensions:ml:postCommit")
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,197 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.sdk.coders;

import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.Arrays;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.SortedMap;
import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
import org.apache.beam.sdk.values.TypeDescriptor;
import org.apache.beam.sdk.values.TypeParameter;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps;

/**
* A {@link Coder} for {@link SortedMap Maps} that encodes them according to provided coders for
* keys and values.
*
* @param <K> the type of the keys of the KVs being transcoded
* @param <V> the type of the values of the KVs being transcoded
*/
public class SortedMapCoder<K extends Comparable<? super K>, V>
extends StructuredCoder<SortedMap<K, V>> {
/** Produces a MapCoder with the given keyCoder and valueCoder. */
public static <K extends Comparable<? super K>, V> SortedMapCoder<K, V> of(
Coder<K> keyCoder, Coder<V> valueCoder) {
return new SortedMapCoder<>(keyCoder, valueCoder);
}

public Coder<K> getKeyCoder() {
return keyCoder;
}

public Coder<V> getValueCoder() {
return valueCoder;
}

/////////////////////////////////////////////////////////////////////////////

private Coder<K> keyCoder;
private Coder<V> valueCoder;

private SortedMapCoder(Coder<K> keyCoder, Coder<V> valueCoder) {
this.keyCoder = keyCoder;
this.valueCoder = valueCoder;
}

@Override
public void encode(SortedMap<K, V> map, OutputStream outStream)
throws IOException, CoderException {
encode(map, outStream, Context.NESTED);
}

@Override
public void encode(SortedMap<K, V> map, OutputStream outStream, Context context)
throws IOException, CoderException {
if (map == null) {
throw new CoderException("cannot encode a null SortedMap");
}
DataOutputStream dataOutStream = new DataOutputStream(outStream);

int size = map.size();
dataOutStream.writeInt(size);
if (size == 0) {
return;
}

// Since we handled size == 0 above, entry is guaranteed to exist before and after loop
Iterator<Entry<K, V>> iterator = map.entrySet().iterator();
Entry<K, V> entry = iterator.next();
while (iterator.hasNext()) {
keyCoder.encode(entry.getKey(), outStream);
valueCoder.encode(entry.getValue(), outStream);
entry = iterator.next();
}

keyCoder.encode(entry.getKey(), outStream);
valueCoder.encode(entry.getValue(), outStream, context);
// no flush needed as DataOutputStream does not buffer
}

@Override
public SortedMap<K, V> decode(InputStream inStream) throws IOException, CoderException {
return decode(inStream, Context.NESTED);
}

@Override
public SortedMap<K, V> decode(InputStream inStream, Context context)
throws IOException, CoderException {
DataInputStream dataInStream = new DataInputStream(inStream);
int size = dataInStream.readInt();
if (size == 0) {
return Collections.emptySortedMap();
}

SortedMap<K, V> retval = Maps.newTreeMap();
for (int i = 0; i < size - 1; ++i) {
K key = keyCoder.decode(inStream);
V value = valueCoder.decode(inStream);
retval.put(key, value);
}

K key = keyCoder.decode(inStream);
V value = valueCoder.decode(inStream, context);
retval.put(key, value);
return retval;
}

/**
* {@inheritDoc}
*
* @return a {@link List} containing the key coder at index 0 at the and value coder at index 1.
*/
@Override
public List<? extends Coder<?>> getCoderArguments() {
return Arrays.asList(keyCoder, valueCoder);
}

/**
* {@inheritDoc}
*
* @throws NonDeterministicException always. Not all maps have a deterministic encoding. For
* example, {@code HashMap} comparison does not depend on element order, so two {@code
* HashMap} instances may be equal but produce different encodings.
*/
@Override
public void verifyDeterministic() throws NonDeterministicException {
throw new NonDeterministicException(
this, "Ordering of entries in a Map may be non-deterministic.");
}

@Override
public boolean consistentWithEquals() {
return keyCoder.consistentWithEquals() && valueCoder.consistentWithEquals();
}

@Override
public Object structuralValue(SortedMap<K, V> value) {
if (consistentWithEquals()) {
return value;
} else {
Map<Object, Object> ret = Maps.newHashMapWithExpectedSize(value.size());
for (Map.Entry<K, V> entry : value.entrySet()) {
ret.put(
keyCoder.structuralValue(entry.getKey()), valueCoder.structuralValue(entry.getValue()));
}
return ret;
}
}

@Override
public void registerByteSizeObserver(SortedMap<K, V> map, ElementByteSizeObserver observer)
throws Exception {
observer.update(4L);
if (map.isEmpty()) {
return;
}
Iterator<Entry<K, V>> entries = map.entrySet().iterator();
Entry<K, V> entry = entries.next();
while (entries.hasNext()) {
keyCoder.registerByteSizeObserver(entry.getKey(), observer);
valueCoder.registerByteSizeObserver(entry.getValue(), observer);
entry = entries.next();
}
keyCoder.registerByteSizeObserver(entry.getKey(), observer);
valueCoder.registerByteSizeObserver(entry.getValue(), observer);
}

@Override
public TypeDescriptor<SortedMap<K, V>> getEncodedTypeDescriptor() {
return new TypeDescriptor<SortedMap<K, V>>() {}.where(
new TypeParameter<K>() {}, keyCoder.getEncodedTypeDescriptor())
.where(new TypeParameter<V>() {}, valueCoder.getEncodedTypeDescriptor());
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,79 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.sdk.schemas.transforms;

import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.schemas.FieldAccessDescriptor;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.schemas.SchemaCoder;
import org.apache.beam.sdk.schemas.utils.RowSelector;
import org.apache.beam.sdk.schemas.utils.SelectHelpers;
import org.apache.beam.sdk.schemas.utils.SelectHelpers.RowSelectorContainer;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.Row;
import org.apache.beam.sdk.values.TypeDescriptor;

public class WithKeys<T> extends PTransform<PCollection<T>, PCollection<KV<Row, T>>> {
private final FieldAccessDescriptor fieldAccessDescriptor;

public static <T> WithKeys<T> of(FieldAccessDescriptor fieldAccessDescriptor) {
return new WithKeys<>(fieldAccessDescriptor);
}

private WithKeys(FieldAccessDescriptor fieldAccessDescriptor) {
this.fieldAccessDescriptor = fieldAccessDescriptor;
}

@Override
public PCollection<KV<Row, T>> expand(PCollection<T> input) {
Schema schema = input.getSchema();
TypeDescriptor<T> typeDescriptor = input.getTypeDescriptor();
if (typeDescriptor == null) {
throw new RuntimeException("Null type descriptor on input.");
}
SerializableFunction<T, Row> toRowFunction = input.getToRowFunction();
SerializableFunction<Row, T> fromRowFunction = input.getFromRowFunction();

FieldAccessDescriptor resolved = fieldAccessDescriptor.resolve(schema);
RowSelector rowSelector = new RowSelectorContainer(schema, resolved, true);
Schema keySchema = SelectHelpers.getOutputSchema(schema, resolved);

return input
.apply(
"selectKeys",
ParDo.of(
new DoFn<T, KV<Row, T>>() {
@ProcessElement
public void process(
@Element Row row, // Beam will convert the element to a row.
@Element T element, // Beam will return the original element.
OutputReceiver<KV<Row, T>> o) {
o.output(KV.of(rowSelector.select(row), element));
}
}))
.setCoder(
KvCoder.of(
SchemaCoder.of(keySchema),
SchemaCoder.of(schema, typeDescriptor, toRowFunction, fromRowFunction)));
}
}
32 changes: 32 additions & 0 deletions sdks/java/extensions/timeseries/build.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* License); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an AS IS BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

plugins { id 'org.apache.beam.module' }
applyJavaNature(
automaticModuleName: 'org.apache.beam.sdk.extensions.timeseries'
)

description = "Apache Beam :: SDKs :: Java :: Extensions :: Timeseries"

dependencies {
implementation library.java.vendored_guava_26_0_jre
implementation library.java.joda_time
implementation project(path: ":sdks:java:core", configuration: "shadow")
testImplementation library.java.junit
testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow")
}
Loading