From 099e7f39121cae94eeb434e1363f6ec3a28b43c0 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 5 May 2017 16:20:37 -0700 Subject: [PATCH 01/11] Remove explicit used of nested contexts. find . -type f -name '*.java' | xargs sed -i '' 's/\([.]..code[(].*\), *context.nested..[)]/\1)/' find . -type f -name '*.java' | xargs sed -i '' 's/\([.]..code[(].*\), *nestedContext[)]/\1)/' find . -type f -name '*.java' | xargs sed -i '' 's/\([.]..code[(].*\), *Context.NESTED[)]/\1)/' find . -type f -name '*.java' | xargs sed -i '' 's/\([.]..code[(].*\), *[^ ]*.Context.NESTED[)]/\1)/' Added back explicit context in CoGbkResult.java due to compile error. --- .../UnboundedReadFromBoundedSource.java | 4 +-- .../core/ElementAndRestrictionCoder.java | 4 +-- .../beam/runners/core/KeyedWorkItemCoder.java | 8 +++--- .../beam/runners/core/TimerInternals.java | 12 ++++---- .../types/CoderTypeSerializer.java | 4 +-- .../SingletonKeyedWorkItemCoder.java | 4 +-- .../state/FlinkKeyGroupStateInternals.java | 8 +++--- .../runners/dataflow/BatchViewOverrides.java | 4 +-- .../runners/dataflow/internal/IsmFormat.java | 24 ++++++++-------- .../spark/aggregators/NamedAggregators.java | 4 +-- .../beam/sdk/coders/BigDecimalCoder.java | 4 +-- .../beam/sdk/coders/IterableLikeCoder.java | 8 +++--- .../org/apache/beam/sdk/coders/KvCoder.java | 4 +-- .../org/apache/beam/sdk/coders/MapCoder.java | 12 ++++---- .../org/apache/beam/sdk/io/FileBasedSink.java | 4 +-- .../sdk/transforms/ApproximateQuantiles.java | 20 ++++++------- .../beam/sdk/transforms/CombineFns.java | 4 +-- .../org/apache/beam/sdk/transforms/Mean.java | 4 +-- .../beam/sdk/transforms/join/CoGbkResult.java | 2 +- .../transforms/windowing/IntervalWindow.java | 4 +-- .../apache/beam/sdk/util/WindowedValue.java | 10 +++---- .../beam/sdk/values/TimestampedValue.java | 4 +-- .../beam/sdk/values/ValueInSingleWindow.java | 12 ++++---- .../beam/sdk/values/ValueWithRecordId.java | 4 +-- .../sdk/coders/SerializableCoderTest.java | 28 +++++++++---------- .../beam/sdk/transforms/CombineTest.java | 4 +-- .../beam/sdk/transforms/CreateTest.java | 4 +-- .../windowing/GlobalWindowTest.java | 2 +- ...fferedElementCountingOutputStreamTest.java | 5 ++-- .../BeamFnDataBufferingOutboundObserver.java | 2 +- .../data/BeamFnDataInboundObserver.java | 2 +- ...amFnDataBufferingOutboundObserverTest.java | 2 +- .../data/BeamFnDataInboundObserverTest.java | 2 +- .../sdk/io/gcp/bigquery/ShardedKeyCoder.java | 10 +++---- .../gcp/bigquery/TableDestinationCoder.java | 10 +++---- .../io/gcp/bigquery/TableRowInfoCoder.java | 4 +-- .../io/gcp/bigquery/WriteBundlesToFiles.java | 12 ++++---- .../PubsubMessageWithAttributesCoder.java | 4 +-- .../io/gcp/pubsub/PubsubUnboundedSink.java | 16 +++++------ .../io/gcp/pubsub/PubsubUnboundedSource.java | 2 +- .../apache/beam/sdk/io/xml/JAXBCoderTest.java | 8 +++--- 41 files changed, 144 insertions(+), 145 deletions(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java index 1424b8be2518..ae28e3adbd3e 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java @@ -223,7 +223,7 @@ static class CheckpointCoder extends StructuredCoder> { @Override public void encode(Checkpoint value, OutputStream outStream, Context context) throws CoderException, IOException { - elemsCoder.encode(value.residualElements, outStream, context.nested()); + elemsCoder.encode(value.residualElements, outStream); sourceCoder.encode(value.residualSource, outStream, context); } @@ -232,7 +232,7 @@ public void encode(Checkpoint value, OutputStream outStream, Context context) public Checkpoint decode(InputStream inStream, Context context) throws CoderException, IOException { return new Checkpoint<>( - elemsCoder.decode(inStream, context.nested()), + elemsCoder.decode(inStream), sourceCoder.decode(inStream, context)); } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java index 83c4e6281d71..5ddd865a518f 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java @@ -55,14 +55,14 @@ public void encode( if (value == null) { throw new CoderException("cannot encode a null ElementAndRestriction"); } - elementCoder.encode(value.element(), outStream, context.nested()); + elementCoder.encode(value.element(), outStream); restrictionCoder.encode(value.restriction(), outStream, context); } @Override public ElementAndRestriction decode(InputStream inStream, Context context) throws IOException { - ElementT key = elementCoder.decode(inStream, context.nested()); + ElementT key = elementCoder.decode(inStream); RestrictionT value = restrictionCoder.decode(inStream, context); return ElementAndRestriction.of(key, value); } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java index e1872b542260..ac8a34cfb9b0 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java @@ -72,8 +72,8 @@ public Coder getElementCoder() { public void encode(KeyedWorkItem value, OutputStream outStream, Coder.Context context) throws CoderException, IOException { Coder.Context nestedContext = context.nested(); - keyCoder.encode(value.key(), outStream, nestedContext); - timersCoder.encode(value.timersIterable(), outStream, nestedContext); + keyCoder.encode(value.key(), outStream); + timersCoder.encode(value.timersIterable(), outStream); elemsCoder.encode(value.elementsIterable(), outStream, context); } @@ -81,8 +81,8 @@ public void encode(KeyedWorkItem value, OutputStream outStream, Coder. public KeyedWorkItem decode(InputStream inStream, Coder.Context context) throws CoderException, IOException { Coder.Context nestedContext = context.nested(); - K key = keyCoder.decode(inStream, nestedContext); - Iterable timers = timersCoder.decode(inStream, nestedContext); + K key = keyCoder.decode(inStream); + Iterable timers = timersCoder.decode(inStream); Iterable> elems = elemsCoder.decode(inStream, context); return KeyedWorkItems.workItem(key, timers, elems); } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java index 888c11f97313..3607fdd6cb6d 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java @@ -241,9 +241,9 @@ private TimerDataCoder(Coder windowCoder) { public void encode(TimerData timer, OutputStream outStream, Context context) throws CoderException, IOException { Context nestedContext = context.nested(); - STRING_CODER.encode(timer.getTimerId(), outStream, nestedContext); - STRING_CODER.encode(timer.getNamespace().stringKey(), outStream, nestedContext); - INSTANT_CODER.encode(timer.getTimestamp(), outStream, nestedContext); + STRING_CODER.encode(timer.getTimerId(), outStream); + STRING_CODER.encode(timer.getNamespace().stringKey(), outStream); + INSTANT_CODER.encode(timer.getTimestamp(), outStream); STRING_CODER.encode(timer.getDomain().name(), outStream, context); } @@ -251,10 +251,10 @@ public void encode(TimerData timer, OutputStream outStream, Context context) public TimerData decode(InputStream inStream, Context context) throws CoderException, IOException { Context nestedContext = context.nested(); - String timerId = STRING_CODER.decode(inStream, nestedContext); + String timerId = STRING_CODER.decode(inStream); StateNamespace namespace = - StateNamespaces.fromString(STRING_CODER.decode(inStream, nestedContext), windowCoder); - Instant timestamp = INSTANT_CODER.decode(inStream, nestedContext); + StateNamespaces.fromString(STRING_CODER.decode(inStream), windowCoder); + Instant timestamp = INSTANT_CODER.decode(inStream); TimeDomain domain = TimeDomain.valueOf(STRING_CODER.decode(inStream, context)); return TimerData.of(timerId, namespace, timestamp, domain); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java index e210ed9d7b98..e003119aa96f 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java @@ -77,14 +77,14 @@ public int getLength() { @Override public void serialize(T t, DataOutputView dataOutputView) throws IOException { DataOutputViewWrapper outputWrapper = new DataOutputViewWrapper(dataOutputView); - coder.encode(t, outputWrapper, Coder.Context.NESTED); + coder.encode(t, outputWrapper); } @Override public T deserialize(DataInputView dataInputView) throws IOException { try { DataInputViewWrapper inputWrapper = new DataInputViewWrapper(dataInputView); - return coder.decode(inputWrapper, Coder.Context.NESTED); + return coder.decode(inputWrapper); } catch (CoderException e) { Throwable cause = e.getCause(); if (cause instanceof EOFException) { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java index f21869341a01..d7bae7e7b3db 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java @@ -70,14 +70,14 @@ public void encode(SingletonKeyedWorkItem value, OutputStream outStream, Context context) throws CoderException, IOException { - keyCoder.encode(value.key(), outStream, context.nested()); + keyCoder.encode(value.key(), outStream); valueCoder.encode(value.value, outStream, context); } @Override public SingletonKeyedWorkItem decode(InputStream inStream, Context context) throws CoderException, IOException { - K key = keyCoder.decode(inStream, context.nested()); + K key = keyCoder.decode(inStream); WindowedValue value = valueCoder.decode(inStream, context); return new SingletonKeyedWorkItem<>(key, value); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java index d6af4f9be944..8d437d56d573 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java @@ -430,8 +430,8 @@ public void snapshotKeyGroupState(int keyGroupIdx, DataOutputStream out) throws Map map = entry.getValue().f1; out.writeInt(map.size()); for (Map.Entry entry1 : map.entrySet()) { - StringUtf8Coder.of().encode(entry1.getKey(), out, Context.NESTED); - coder.encode(entry1.getValue(), out, Context.NESTED); + StringUtf8Coder.of().encode(entry1.getKey(), out); + coder.encode(entry1.getValue(), out); } } } @@ -463,8 +463,8 @@ public void restoreKeyGroupState(int keyGroupIdx, DataInputStream in, Map map = (Map) tuple2.f1; int mapSize = in.readInt(); for (int j = 0; j < mapSize; j++) { - String namespace = StringUtf8Coder.of().decode(in, Context.NESTED); - Object value = coder.decode(in, Context.NESTED); + String namespace = StringUtf8Coder.of().decode(in); + Object value = coder.decode(in); map.put(namespace, value); } } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java index ecd0365ed88c..0e60fa09dba9 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java @@ -1353,7 +1353,7 @@ public static TransformedMapCoder of( @Override public void encode(TransformedMap value, OutputStream outStream, Coder.Context context) throws CoderException, IOException { - transformCoder.encode(value.transform, outStream, context.nested()); + transformCoder.encode(value.transform, outStream); originalMapCoder.encode(value.originalMap, outStream, context); } @@ -1361,7 +1361,7 @@ public void encode(TransformedMap value, OutputStream outStream, public TransformedMap decode( InputStream inStream, Coder.Context context) throws CoderException, IOException { return new TransformedMap<>( - transformCoder.decode(inStream, context.nested()), + transformCoder.decode(inStream), originalMapCoder.decode(inStream, context)); } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java index 00e0c54c2d5f..0f0cd4d940d7 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java @@ -239,12 +239,12 @@ public void encode(IsmRecord value, OutputStream outStream, keyComponentCoders.size(), value.getKeyComponents())); } for (int i = 0; i < keyComponentCoders.size(); ++i) { - getKeyComponentCoder(i).encode(value.getKeyComponent(i), outStream, context.nested()); + getKeyComponentCoder(i).encode(value.getKeyComponent(i), outStream); } if (isMetadataKey(value.getKeyComponents())) { - ByteArrayCoder.of().encode(value.getMetadata(), outStream, context.nested()); + ByteArrayCoder.of().encode(value.getMetadata(), outStream); } else { - valueCoder.encode(value.getValue(), outStream, context.nested()); + valueCoder.encode(value.getValue(), outStream); } } @@ -253,13 +253,13 @@ public IsmRecord decode(InputStream inStream, Coder.Context context) throws CoderException, IOException { List keyComponents = new ArrayList<>(keyComponentCoders.size()); for (Coder keyCoder : keyComponentCoders) { - keyComponents.add(keyCoder.decode(inStream, context.nested())); + keyComponents.add(keyCoder.decode(inStream)); } if (isMetadataKey(keyComponents)) { return IsmRecord.meta( - keyComponents, ByteArrayCoder.of().decode(inStream, context.nested())); + keyComponents, ByteArrayCoder.of().decode(inStream)); } else { - return IsmRecord.of(keyComponents, valueCoder.decode(inStream, context.nested())); + return IsmRecord.of(keyComponents, valueCoder.decode(inStream)); } } @@ -499,7 +499,7 @@ public void encode(K value, OutputStream outStream, Coder.Context context) outStream.write(0); } else { outStream.write(1); - keyCoder.encode(value, outStream, context.nested()); + keyCoder.encode(value, outStream); } } @@ -510,7 +510,7 @@ public K decode(InputStream inStream, Coder.Context context) if (marker == 0) { return (K) getMetadataKey(); } else if (marker == 1) { - return keyCoder.decode(inStream, context.nested()); + return keyCoder.decode(inStream); } else { throw new CoderException(String.format("Expected marker but got %s.", marker)); } @@ -626,8 +626,8 @@ public void encode(IsmShard value, OutputStream outStream, Coder.Context context checkState(value.getIndexOffset() >= 0, "%s attempting to be written without index offset.", value); - VarIntCoder.of().encode(value.getId(), outStream, context.nested()); - VarLongCoder.of().encode(value.getBlockOffset(), outStream, context.nested()); + VarIntCoder.of().encode(value.getId(), outStream); + VarLongCoder.of().encode(value.getBlockOffset(), outStream); VarLongCoder.of().encode(value.getIndexOffset(), outStream, context); } @@ -635,8 +635,8 @@ public void encode(IsmShard value, OutputStream outStream, Coder.Context context public IsmShard decode( InputStream inStream, Coder.Context context) throws CoderException, IOException { return IsmShard.of( - VarIntCoder.of().decode(inStream, context.nested()), - VarLongCoder.of().decode(inStream, context.nested()), + VarIntCoder.of().decode(inStream), + VarLongCoder.of().decode(inStream), VarLongCoder.of().decode(inStream, context)); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java index c836ca5801b8..27f2ec8897c1 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java @@ -207,7 +207,7 @@ private void writeObject(ObjectOutputStream oos) throws IOException { oos.writeObject(inCoder); try { combineFn.getAccumulatorCoder(ctxt.getCoderRegistry(), inCoder) - .encode(state, oos, Coder.Context.NESTED); + .encode(state, oos); } catch (CannotProvideCoderException e) { throw new IllegalStateException("Could not determine coder for accumulator", e); } @@ -220,7 +220,7 @@ private void readObject(ObjectInputStream ois) throws IOException, ClassNotFound inCoder = (Coder) ois.readObject(); try { state = combineFn.getAccumulatorCoder(ctxt.getCoderRegistry(), inCoder) - .decode(ois, Coder.Context.NESTED); + .decode(ois); } catch (CannotProvideCoderException e) { throw new IllegalStateException("Could not determine coder for accumulator", e); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java index 97559a9abb4f..e2166cf25d58 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java @@ -51,14 +51,14 @@ private BigDecimalCoder() {} public void encode(BigDecimal value, OutputStream outStream, Context context) throws IOException, CoderException { checkNotNull(value, String.format("cannot encode a null %s", BigDecimal.class.getSimpleName())); - VAR_INT_CODER.encode(value.scale(), outStream, context.nested()); + VAR_INT_CODER.encode(value.scale(), outStream); BIG_INT_CODER.encode(value.unscaledValue(), outStream, context); } @Override public BigDecimal decode(InputStream inStream, Context context) throws IOException, CoderException { - int scale = VAR_INT_CODER.decode(inStream, context.nested()); + int scale = VAR_INT_CODER.decode(inStream); BigInteger bigInteger = BIG_INT_CODER.decode(inStream, context); return new BigDecimal(bigInteger, scale); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java index 9994b3f671d7..59d5424d52cc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java @@ -97,7 +97,7 @@ public void encode( Collection collection = (Collection) iterable; dataOutStream.writeInt(collection.size()); for (T elem : collection) { - elementCoder.encode(elem, dataOutStream, nestedContext); + elementCoder.encode(elem, dataOutStream); } } else { // We don't know the size without traversing it so use a fixed size buffer @@ -108,7 +108,7 @@ public void encode( new BufferedElementCountingOutputStream(dataOutStream); for (T elem : iterable) { countingOutputStream.markElementStart(); - elementCoder.encode(elem, countingOutputStream, nestedContext); + elementCoder.encode(elem, countingOutputStream); } countingOutputStream.finish(); } @@ -125,7 +125,7 @@ public IterableT decode(InputStream inStream, Context context) if (size >= 0) { List elements = new ArrayList<>(size); for (int i = 0; i < size; i++) { - elements.add(elementCoder.decode(dataInStream, nestedContext)); + elements.add(elementCoder.decode(dataInStream)); } return decodeToIterable(elements); } @@ -134,7 +134,7 @@ public IterableT decode(InputStream inStream, Context context) // each block of elements. long count = VarInt.decodeLong(dataInStream); while (count > 0L) { - elements.add(elementCoder.decode(dataInStream, nestedContext)); + elements.add(elementCoder.decode(dataInStream)); --count; if (count == 0L) { count = VarInt.decodeLong(dataInStream); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoder.java index 1df446075f83..0bb53ecbe006 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoder.java @@ -63,14 +63,14 @@ public void encode(KV kv, OutputStream outStream, Context context) if (kv == null) { throw new CoderException("cannot encode a null KV"); } - keyCoder.encode(kv.getKey(), outStream, context.nested()); + keyCoder.encode(kv.getKey(), outStream); valueCoder.encode(kv.getValue(), outStream, context); } @Override public KV decode(InputStream inStream, Context context) throws IOException, CoderException { - K key = keyCoder.decode(inStream, context.nested()); + K key = keyCoder.decode(inStream); V value = valueCoder.decode(inStream, context); return KV.of(key, value); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java index 7df9ca96bf38..f20eb93422e0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java @@ -89,12 +89,12 @@ public void encode( Iterator> iterator = map.entrySet().iterator(); Entry entry = iterator.next(); while (iterator.hasNext()) { - keyCoder.encode(entry.getKey(), outStream, context.nested()); - valueCoder.encode(entry.getValue(), outStream, context.nested()); + keyCoder.encode(entry.getKey(), outStream); + valueCoder.encode(entry.getValue(), outStream); entry = iterator.next(); } - keyCoder.encode(entry.getKey(), outStream, context.nested()); + keyCoder.encode(entry.getKey(), outStream); valueCoder.encode(entry.getValue(), outStream, context); // no flush needed as DataOutputStream does not buffer } @@ -110,12 +110,12 @@ public Map decode(InputStream inStream, Context context) Map retval = Maps.newHashMapWithExpectedSize(size); for (int i = 0; i < size - 1; ++i) { - K key = keyCoder.decode(inStream, context.nested()); - V value = valueCoder.decode(inStream, context.nested()); + K key = keyCoder.decode(inStream); + V value = valueCoder.decode(inStream); retval.put(key, value); } - K key = keyCoder.decode(inStream, context.nested()); + K key = keyCoder.decode(inStream); V value = valueCoder.decode(inStream, context); retval.put(key, value); return retval; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java index 20fab9b68b62..d8a98cd2e2c6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java @@ -952,7 +952,7 @@ public void encode(FileResult value, OutputStream outStream, Context context) if (value == null) { throw new CoderException("cannot encode a null value"); } - stringCoder.encode(value.getFilename().toString(), outStream, context.nested()); + stringCoder.encode(value.getFilename().toString(), outStream); if (value.getDestinationFilename() == null) { stringCoder.encode(null, outStream, context); } else { @@ -963,7 +963,7 @@ public void encode(FileResult value, OutputStream outStream, Context context) @Override public FileResult decode(InputStream inStream, Context context) throws IOException { - String filename = stringCoder.decode(inStream, context.nested()); + String filename = stringCoder.decode(inStream); assert filename != null; // fixes a compiler warning @Nullable String destinationFilename = stringCoder.decode(inStream, context); return new FileResult( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java index 37d5a551ccbf..348cc5fca42e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java @@ -682,10 +682,10 @@ public void encode( QuantileState state, OutputStream outStream, Coder.Context context) throws CoderException, IOException { Coder.Context nestedContext = context.nested(); - intCoder.encode(state.numQuantiles, outStream, nestedContext); - intCoder.encode(state.bufferSize, outStream, nestedContext); - elementCoder.encode(state.min, outStream, nestedContext); - elementCoder.encode(state.max, outStream, nestedContext); + intCoder.encode(state.numQuantiles, outStream); + intCoder.encode(state.bufferSize, outStream); + elementCoder.encode(state.min, outStream); + elementCoder.encode(state.max, outStream); elementListCoder.encode( state.unbufferedElements, outStream, nestedContext); BigEndianIntegerCoder.of().encode( @@ -699,14 +699,14 @@ public void encode( public QuantileState decode(InputStream inStream, Coder.Context context) throws CoderException, IOException { Coder.Context nestedContext = context.nested(); - int numQuantiles = intCoder.decode(inStream, nestedContext); - int bufferSize = intCoder.decode(inStream, nestedContext); - T min = elementCoder.decode(inStream, nestedContext); - T max = elementCoder.decode(inStream, nestedContext); + int numQuantiles = intCoder.decode(inStream); + int bufferSize = intCoder.decode(inStream); + T min = elementCoder.decode(inStream); + T max = elementCoder.decode(inStream); List unbufferedElements = - elementListCoder.decode(inStream, nestedContext); + elementListCoder.decode(inStream); int numBuffers = - BigEndianIntegerCoder.of().decode(inStream, nestedContext); + BigEndianIntegerCoder.of().decode(inStream); List> buffers = new ArrayList<>(numBuffers); for (int i = 0; i < numBuffers; i++) { buffers.add(decodeBuffer(inStream, nestedContext)); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java index 0515ed527e50..c45df046f60f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java @@ -543,7 +543,7 @@ public void encode(Object[] value, OutputStream outStream, Context context) int lastIndex = codersCount - 1; Context nestedContext = context.nested(); for (int i = 0; i < lastIndex; ++i) { - coders.get(i).encode(value[i], outStream, nestedContext); + coders.get(i).encode(value[i], outStream); } coders.get(lastIndex).encode(value[lastIndex], outStream, context); } @@ -558,7 +558,7 @@ public Object[] decode(InputStream inStream, Context context) int lastIndex = codersCount - 1; Context nestedContext = context.nested(); for (int i = 0; i < lastIndex; ++i) { - ret[i] = coders.get(i).decode(inStream, nestedContext); + ret[i] = coders.get(i).decode(inStream); } ret[lastIndex] = coders.get(lastIndex).decode(inStream, context); return ret; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java index a309954982f1..a46a21f43ea5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java @@ -187,7 +187,7 @@ static class CountSumCoder extends AtomicCoder value, OutputStream outStream, Coder.Context context) throws CoderException, IOException { - LONG_CODER.encode(value.count, outStream, context.nested()); + LONG_CODER.encode(value.count, outStream); DOUBLE_CODER.encode(value.sum, outStream, context); } @@ -195,7 +195,7 @@ public void encode(CountSum value, OutputStream outStream, Coder.Context c public CountSum decode(InputStream inStream, Coder.Context context) throws CoderException, IOException { return new CountSum<>( - LONG_CODER.decode(inStream, context.nested()), + LONG_CODER.decode(inStream), DOUBLE_CODER.decode(inStream, context)); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java index e9a3571216ff..bd669efb91ae 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java @@ -253,7 +253,7 @@ public void encode( } int lastIndex = schema.size() - 1; for (int unionTag = 0; unionTag < lastIndex; unionTag++) { - tagListCoder(unionTag).encode(value.valueMap.get(unionTag), outStream, context.nested()); + tagListCoder(unionTag).encode(value.valueMap.get(unionTag), outStream); } tagListCoder(lastIndex).encode(value.valueMap.get(lastIndex), outStream, context); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java index 46ece09e0645..cb5a7cf313be 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java @@ -181,14 +181,14 @@ public static IntervalWindowCoder of() { @Override public void encode(IntervalWindow window, OutputStream outStream, Context context) throws IOException, CoderException { - instantCoder.encode(window.end, outStream, context.nested()); + instantCoder.encode(window.end, outStream); durationCoder.encode(new Duration(window.start, window.end), outStream, context); } @Override public IntervalWindow decode(InputStream inStream, Context context) throws IOException, CoderException { - Instant end = instantCoder.decode(inStream, context.nested()); + Instant end = instantCoder.decode(inStream); ReadableDuration duration = durationCoder.decode(inStream, context); return new IntervalWindow(end.minus(duration), end); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java index 1b7e335deacf..e3e61cffb1c5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java @@ -643,8 +643,8 @@ public void encode(WindowedValue windowedElem, Context nestedContext = context.nested(); InstantCoder.of().encode( windowedElem.getTimestamp(), outStream, nestedContext); - windowsCoder.encode(windowedElem.getWindows(), outStream, nestedContext); - PaneInfoCoder.INSTANCE.encode(windowedElem.getPane(), outStream, nestedContext); + windowsCoder.encode(windowedElem.getWindows(), outStream); + PaneInfoCoder.INSTANCE.encode(windowedElem.getPane(), outStream); valueCoder.encode(windowedElem.getValue(), outStream, context); } @@ -652,10 +652,10 @@ public void encode(WindowedValue windowedElem, public WindowedValue decode(InputStream inStream, Context context) throws CoderException, IOException { Context nestedContext = context.nested(); - Instant timestamp = InstantCoder.of().decode(inStream, nestedContext); + Instant timestamp = InstantCoder.of().decode(inStream); Collection windows = - windowsCoder.decode(inStream, nestedContext); - PaneInfo pane = PaneInfoCoder.INSTANCE.decode(inStream, nestedContext); + windowsCoder.decode(inStream); + PaneInfo pane = PaneInfoCoder.INSTANCE.decode(inStream); T value = valueCoder.decode(inStream, context); return WindowedValue.of(value, timestamp, windows, pane); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java index c1728857dfbb..89747a7a94a2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java @@ -104,7 +104,7 @@ public void encode(TimestampedValue windowedElem, OutputStream outStream, Context context) throws IOException { - valueCoder.encode(windowedElem.getValue(), outStream, context.nested()); + valueCoder.encode(windowedElem.getValue(), outStream); InstantCoder.of().encode( windowedElem.getTimestamp(), outStream, context); } @@ -112,7 +112,7 @@ public void encode(TimestampedValue windowedElem, @Override public TimestampedValue decode(InputStream inStream, Context context) throws IOException { - T value = valueCoder.decode(inStream, context.nested()); + T value = valueCoder.decode(inStream); Instant timestamp = InstantCoder.of().decode(inStream, context); return TimestampedValue.of(value, timestamp); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java index 3ecbaa2072ad..e8a2dfdb516d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java @@ -78,18 +78,18 @@ public static Coder of( public void encode(ValueInSingleWindow windowedElem, OutputStream outStream, Context context) throws IOException { Context nestedContext = context.nested(); - InstantCoder.of().encode(windowedElem.getTimestamp(), outStream, nestedContext); - windowCoder.encode(windowedElem.getWindow(), outStream, nestedContext); - PaneInfo.PaneInfoCoder.INSTANCE.encode(windowedElem.getPane(), outStream, nestedContext); + InstantCoder.of().encode(windowedElem.getTimestamp(), outStream); + windowCoder.encode(windowedElem.getWindow(), outStream); + PaneInfo.PaneInfoCoder.INSTANCE.encode(windowedElem.getPane(), outStream); valueCoder.encode(windowedElem.getValue(), outStream, context); } @Override public ValueInSingleWindow decode(InputStream inStream, Context context) throws IOException { Context nestedContext = context.nested(); - Instant timestamp = InstantCoder.of().decode(inStream, nestedContext); - BoundedWindow window = windowCoder.decode(inStream, nestedContext); - PaneInfo pane = PaneInfo.PaneInfoCoder.INSTANCE.decode(inStream, nestedContext); + Instant timestamp = InstantCoder.of().decode(inStream); + BoundedWindow window = windowCoder.decode(inStream); + PaneInfo pane = PaneInfo.PaneInfoCoder.INSTANCE.decode(inStream); T value = valueCoder.decode(inStream, context); return new AutoValue_ValueInSingleWindow<>(value, timestamp, window, pane); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueWithRecordId.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueWithRecordId.java index 3f057e16cf2d..f06317b38571 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueWithRecordId.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueWithRecordId.java @@ -103,7 +103,7 @@ public List> getCoderArguments() { @Override public void encode(ValueWithRecordId value, OutputStream outStream, Context context) throws IOException { - valueCoder.encode(value.value, outStream, context.nested()); + valueCoder.encode(value.value, outStream); idCoder.encode(value.id, outStream, context); } @@ -111,7 +111,7 @@ public void encode(ValueWithRecordId value, OutputStream outStream, Cont public ValueWithRecordId decode(InputStream inStream, Context context) throws IOException { return new ValueWithRecordId( - valueCoder.decode(inStream, context.nested()), + valueCoder.decode(inStream), idCoder.decode(inStream, context)); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java index d97eea6b9ed4..adb665286eee 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java @@ -182,15 +182,15 @@ public void testLongStringEncoding() throws Exception { // Encode both strings into NESTED form. byte[] nestedEncoding; try (ByteArrayOutputStream os = new ByteArrayOutputStream()) { - coder.encode(source, os, Coder.Context.NESTED); - coder.encode(source2, os, Coder.Context.NESTED); + coder.encode(source, os); + coder.encode(source2, os); nestedEncoding = os.toByteArray(); } // Decode from NESTED form. try (ByteArrayInputStream is = new ByteArrayInputStream(nestedEncoding)) { - assertEquals(source, coder.decode(is, Coder.Context.NESTED)); - assertEquals(source2, coder.decode(is, Coder.Context.NESTED)); + assertEquals(source, coder.decode(is)); + assertEquals(source2, coder.decode(is)); assertEquals(0, is.available()); } } @@ -207,20 +207,20 @@ public void testMixedWithNullsEncoding() throws Exception { Coder coder = SerializableCoder.of(String.class); byte[] encodedBytes; try (ByteArrayOutputStream os = new ByteArrayOutputStream()) { - coder.encode(null, os, Coder.Context.NESTED); - coder.encode("TestValue", os, Coder.Context.NESTED); - coder.encode(null, os, Coder.Context.NESTED); - coder.encode("TestValue2", os, Coder.Context.NESTED); - coder.encode(null, os, Coder.Context.NESTED); + coder.encode(null, os); + coder.encode("TestValue", os); + coder.encode(null, os); + coder.encode("TestValue2", os); + coder.encode(null, os); encodedBytes = os.toByteArray(); } try (ByteArrayInputStream is = new ByteArrayInputStream(encodedBytes)) { - assertNull(coder.decode(is, Coder.Context.NESTED)); - assertEquals("TestValue", coder.decode(is, Coder.Context.NESTED)); - assertNull(coder.decode(is, Coder.Context.NESTED)); - assertEquals("TestValue2", coder.decode(is, Coder.Context.NESTED)); - assertNull(coder.decode(is, Coder.Context.NESTED)); + assertNull(coder.decode(is)); + assertEquals("TestValue", coder.decode(is)); + assertNull(coder.decode(is)); + assertEquals("TestValue2", coder.decode(is)); + assertNull(coder.decode(is)); assertEquals(0, is.available()); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java index a70af94f3768..e4b016be10bb 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java @@ -878,14 +878,14 @@ private class CountSumCoder extends AtomicCoder { @Override public void encode(CountSum value, OutputStream outStream, Context context) throws CoderException, IOException { - LONG_CODER.encode(value.count, outStream, context.nested()); + LONG_CODER.encode(value.count, outStream); DOUBLE_CODER.encode(value.sum, outStream, context); } @Override public CountSum decode(InputStream inStream, Coder.Context context) throws CoderException, IOException { - long count = LONG_CODER.decode(inStream, context.nested()); + long count = LONG_CODER.decode(inStream); double sum = DOUBLE_CODER.decode(inStream, context); return new CountSum(count, sum); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java index a4588128604e..7e8a1dd7bfcc 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java @@ -210,14 +210,14 @@ public void encode( OutputStream outStream, org.apache.beam.sdk.coders.Coder.Context context) throws CoderException, IOException { - stringCoder.encode(value.myString, outStream, context.nested()); + stringCoder.encode(value.myString, outStream); } @Override public UnserializableRecord decode( InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context) throws CoderException, IOException { - return new UnserializableRecord(stringCoder.decode(inStream, context.nested())); + return new UnserializableRecord(stringCoder.decode(inStream)); } } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/GlobalWindowTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/GlobalWindowTest.java index 314b96965fcd..9ae5d688292b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/GlobalWindowTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/GlobalWindowTest.java @@ -35,7 +35,7 @@ public void testCoderBinaryRepresentation() throws Exception { CountingOutputStream out = new CountingOutputStream(ByteStreams.nullOutputStream()); GlobalWindow.Coder.INSTANCE.encode(GlobalWindow.INSTANCE, out, Context.OUTER); assertEquals(0, out.getCount()); - GlobalWindow.Coder.INSTANCE.encode(GlobalWindow.INSTANCE, out, Context.NESTED); + GlobalWindow.Coder.INSTANCE.encode(GlobalWindow.INSTANCE, out); assertEquals(0, out.getCount()); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java index 36f7028627ee..894d8a98021b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java @@ -32,7 +32,6 @@ import java.util.List; import java.util.Random; import org.apache.beam.sdk.coders.ByteArrayCoder; -import org.apache.beam.sdk.coders.Coder.Context; import org.hamcrest.collection.IsIterableContainingInOrder; import org.junit.Rule; import org.junit.Test; @@ -180,7 +179,7 @@ private void verifyValues(List expectedValues, InputStream is) throws Ex do { count = VarInt.decodeLong(is); for (int i = 0; i < count; ++i) { - values.add(ByteArrayCoder.of().decode(is, Context.NESTED)); + values.add(ByteArrayCoder.of().decode(is)); } } while(count > 0); @@ -198,7 +197,7 @@ private void verifyValues(List expectedValues, InputStream is) throws Ex for (byte[] value : values) { os.markElementStart(); - ByteArrayCoder.of().encode(value, os, Context.NESTED); + ByteArrayCoder.of().encode(value, os); } return os; } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserver.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserver.java index 18e0d9509da3..37745be9cb9f 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserver.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserver.java @@ -110,7 +110,7 @@ public void close() throws Exception { @Override public void accept(WindowedValue t) throws IOException { - coder.encode(t, bufferedElements, Context.NESTED); + coder.encode(t, bufferedElements); counter += 1; if (bufferedElements.size() >= bufferLimit) { outboundObserver.onNext(convertBufferForTransmission().build()); diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserver.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserver.java index 24365d8866f8..ece87d26c8e5 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserver.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserver.java @@ -71,7 +71,7 @@ public void accept(BeamFnApi.Elements.Data t) { InputStream inputStream = t.getData().newInput(); while (inputStream.available() > 0) { counter += 1; - WindowedValue value = coder.decode(inputStream, Context.NESTED); + WindowedValue value = coder.decode(inputStream); consumer.accept(value); } } catch (Exception e) { diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserverTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserverTest.java index 7cbf8ebe66b0..3f6ece71063b 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserverTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserverTest.java @@ -135,7 +135,7 @@ public void testExperimentConfiguresBufferLimit() throws Exception { private static BeamFnApi.Elements messageWithData(byte[] ... datum) throws IOException { ByteString.Output output = ByteString.newOutput(); for (byte[] data : datum) { - CODER.encode(valueInGlobalWindow(data), output, Context.NESTED); + CODER.encode(valueInGlobalWindow(data), output); } return BeamFnApi.Elements.newBuilder() .addData(BeamFnApi.Elements.Data.newBuilder() diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserverTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserverTest.java index c53f99d49fc1..4b0bf0ce932b 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserverTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserverTest.java @@ -108,7 +108,7 @@ private BeamFnApi.Elements.Data dataWith(String ... values) throws Exception { .setName("Test")); ByteString.Output output = ByteString.newOutput(); for (String value : values) { - CODER.encode(valueInGlobalWindow(value), output, Context.NESTED); + CODER.encode(valueInGlobalWindow(value), output); } builder.setData(output.toByteString()); return builder.build(); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ShardedKeyCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ShardedKeyCoder.java index 7aefcfa606c0..a06aacec590f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ShardedKeyCoder.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ShardedKeyCoder.java @@ -53,17 +53,17 @@ public List> getCoderArguments() { } @Override - public void encode(ShardedKey key, OutputStream outStream, Context context) + public void encode(ShardedKey key, OutputStream outStream) throws IOException { - keyCoder.encode(key.getKey(), outStream, context.nested()); - shardNumberCoder.encode(key.getShardNumber(), outStream, context); + keyCoder.encode(key.getKey(), outStream); + shardNumberCoder.encode(key.getShardNumber(), outStream); } @Override - public ShardedKey decode(InputStream inStream, Context context) + public ShardedKey decode(InputStream inStream) throws IOException { return new ShardedKey<>( - keyCoder.decode(inStream, context.nested()), shardNumberCoder.decode(inStream, context)); + keyCoder.decode(inStream), shardNumberCoder.decode(inStream)); } @Override diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java index 01bc558214fa..33b9f779ab9d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java @@ -43,14 +43,14 @@ public void encode(TableDestination value, OutputStream outStream, Context conte if (value == null) { throw new CoderException("cannot encode a null value"); } - tableSpecCoder.encode(value.getTableSpec(), outStream, context.nested()); - tableDescriptionCoder.encode(value.getTableDescription(), outStream, context); + tableSpecCoder.encode(value.getTableSpec(), outStream); + tableDescriptionCoder.encode(value.getTableDescription(), outStream); } @Override - public TableDestination decode(InputStream inStream, Context context) throws IOException { - String tableSpec = tableSpecCoder.decode(inStream, context.nested()); - String tableDescription = tableDescriptionCoder.decode(inStream, context); + public TableDestination decode(InputStream inStream) throws IOException { + String tableSpec = tableSpecCoder.decode(inStream); + String tableDescription = tableDescriptionCoder.decode(inStream); return new TableDestination(tableSpec, tableDescription); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowInfoCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowInfoCoder.java index 2b1988a2bda7..8ae75c56581e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowInfoCoder.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowInfoCoder.java @@ -43,7 +43,7 @@ public void encode(TableRowInfo value, OutputStream outStream, Context context) if (value == null) { throw new CoderException("cannot encode a null value"); } - tableRowCoder.encode(value.tableRow, outStream, context.nested()); + tableRowCoder.encode(value.tableRow, outStream); idCoder.encode(value.uniqueId, outStream, context); } @@ -51,7 +51,7 @@ public void encode(TableRowInfo value, OutputStream outStream, Context context) public TableRowInfo decode(InputStream inStream, Context context) throws IOException { return new TableRowInfo( - tableRowCoder.decode(inStream, context.nested()), + tableRowCoder.decode(inStream), idCoder.decode(inStream, context)); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java index 890979ba2c8d..9e83271a5865 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java @@ -106,16 +106,16 @@ public void encode(Result value, OutputStream outStream, Context c if (value == null) { throw new CoderException("cannot encode a null value"); } - stringCoder.encode(value.filename, outStream, context.nested()); - longCoder.encode(value.fileByteSize, outStream, context.nested()); - destinationCoder.encode(value.destination, outStream, context.nested()); + stringCoder.encode(value.filename, outStream); + longCoder.encode(value.fileByteSize, outStream); + destinationCoder.encode(value.destination, outStream); } @Override public Result decode(InputStream inStream, Context context) throws IOException { - String filename = stringCoder.decode(inStream, context.nested()); - long fileByteSize = longCoder.decode(inStream, context.nested()); - DestinationT destination = destinationCoder.decode(inStream, context.nested()); + String filename = stringCoder.decode(inStream); + long fileByteSize = longCoder.decode(inStream); + DestinationT destination = destinationCoder.decode(inStream); return new Result<>(filename, fileByteSize, destination); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithAttributesCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithAttributesCoder.java index e061edc50106..5907c9e7a214 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithAttributesCoder.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithAttributesCoder.java @@ -47,13 +47,13 @@ public static PubsubMessageWithAttributesCoder of() { public void encode(PubsubMessage value, OutputStream outStream, Context context) throws IOException { - PAYLOAD_CODER.encode(value.getPayload(), outStream, context.nested()); + PAYLOAD_CODER.encode(value.getPayload(), outStream); ATTRIBUTES_CODER.encode(value.getAttributeMap(), outStream, context); } @Override public PubsubMessage decode(InputStream inStream, Context context) throws IOException { - byte[] payload = PAYLOAD_CODER.decode(inStream, context.nested()); + byte[] payload = PAYLOAD_CODER.decode(inStream); Map attributes = ATTRIBUTES_CODER.decode(inStream, context); return new PubsubMessage(payload, attributes); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java index 9f04a6c3fe07..ae320c73a05b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java @@ -110,19 +110,19 @@ private static class OutgoingMessageCoder extends AtomicCoder { public void encode( OutgoingMessage value, OutputStream outStream, Context context) throws CoderException, IOException { - ByteArrayCoder.of().encode(value.elementBytes, outStream, context.nested()); - ATTRIBUTES_CODER.encode(value.attributes, outStream, context.nested()); - BigEndianLongCoder.of().encode(value.timestampMsSinceEpoch, outStream, context.nested()); - RECORD_ID_CODER.encode(value.recordId, outStream, context.nested()); + ByteArrayCoder.of().encode(value.elementBytes, outStream); + ATTRIBUTES_CODER.encode(value.attributes, outStream); + BigEndianLongCoder.of().encode(value.timestampMsSinceEpoch, outStream); + RECORD_ID_CODER.encode(value.recordId, outStream); } @Override public OutgoingMessage decode( InputStream inStream, Context context) throws CoderException, IOException { - byte[] elementBytes = ByteArrayCoder.of().decode(inStream, context.nested()); - Map attributes = ATTRIBUTES_CODER.decode(inStream, context.nested()); - long timestampMsSinceEpoch = BigEndianLongCoder.of().decode(inStream, context.nested()); - @Nullable String recordId = RECORD_ID_CODER.decode(inStream, context.nested()); + byte[] elementBytes = ByteArrayCoder.of().decode(inStream); + Map attributes = ATTRIBUTES_CODER.decode(inStream); + long timestampMsSinceEpoch = BigEndianLongCoder.of().decode(inStream); + @Nullable String recordId = RECORD_ID_CODER.decode(inStream); return new OutgoingMessage(elementBytes, attributes, timestampMsSinceEpoch, recordId); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java index c16b8fb70eb2..e53976e5ffb8 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java @@ -380,7 +380,7 @@ public void encode(PubsubCheckpoint value, OutputStream outStream, Context conte @Override public PubsubCheckpoint decode(InputStream inStream, Context context) throws IOException { - String path = SUBSCRIPTION_PATH_CODER.decode(inStream, context.nested()); + String path = SUBSCRIPTION_PATH_CODER.decode(inStream); List notYetReadIds = LIST_CODER.decode(inStream, context); return new PubsubCheckpoint(path, null, null, notYetReadIds); } diff --git a/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/JAXBCoderTest.java b/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/JAXBCoderTest.java index 2b4503ac5bca..5386a613fcbc 100644 --- a/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/JAXBCoderTest.java +++ b/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/JAXBCoderTest.java @@ -181,8 +181,8 @@ public TestCoder(JAXBCoder jaxbCoder) { public void encode(TestType value, OutputStream outStream, Context context) throws CoderException, IOException { Context nestedContext = context.nested(); - VarIntCoder.of().encode(3, outStream, nestedContext); - jaxbCoder.encode(value, outStream, nestedContext); + VarIntCoder.of().encode(3, outStream); + jaxbCoder.encode(value, outStream); VarLongCoder.of().encode(22L, outStream, context); } @@ -190,8 +190,8 @@ public void encode(TestType value, OutputStream outStream, Context context) public TestType decode(InputStream inStream, Context context) throws CoderException, IOException { Context nestedContext = context.nested(); - VarIntCoder.of().decode(inStream, nestedContext); - TestType result = jaxbCoder.decode(inStream, nestedContext); + VarIntCoder.of().decode(inStream); + TestType result = jaxbCoder.decode(inStream); VarLongCoder.of().decode(inStream, context); return result; } From dcacad1d28a82d0bded9c3b7e24a25faa30b2f4d Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 5 May 2017 16:36:47 -0700 Subject: [PATCH 02/11] Remove contexts from coders where they'll never be used. --- .../UnboundedReadFromBoundedSource.java | 4 +-- .../core/ElementAndRestrictionCoder.java | 4 +-- .../beam/runners/core/KeyedWorkItemCoder.java | 4 +-- .../beam/runners/core/TimerInternals.java | 4 +-- .../apache/beam/sdk/coders/DurationCoder.java | 4 +-- .../apache/beam/sdk/coders/InstantCoder.java | 4 +-- .../sdk/transforms/ApproximateQuantiles.java | 20 +++++------ .../org/apache/beam/sdk/transforms/Mean.java | 4 +-- .../org/apache/beam/sdk/transforms/Top.java | 4 +-- .../beam/sdk/transforms/join/CoGbkResult.java | 10 ++---- .../transforms/windowing/IntervalWindow.java | 4 +-- .../beam/sdk/values/TimestampedValue.java | 4 +-- .../sdk/io/kinesis/KinesisRecordCoder.java | 34 +++++++++---------- 13 files changed, 47 insertions(+), 57 deletions(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java index ae28e3adbd3e..b74da808da8d 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java @@ -224,7 +224,7 @@ static class CheckpointCoder extends StructuredCoder> { public void encode(Checkpoint value, OutputStream outStream, Context context) throws CoderException, IOException { elemsCoder.encode(value.residualElements, outStream); - sourceCoder.encode(value.residualSource, outStream, context); + sourceCoder.encode(value.residualSource, outStream); } @SuppressWarnings("unchecked") @@ -233,7 +233,7 @@ public Checkpoint decode(InputStream inStream, Context context) throws CoderException, IOException { return new Checkpoint<>( elemsCoder.decode(inStream), - sourceCoder.decode(inStream, context)); + sourceCoder.decode(inStream)); } @Override diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java index 5ddd865a518f..fcb1debb3300 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java @@ -56,14 +56,14 @@ public void encode( throw new CoderException("cannot encode a null ElementAndRestriction"); } elementCoder.encode(value.element(), outStream); - restrictionCoder.encode(value.restriction(), outStream, context); + restrictionCoder.encode(value.restriction(), outStream); } @Override public ElementAndRestriction decode(InputStream inStream, Context context) throws IOException { ElementT key = elementCoder.decode(inStream); - RestrictionT value = restrictionCoder.decode(inStream, context); + RestrictionT value = restrictionCoder.decode(inStream); return ElementAndRestriction.of(key, value); } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java index ac8a34cfb9b0..0869244c8e4d 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java @@ -74,7 +74,7 @@ public void encode(KeyedWorkItem value, OutputStream outStream, Coder. Coder.Context nestedContext = context.nested(); keyCoder.encode(value.key(), outStream); timersCoder.encode(value.timersIterable(), outStream); - elemsCoder.encode(value.elementsIterable(), outStream, context); + elemsCoder.encode(value.elementsIterable(), outStream); } @Override @@ -83,7 +83,7 @@ public KeyedWorkItem decode(InputStream inStream, Coder.Context contex Coder.Context nestedContext = context.nested(); K key = keyCoder.decode(inStream); Iterable timers = timersCoder.decode(inStream); - Iterable> elems = elemsCoder.decode(inStream, context); + Iterable> elems = elemsCoder.decode(inStream); return KeyedWorkItems.workItem(key, timers, elems); } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java index 3607fdd6cb6d..f0a62cd6973a 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java @@ -244,7 +244,7 @@ public void encode(TimerData timer, OutputStream outStream, Context context) STRING_CODER.encode(timer.getTimerId(), outStream); STRING_CODER.encode(timer.getNamespace().stringKey(), outStream); INSTANT_CODER.encode(timer.getTimestamp(), outStream); - STRING_CODER.encode(timer.getDomain().name(), outStream, context); + STRING_CODER.encode(timer.getDomain().name(), outStream); } @Override @@ -255,7 +255,7 @@ public TimerData decode(InputStream inStream, Context context) StateNamespace namespace = StateNamespaces.fromString(STRING_CODER.decode(inStream), windowCoder); Instant timestamp = INSTANT_CODER.decode(inStream); - TimeDomain domain = TimeDomain.valueOf(STRING_CODER.decode(inStream, context)); + TimeDomain domain = TimeDomain.valueOf(STRING_CODER.decode(inStream)); return TimerData.of(timerId, namespace, timestamp, domain); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DurationCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DurationCoder.java index 8b4ae1dde31e..b7db3057c08f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DurationCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DurationCoder.java @@ -59,13 +59,13 @@ public void encode(ReadableDuration value, OutputStream outStream, Context conte if (value == null) { throw new CoderException("cannot encode a null ReadableDuration"); } - LONG_CODER.encode(toLong(value), outStream, context); + LONG_CODER.encode(toLong(value), outStream); } @Override public ReadableDuration decode(InputStream inStream, Context context) throws CoderException, IOException { - return fromLong(LONG_CODER.decode(inStream, context)); + return fromLong(LONG_CODER.decode(inStream)); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java index 000f40672f10..22b11a3ed08e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java @@ -73,13 +73,13 @@ public void encode(Instant value, OutputStream outStream, Context context) if (value == null) { throw new CoderException("cannot encode a null Instant"); } - LONG_CODER.encode(ORDER_PRESERVING_CONVERTER.convert(value), outStream, context); + LONG_CODER.encode(ORDER_PRESERVING_CONVERTER.convert(value), outStream); } @Override public Instant decode(InputStream inStream, Context context) throws CoderException, IOException { - return ORDER_PRESERVING_CONVERTER.reverse().convert(LONG_CODER.decode(inStream, context)); + return ORDER_PRESERVING_CONVERTER.reverse().convert(LONG_CODER.decode(inStream)); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java index 348cc5fca42e..9b9d3f8a106f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java @@ -681,24 +681,22 @@ public QuantileStateCoder(ComparatorT compareFn, Coder elementCoder) { public void encode( QuantileState state, OutputStream outStream, Coder.Context context) throws CoderException, IOException { - Coder.Context nestedContext = context.nested(); intCoder.encode(state.numQuantiles, outStream); intCoder.encode(state.bufferSize, outStream); elementCoder.encode(state.min, outStream); elementCoder.encode(state.max, outStream); elementListCoder.encode( - state.unbufferedElements, outStream, nestedContext); + state.unbufferedElements, outStream); BigEndianIntegerCoder.of().encode( - state.buffers.size(), outStream, nestedContext); + state.buffers.size(), outStream); for (QuantileBuffer buffer : state.buffers) { - encodeBuffer(buffer, outStream, nestedContext); + encodeBuffer(buffer, outStream); } } @Override public QuantileState decode(InputStream inStream, Coder.Context context) throws CoderException, IOException { - Coder.Context nestedContext = context.nested(); int numQuantiles = intCoder.decode(inStream); int bufferSize = intCoder.decode(inStream); T min = elementCoder.decode(inStream); @@ -709,29 +707,27 @@ public QuantileState decode(InputStream inStream, Coder.Context BigEndianIntegerCoder.of().decode(inStream); List> buffers = new ArrayList<>(numBuffers); for (int i = 0; i < numBuffers; i++) { - buffers.add(decodeBuffer(inStream, nestedContext)); + buffers.add(decodeBuffer(inStream)); } return new QuantileState( compareFn, numQuantiles, min, max, numBuffers, bufferSize, unbufferedElements, buffers); } - private void encodeBuffer( - QuantileBuffer buffer, OutputStream outStream, Coder.Context context) + private void encodeBuffer(QuantileBuffer buffer, OutputStream outStream) throws CoderException, IOException { DataOutputStream outData = new DataOutputStream(outStream); outData.writeInt(buffer.level); outData.writeLong(buffer.weight); - elementListCoder.encode(buffer.elements, outStream, context); + elementListCoder.encode(buffer.elements, outStream); } - private QuantileBuffer decodeBuffer( - InputStream inStream, Coder.Context context) + private QuantileBuffer decodeBuffer(InputStream inStream) throws IOException, CoderException { DataInputStream inData = new DataInputStream(inStream); return new QuantileBuffer<>( inData.readInt(), inData.readLong(), - elementListCoder.decode(inStream, context)); + elementListCoder.decode(inStream)); } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java index a46a21f43ea5..c8e0d954d174 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java @@ -188,7 +188,7 @@ static class CountSumCoder extends AtomicCoder value, OutputStream outStream, Coder.Context context) throws CoderException, IOException { LONG_CODER.encode(value.count, outStream); - DOUBLE_CODER.encode(value.sum, outStream, context); + DOUBLE_CODER.encode(value.sum, outStream); } @Override @@ -196,7 +196,7 @@ public CountSum decode(InputStream inStream, Coder.Context context) throws CoderException, IOException { return new CountSum<>( LONG_CODER.decode(inStream), - DOUBLE_CODER.decode(inStream, context)); + DOUBLE_CODER.decode(inStream)); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java index c0381a71ee5d..7aec66730288 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java @@ -541,13 +541,13 @@ public BoundedHeapCoder(int maximumSize, ComparatorT compareFn, Coder element public void encode( BoundedHeap value, OutputStream outStream, Context context) throws CoderException, IOException { - listCoder.encode(value.asList(), outStream, context); + listCoder.encode(value.asList(), outStream); } @Override public BoundedHeap decode(InputStream inStream, Coder.Context context) throws CoderException, IOException { - return new BoundedHeap<>(maximumSize, compareFn, listCoder.decode(inStream, context)); + return new BoundedHeap<>(maximumSize, compareFn, listCoder.decode(inStream)); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java index bd669efb91ae..66033255184d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java @@ -251,11 +251,9 @@ public void encode( if (schema.size() == 0) { return; } - int lastIndex = schema.size() - 1; - for (int unionTag = 0; unionTag < lastIndex; unionTag++) { + for (int unionTag = 0; unionTag < schema.size(); unionTag++) { tagListCoder(unionTag).encode(value.valueMap.get(unionTag), outStream); } - tagListCoder(lastIndex).encode(value.valueMap.get(lastIndex), outStream, context); } @Override @@ -266,12 +264,10 @@ public CoGbkResult decode( if (schema.size() == 0) { return new CoGbkResult(schema, ImmutableList.>of()); } - int lastIndex = schema.size() - 1; List> valueMap = Lists.newArrayListWithExpectedSize(schema.size()); - for (int unionTag = 0; unionTag < lastIndex; unionTag++) { - valueMap.add(tagListCoder(unionTag).decode(inStream, context.nested())); + for (int unionTag = 0; unionTag < schema.size(); unionTag++) { + valueMap.add(tagListCoder(unionTag).decode(inStream, Coder.Context.NESTED)); } - valueMap.add(tagListCoder(lastIndex).decode(inStream, context)); return new CoGbkResult(schema, valueMap); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java index cb5a7cf313be..318dc4ca97bb 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java @@ -182,14 +182,14 @@ public static IntervalWindowCoder of() { public void encode(IntervalWindow window, OutputStream outStream, Context context) throws IOException, CoderException { instantCoder.encode(window.end, outStream); - durationCoder.encode(new Duration(window.start, window.end), outStream, context); + durationCoder.encode(new Duration(window.start, window.end), outStream); } @Override public IntervalWindow decode(InputStream inStream, Context context) throws IOException, CoderException { Instant end = instantCoder.decode(inStream); - ReadableDuration duration = durationCoder.decode(inStream, context); + ReadableDuration duration = durationCoder.decode(inStream); return new IntervalWindow(end.minus(duration), end); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java index 89747a7a94a2..95a3152d874e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java @@ -106,14 +106,14 @@ public void encode(TimestampedValue windowedElem, throws IOException { valueCoder.encode(windowedElem.getValue(), outStream); InstantCoder.of().encode( - windowedElem.getTimestamp(), outStream, context); + windowedElem.getTimestamp(), outStream); } @Override public TimestampedValue decode(InputStream inStream, Context context) throws IOException { T value = valueCoder.decode(inStream); - Instant timestamp = InstantCoder.of().decode(inStream, context); + Instant timestamp = InstantCoder.of().decode(inStream); return TimestampedValue.of(value, timestamp); } diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java index 77fe127bcba6..c6a017461ad4 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java @@ -45,28 +45,26 @@ public static KinesisRecordCoder of() { @Override public void encode(KinesisRecord value, OutputStream outStream, Context context) throws IOException { - Context nested = context.nested(); - BYTE_ARRAY_CODER.encode(value.getData().array(), outStream, nested); - STRING_CODER.encode(value.getSequenceNumber(), outStream, nested); - STRING_CODER.encode(value.getPartitionKey(), outStream, nested); - INSTANT_CODER.encode(value.getApproximateArrivalTimestamp(), outStream, nested); - VAR_LONG_CODER.encode(value.getSubSequenceNumber(), outStream, nested); - INSTANT_CODER.encode(value.getReadTime(), outStream, nested); - STRING_CODER.encode(value.getStreamName(), outStream, nested); - STRING_CODER.encode(value.getShardId(), outStream, context); + BYTE_ARRAY_CODER.encode(value.getData().array(), outStream); + STRING_CODER.encode(value.getSequenceNumber(), outStream); + STRING_CODER.encode(value.getPartitionKey(), outStream); + INSTANT_CODER.encode(value.getApproximateArrivalTimestamp(), outStream); + VAR_LONG_CODER.encode(value.getSubSequenceNumber(), outStream); + INSTANT_CODER.encode(value.getReadTime(), outStream); + STRING_CODER.encode(value.getStreamName(), outStream); + STRING_CODER.encode(value.getShardId(), outStream); } @Override public KinesisRecord decode(InputStream inStream, Context context) throws IOException { - Context nested = context.nested(); - ByteBuffer data = ByteBuffer.wrap(BYTE_ARRAY_CODER.decode(inStream, nested)); - String sequenceNumber = STRING_CODER.decode(inStream, nested); - String partitionKey = STRING_CODER.decode(inStream, nested); - Instant approximateArrivalTimestamp = INSTANT_CODER.decode(inStream, nested); - long subSequenceNumber = VAR_LONG_CODER.decode(inStream, nested); - Instant readTimestamp = INSTANT_CODER.decode(inStream, nested); - String streamName = STRING_CODER.decode(inStream, nested); - String shardId = STRING_CODER.decode(inStream, context); + ByteBuffer data = ByteBuffer.wrap(BYTE_ARRAY_CODER.decode(inStream)); + String sequenceNumber = STRING_CODER.decode(inStream); + String partitionKey = STRING_CODER.decode(inStream); + Instant approximateArrivalTimestamp = INSTANT_CODER.decode(inStream); + long subSequenceNumber = VAR_LONG_CODER.decode(inStream); + Instant readTimestamp = INSTANT_CODER.decode(inStream); + String streamName = STRING_CODER.decode(inStream); + String shardId = STRING_CODER.decode(inStream); return new KinesisRecord(data, sequenceNumber, subSequenceNumber, partitionKey, approximateArrivalTimestamp, readTimestamp, streamName, shardId ); From 8097dc0dfb946a83acdd3d70d5c818e06555aba3 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 5 May 2017 17:24:02 -0700 Subject: [PATCH 03/11] automated context removal or redirection --- .../translation/utils/ApexStreamTuple.java | 11 ++++++++ .../UnboundedReadFromBoundedSource.java | 4 +-- .../runners/core/construction/CodersTest.java | 4 +-- .../core/ElementAndRestrictionCoder.java | 4 +-- .../beam/runners/core/KeyedWorkItemCoder.java | 6 ++-- .../beam/runners/core/TimerInternals.java | 6 ++-- .../direct/CloningBundleFactoryTest.java | 20 ++++++------- .../UnboundedReadEvaluatorFactoryTest.java | 5 ++-- .../SingletonKeyedWorkItemCoder.java | 11 ++++++++ .../runners/dataflow/BatchViewOverrides.java | 11 ++++++++ .../runners/dataflow/internal/IsmFormat.java | 28 +++++++++++++------ .../dataflow/util/RandomAccessData.java | 11 ++++++++ .../org/apache/beam/sdk/coders/AvroCoder.java | 4 +-- .../beam/sdk/coders/BigDecimalCoder.java | 11 ++++++++ .../sdk/coders/BigEndianIntegerCoder.java | 4 +-- .../beam/sdk/coders/BigEndianLongCoder.java | 4 +-- .../beam/sdk/coders/BigIntegerCoder.java | 11 ++++++++ .../apache/beam/sdk/coders/BitSetCoder.java | 11 ++++++++ .../beam/sdk/coders/ByteArrayCoder.java | 11 ++++++++ .../org/apache/beam/sdk/coders/ByteCoder.java | 4 +-- .../apache/beam/sdk/coders/DelegateCoder.java | 11 ++++++++ .../apache/beam/sdk/coders/DoubleCoder.java | 4 +-- .../apache/beam/sdk/coders/DurationCoder.java | 4 +-- .../apache/beam/sdk/coders/InstantCoder.java | 4 +-- .../beam/sdk/coders/IterableLikeCoder.java | 6 ++-- .../org/apache/beam/sdk/coders/KvCoder.java | 11 ++++++++ .../beam/sdk/coders/LengthPrefixCoder.java | 4 +-- .../org/apache/beam/sdk/coders/MapCoder.java | 11 ++++++++ .../apache/beam/sdk/coders/NullableCoder.java | 11 ++++++++ .../beam/sdk/coders/SerializableCoder.java | 4 +-- .../beam/sdk/coders/StringDelegateCoder.java | 11 ++++++++ .../beam/sdk/coders/StringUtf8Coder.java | 11 ++++++++ .../beam/sdk/coders/TextualIntegerCoder.java | 11 ++++++++ .../apache/beam/sdk/coders/VarIntCoder.java | 4 +-- .../apache/beam/sdk/coders/VarLongCoder.java | 4 +-- .../org/apache/beam/sdk/coders/VoidCoder.java | 4 +-- .../org/apache/beam/sdk/io/FileBasedSink.java | 11 ++++++++ .../sdk/transforms/ApproximateQuantiles.java | 4 +-- .../apache/beam/sdk/transforms/Combine.java | 22 +++++++++++++++ .../beam/sdk/transforms/CombineFns.java | 13 +++++++-- .../org/apache/beam/sdk/transforms/Count.java | 4 +-- .../org/apache/beam/sdk/transforms/Mean.java | 4 +-- .../org/apache/beam/sdk/transforms/Top.java | 4 +-- .../beam/sdk/transforms/join/CoGbkResult.java | 6 ++-- .../beam/sdk/transforms/join/UnionCoder.java | 11 ++++++++ .../transforms/windowing/GlobalWindow.java | 4 +-- .../transforms/windowing/IntervalWindow.java | 4 +-- .../sdk/transforms/windowing/PaneInfo.java | 4 +-- .../org/apache/beam/sdk/util/BitSetCoder.java | 11 ++++++++ .../apache/beam/sdk/util/WindowedValue.java | 24 ++++++++++++++-- .../beam/sdk/values/TimestampedValue.java | 5 ++-- .../beam/sdk/values/ValueInSingleWindow.java | 13 +++++++-- .../beam/sdk/values/ValueWithRecordId.java | 11 ++++++++ .../beam/sdk/coders/CoderRegistryTest.java | 8 +++--- .../beam/sdk/coders/CustomCoderTest.java | 4 +-- .../beam/sdk/coders/NullableCoderTest.java | 11 ++++++++ .../beam/sdk/coders/StructuredCoderTest.java | 12 ++++---- .../apache/beam/sdk/testing/PAssertTest.java | 4 +-- .../sdk/testing/SerializableMatchersTest.java | 4 +-- .../beam/sdk/testing/WindowSupplierTest.java | 4 +-- .../beam/sdk/transforms/CombineFnsTest.java | 11 ++++++++ .../beam/sdk/transforms/CombineTest.java | 22 +++++++++++++++ .../beam/sdk/transforms/CreateTest.java | 9 +++--- .../beam/sdk/transforms/GroupByKeyTest.java | 4 +-- .../apache/beam/sdk/transforms/ParDoTest.java | 15 ++++++++-- .../apache/beam/sdk/transforms/ViewTest.java | 11 ++++++++ .../transforms/reflect/DoFnInvokersTest.java | 8 +++--- .../apache/beam/sdk/util/CoderUtilsTest.java | 4 +-- .../beam/sdk/util/SerializableUtilsTest.java | 4 +-- .../extensions/protobuf/ByteStringCoder.java | 11 ++++++++ .../sdk/extensions/protobuf/ProtoCoder.java | 11 ++++++++ .../gcp/bigquery/TableDestinationCoder.java | 2 +- .../io/gcp/bigquery/TableRowInfoCoder.java | 11 ++++++++ .../io/gcp/bigquery/TableRowJsonCoder.java | 11 ++++++++ .../io/gcp/bigquery/WriteBundlesToFiles.java | 4 +-- .../pubsub/PubsubMessagePayloadOnlyCoder.java | 11 ++++++++ .../PubsubMessageWithAttributesCoder.java | 11 ++++++++ .../io/gcp/pubsub/PubsubUnboundedSink.java | 4 +-- .../io/gcp/pubsub/PubsubUnboundedSource.java | 11 ++++++++ .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 11 ++++++++ .../beam/sdk/io/hadoop/WritableCoder.java | 4 +-- .../beam/sdk/io/hbase/HBaseMutationCoder.java | 6 ++-- .../beam/sdk/io/hbase/HBaseResultCoder.java | 4 +-- .../org/apache/beam/sdk/io/kafka/KafkaIO.java | 4 +-- .../beam/sdk/io/kafka/KafkaRecordCoder.java | 11 ++++++++ .../sdk/io/kinesis/KinesisRecordCoder.java | 4 +-- .../org/apache/beam/sdk/io/xml/JAXBCoder.java | 25 ++++++++++------- .../apache/beam/sdk/io/xml/JAXBCoderTest.java | 13 +++++++-- 88 files changed, 598 insertions(+), 157 deletions(-) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStreamTuple.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStreamTuple.java index 4aa6ee82017e..1d402eb450fc 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStreamTuple.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStreamTuple.java @@ -161,6 +161,12 @@ protected ApexStreamTupleCoder(Coder valueCoder) { this.valueCoder = checkNotNull(valueCoder); } + @Override + public void encode(ApexStreamTuple value, OutputStream outStream) + throws CoderException, IOException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(ApexStreamTuple value, OutputStream outStream, Context context) throws CoderException, IOException { @@ -174,6 +180,11 @@ public void encode(ApexStreamTuple value, OutputStream outStream, Context con } } + @Override + public ApexStreamTuple decode(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Context.NESTED); + } + @Override public ApexStreamTuple decode(InputStream inStream, Context context) throws CoderException, IOException { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java index b74da808da8d..24eb38479268 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java @@ -221,7 +221,7 @@ static class CheckpointCoder extends StructuredCoder> { } @Override - public void encode(Checkpoint value, OutputStream outStream, Context context) + public void encode(Checkpoint value, OutputStream outStream) throws CoderException, IOException { elemsCoder.encode(value.residualElements, outStream); sourceCoder.encode(value.residualSource, outStream); @@ -229,7 +229,7 @@ public void encode(Checkpoint value, OutputStream outStream, Context context) @SuppressWarnings("unchecked") @Override - public Checkpoint decode(InputStream inStream, Context context) + public Checkpoint decode(InputStream inStream) throws CoderException, IOException { return new Checkpoint<>( elemsCoder.decode(inStream), diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CodersTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CodersTest.java index 765723c9bb3d..42fba7cda508 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CodersTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CodersTest.java @@ -151,11 +151,11 @@ static class Record implements Serializable {} private static class RecordCoder extends AtomicCoder { @Override - public void encode(Record value, OutputStream outStream, Context context) + public void encode(Record value, OutputStream outStream) throws CoderException, IOException {} @Override - public Record decode(InputStream inStream, Context context) + public Record decode(InputStream inStream) throws CoderException, IOException { return new Record(); } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java index fcb1debb3300..4440b856318b 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java @@ -50,7 +50,7 @@ private ElementAndRestrictionCoder( @Override public void encode( - ElementAndRestriction value, OutputStream outStream, Context context) + ElementAndRestriction value, OutputStream outStream) throws IOException { if (value == null) { throw new CoderException("cannot encode a null ElementAndRestriction"); @@ -60,7 +60,7 @@ public void encode( } @Override - public ElementAndRestriction decode(InputStream inStream, Context context) + public ElementAndRestriction decode(InputStream inStream) throws IOException { ElementT key = elementCoder.decode(inStream); RestrictionT value = restrictionCoder.decode(inStream); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java index 0869244c8e4d..b1cb1a61eb5b 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java @@ -69,18 +69,16 @@ public Coder getElementCoder() { } @Override - public void encode(KeyedWorkItem value, OutputStream outStream, Coder.Context context) + public void encode(KeyedWorkItem value, OutputStream outStream) throws CoderException, IOException { - Coder.Context nestedContext = context.nested(); keyCoder.encode(value.key(), outStream); timersCoder.encode(value.timersIterable(), outStream); elemsCoder.encode(value.elementsIterable(), outStream); } @Override - public KeyedWorkItem decode(InputStream inStream, Coder.Context context) + public KeyedWorkItem decode(InputStream inStream) throws CoderException, IOException { - Coder.Context nestedContext = context.nested(); K key = keyCoder.decode(inStream); Iterable timers = timersCoder.decode(inStream); Iterable> elems = elemsCoder.decode(inStream); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java index f0a62cd6973a..f4a12d089116 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java @@ -238,9 +238,8 @@ private TimerDataCoder(Coder windowCoder) { } @Override - public void encode(TimerData timer, OutputStream outStream, Context context) + public void encode(TimerData timer, OutputStream outStream) throws CoderException, IOException { - Context nestedContext = context.nested(); STRING_CODER.encode(timer.getTimerId(), outStream); STRING_CODER.encode(timer.getNamespace().stringKey(), outStream); INSTANT_CODER.encode(timer.getTimestamp(), outStream); @@ -248,9 +247,8 @@ public void encode(TimerData timer, OutputStream outStream, Context context) } @Override - public TimerData decode(InputStream inStream, Context context) + public TimerData decode(InputStream inStream) throws CoderException, IOException { - Context nestedContext = context.nested(); String timerId = STRING_CODER.decode(inStream); StateNamespace namespace = StateNamespaces.fromString(STRING_CODER.decode(inStream), windowCoder); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java index 33d171e50449..5bc807748ad3 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java @@ -178,15 +178,14 @@ static class RecordNoEncodeCoder extends AtomicCoder { @Override public void encode( Record value, - OutputStream outStream, - org.apache.beam.sdk.coders.Coder.Context context) + OutputStream outStream) throws IOException { throw new CoderException("Encode not allowed"); } @Override public Record decode( - InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context) + InputStream inStream) throws IOException { return null; } @@ -196,13 +195,12 @@ static class RecordNoDecodeCoder extends AtomicCoder { @Override public void encode( Record value, - OutputStream outStream, - org.apache.beam.sdk.coders.Coder.Context context) + OutputStream outStream) throws IOException {} @Override public Record decode( - InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context) + InputStream inStream) throws IOException { throw new CoderException("Decode not allowed"); } @@ -212,13 +210,12 @@ private static class RecordStructuralValueCoder extends AtomicCoder { @Override public void encode( Record value, - OutputStream outStream, - org.apache.beam.sdk.coders.Coder.Context context) + OutputStream outStream) throws CoderException, IOException {} @Override public Record decode( - InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context) + InputStream inStream) throws CoderException, IOException { return new Record() { @Override @@ -244,13 +241,12 @@ private static class RecordNotConsistentWithEqualsStructuralValueCoder @Override public void encode( Record value, - OutputStream outStream, - org.apache.beam.sdk.coders.Coder.Context context) + OutputStream outStream) throws CoderException, IOException {} @Override public Record decode( - InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context) + InputStream inStream) throws CoderException, IOException { return new Record() { @Override diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java index b9ba7f49df79..2a01db55ae1e 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java @@ -590,15 +590,14 @@ public static class Coder extends AtomicCoder { @Override public void encode( TestCheckpointMark value, - OutputStream outStream, - org.apache.beam.sdk.coders.Coder.Context context) + OutputStream outStream) throws IOException { VarInt.encode(value.index, outStream); } @Override public TestCheckpointMark decode( - InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context) + InputStream inStream) throws IOException { TestCheckpointMark decoded = new TestCheckpointMark(VarInt.decodeInt(inStream)); decoded.decoded = true; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java index d7bae7e7b3db..b62fc16fbe6c 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java @@ -65,6 +65,12 @@ public Coder getElementCoder() { return elemCoder; } + @Override + public void encode(SingletonKeyedWorkItem value, OutputStream outStream) + throws CoderException, IOException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(SingletonKeyedWorkItem value, OutputStream outStream, @@ -74,6 +80,11 @@ public void encode(SingletonKeyedWorkItem value, valueCoder.encode(value.value, outStream, context); } + @Override + public SingletonKeyedWorkItem decode(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Context.NESTED); + } + @Override public SingletonKeyedWorkItem decode(InputStream inStream, Context context) throws CoderException, IOException { diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java index 0e60fa09dba9..34609dfecce3 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java @@ -1350,6 +1350,12 @@ public static TransformedMapCoder of( return new TransformedMapCoder<>(transformCoder, originalMapCoder); } + @Override + public void encode(TransformedMap value, OutputStream outStream, OutputStream outStream) + throws CoderException, IOException { + encode(outStream, outStream, Coder.Context.NESTED); + } + @Override public void encode(TransformedMap value, OutputStream outStream, Coder.Context context) throws CoderException, IOException { @@ -1357,6 +1363,11 @@ public void encode(TransformedMap value, OutputStream outStream, originalMapCoder.encode(value.originalMap, outStream, context); } + @Override + public TransformedMap decode(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Coder.Context.NESTED); + } + @Override public TransformedMap decode( InputStream inStream, Coder.Context context) throws CoderException, IOException { diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java index 0f0cd4d940d7..8cfae816d217 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java @@ -231,8 +231,7 @@ public Coder getValueCoder() { } @Override - public void encode(IsmRecord value, OutputStream outStream, - Coder.Context context) throws CoderException, IOException { + public void encode(IsmRecord value, OutputStream outStream) throws CoderException, IOException { if (value.getKeyComponents().size() != keyComponentCoders.size()) { throw new CoderException(String.format( "Expected %s key component(s) but received key component(s) %s.", @@ -249,7 +248,7 @@ public void encode(IsmRecord value, OutputStream outStream, } @Override - public IsmRecord decode(InputStream inStream, Coder.Context context) + public IsmRecord decode(InputStream inStream) throws CoderException, IOException { List keyComponents = new ArrayList<>(keyComponentCoders.size()); for (Coder keyCoder : keyComponentCoders) { @@ -493,7 +492,7 @@ public Coder getKeyCoder() { } @Override - public void encode(K value, OutputStream outStream, Coder.Context context) + public void encode(K value, OutputStream outStream) throws CoderException, IOException { if (value == METADATA_KEY) { outStream.write(0); @@ -504,7 +503,7 @@ public void encode(K value, OutputStream outStream, Coder.Context context) } @Override - public K decode(InputStream inStream, Coder.Context context) + public K decode(InputStream inStream) throws CoderException, IOException { int marker = inStream.read(); if (marker == 0) { @@ -620,6 +619,12 @@ public static IsmShardCoder of() { private IsmShardCoder() {} + @Override + public void encode(IsmShard value, OutputStream outStream) + throws CoderException, IOException { + encode(value, outStream, Coder.Context.NESTED); + } + @Override public void encode(IsmShard value, OutputStream outStream, Coder.Context context) throws CoderException, IOException { @@ -631,6 +636,11 @@ public void encode(IsmShard value, OutputStream outStream, Coder.Context context VarLongCoder.of().encode(value.getIndexOffset(), outStream, context); } + @Override + public IsmShard decode(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Coder.Context.NESTED); + } + @Override public IsmShard decode( InputStream inStream, Coder.Context context) throws CoderException, IOException { @@ -683,14 +693,14 @@ public static KeyPrefixCoder of() { } @Override - public void encode(KeyPrefix value, OutputStream outStream, Coder.Context context) + public void encode(KeyPrefix value, OutputStream outStream) throws CoderException, IOException { VarInt.encode(value.getSharedKeySize(), outStream); VarInt.encode(value.getUnsharedKeySize(), outStream); } @Override - public KeyPrefix decode(InputStream inStream, Coder.Context context) + public KeyPrefix decode(InputStream inStream) throws CoderException, IOException { return KeyPrefix.of(VarInt.decodeInt(inStream), VarInt.decodeInt(inStream)); } @@ -755,7 +765,7 @@ public static FooterCoder of() { } @Override - public void encode(Footer value, OutputStream outStream, Coder.Context context) + public void encode(Footer value, OutputStream outStream) throws CoderException, IOException { DataOutputStream dataOut = new DataOutputStream(outStream); dataOut.writeLong(value.getIndexPosition()); @@ -765,7 +775,7 @@ public void encode(Footer value, OutputStream outStream, Coder.Context context) } @Override - public Footer decode(InputStream inStream, Coder.Context context) + public Footer decode(InputStream inStream) throws CoderException, IOException { DataInputStream dataIn = new DataInputStream(inStream); Footer footer = Footer.of(dataIn.readLong(), dataIn.readLong(), dataIn.readLong()); diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java index f36bd78511e2..5ea9f07ff6a9 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java @@ -62,6 +62,12 @@ public static RandomAccessDataCoder of() { return INSTANCE; } + @Override + public void encode(RandomAccessData value, OutputStream outStream) + throws CoderException, IOException { + encode(value, outStream, Coder.Context.NESTED); + } + @Override public void encode(RandomAccessData value, OutputStream outStream, Coder.Context context) throws CoderException, IOException { @@ -74,6 +80,11 @@ public void encode(RandomAccessData value, OutputStream outStream, Coder.Context value.writeTo(outStream, 0, value.size); } + @Override + public RandomAccessData decode(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Coder.Context.NESTED); + } + @Override public RandomAccessData decode(InputStream inStream, Coder.Context context) throws CoderException, IOException { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java index f82c065f488e..bba669de2302 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java @@ -300,7 +300,7 @@ public Class getType() { } @Override - public void encode(T value, OutputStream outStream, Context context) throws IOException { + public void encode(T value, OutputStream outStream) throws IOException { // Get a BinaryEncoder instance from the ThreadLocal cache and attempt to reuse it. BinaryEncoder encoderInstance = ENCODER_FACTORY.directBinaryEncoder(outStream, encoder.get()); // Save the potentially-new instance for reuse later. @@ -310,7 +310,7 @@ public void encode(T value, OutputStream outStream, Context context) throws IOEx } @Override - public T decode(InputStream inStream, Context context) throws IOException { + public T decode(InputStream inStream) throws IOException { // Get a BinaryDecoder instance from the ThreadLocal cache and attempt to reuse it. BinaryDecoder decoderInstance = DECODER_FACTORY.directBinaryDecoder(inStream, decoder.get()); // Save the potentially-new instance for later. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java index e2166cf25d58..e890d118648d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java @@ -47,6 +47,12 @@ public static BigDecimalCoder of() { private BigDecimalCoder() {} + @Override + public void encode(BigDecimal value, OutputStream outStream) + throws IOException, CoderException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(BigDecimal value, OutputStream outStream, Context context) throws IOException, CoderException { @@ -55,6 +61,11 @@ public void encode(BigDecimal value, OutputStream outStream, Context context) BIG_INT_CODER.encode(value.unscaledValue(), outStream, context); } + @Override + public BigDecimal decode(InputStream inStream) throws IOException, CoderException { + return decode(inStream, Context.NESTED); + } + @Override public BigDecimal decode(InputStream inStream, Context context) throws IOException, CoderException { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianIntegerCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianIntegerCoder.java index a61f09951173..efb1e4b7a8e9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianIntegerCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianIntegerCoder.java @@ -43,7 +43,7 @@ public static BigEndianIntegerCoder of() { private BigEndianIntegerCoder() {} @Override - public void encode(Integer value, OutputStream outStream, Context context) + public void encode(Integer value, OutputStream outStream) throws IOException, CoderException { if (value == null) { throw new CoderException("cannot encode a null Integer"); @@ -52,7 +52,7 @@ public void encode(Integer value, OutputStream outStream, Context context) } @Override - public Integer decode(InputStream inStream, Context context) + public Integer decode(InputStream inStream) throws IOException, CoderException { try { return new DataInputStream(inStream).readInt(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianLongCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianLongCoder.java index 868160aad1e0..ab85e17e759a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianLongCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianLongCoder.java @@ -43,7 +43,7 @@ public static BigEndianLongCoder of() { private BigEndianLongCoder() {} @Override - public void encode(Long value, OutputStream outStream, Context context) + public void encode(Long value, OutputStream outStream) throws IOException, CoderException { if (value == null) { throw new CoderException("cannot encode a null Long"); @@ -52,7 +52,7 @@ public void encode(Long value, OutputStream outStream, Context context) } @Override - public Long decode(InputStream inStream, Context context) + public Long decode(InputStream inStream) throws IOException, CoderException { try { return new DataInputStream(inStream).readLong(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigIntegerCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigIntegerCoder.java index 3b038afefbe3..d54accfb0fbb 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigIntegerCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigIntegerCoder.java @@ -41,6 +41,12 @@ public static BigIntegerCoder of() { private BigIntegerCoder() {} + @Override + public void encode(BigInteger value, OutputStream outStream) + throws IOException, CoderException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(BigInteger value, OutputStream outStream, Context context) throws IOException, CoderException { @@ -48,6 +54,11 @@ public void encode(BigInteger value, OutputStream outStream, Context context) BYTE_ARRAY_CODER.encode(value.toByteArray(), outStream, context); } + @Override + public BigInteger decode(InputStream inStream) throws IOException, CoderException { + return decode(inStream, Context.NESTED); + } + @Override public BigInteger decode(InputStream inStream, Context context) throws IOException, CoderException { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BitSetCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BitSetCoder.java index f49776b28a5f..811501726237 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BitSetCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BitSetCoder.java @@ -35,6 +35,12 @@ public static BitSetCoder of() { return INSTANCE; } + @Override + public void encode(BitSet value, OutputStream outStream) + throws CoderException, IOException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(BitSet value, OutputStream outStream, Context context) throws CoderException, IOException { @@ -44,6 +50,11 @@ public void encode(BitSet value, OutputStream outStream, Context context) BYTE_ARRAY_CODER.encodeAndOwn(value.toByteArray(), outStream, context); } + @Override + public BitSet decode(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Context.NESTED); + } + @Override public BitSet decode(InputStream inStream, Context context) throws CoderException, IOException { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteArrayCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteArrayCoder.java index c9393a180046..3b3838823814 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteArrayCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteArrayCoder.java @@ -51,6 +51,12 @@ public static ByteArrayCoder of() { private ByteArrayCoder() {} + @Override + public void encode(byte[] value, OutputStream outStream) + throws IOException, CoderException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(byte[] value, OutputStream outStream, Context context) throws IOException, CoderException { @@ -85,6 +91,11 @@ public void encodeAndOwn(byte[] value, OutputStream outStream, Context context) } } + @Override + public byte[] decode(InputStream inStream) throws IOException, CoderException { + return decode(inStream, Context.NESTED); + } + @Override public byte[] decode(InputStream inStream, Context context) throws IOException, CoderException { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteCoder.java index 7f449d6ae895..2d23a642f27b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteCoder.java @@ -41,7 +41,7 @@ public static ByteCoder of() { private ByteCoder() {} @Override - public void encode(Byte value, OutputStream outStream, Context context) + public void encode(Byte value, OutputStream outStream) throws IOException, CoderException { if (value == null) { throw new CoderException("cannot encode a null Byte"); @@ -50,7 +50,7 @@ public void encode(Byte value, OutputStream outStream, Context context) } @Override - public Byte decode(InputStream inStream, Context context) + public Byte decode(InputStream inStream) throws IOException, CoderException { try { // value will be between 0-255, -1 for EOF diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DelegateCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DelegateCoder.java index 86077eb120ce..f51b1564b36f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DelegateCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DelegateCoder.java @@ -65,12 +65,23 @@ public static DelegateCoder of( return new DelegateCoder(coder, toFn, fromFn, typeDescriptor); } + @Override + public void encode(T value, OutputStream outStream) + throws CoderException, IOException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(T value, OutputStream outStream, Context context) throws CoderException, IOException { coder.encode(applyAndWrapExceptions(toFn, value), outStream, context); } + @Override + public T decode(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Context.NESTED); + } + @Override public T decode(InputStream inStream, Context context) throws CoderException, IOException { return applyAndWrapExceptions(fromFn, coder.decode(inStream, context)); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DoubleCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DoubleCoder.java index 8eff6ba9b6fb..deb18f22401c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DoubleCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DoubleCoder.java @@ -43,7 +43,7 @@ public static DoubleCoder of() { private DoubleCoder() {} @Override - public void encode(Double value, OutputStream outStream, Context context) + public void encode(Double value, OutputStream outStream) throws IOException, CoderException { if (value == null) { throw new CoderException("cannot encode a null Double"); @@ -52,7 +52,7 @@ public void encode(Double value, OutputStream outStream, Context context) } @Override - public Double decode(InputStream inStream, Context context) + public Double decode(InputStream inStream) throws IOException, CoderException { try { return new DataInputStream(inStream).readDouble(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DurationCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DurationCoder.java index b7db3057c08f..90de26fffa0a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DurationCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DurationCoder.java @@ -54,7 +54,7 @@ private ReadableDuration fromLong(Long decoded) { } @Override - public void encode(ReadableDuration value, OutputStream outStream, Context context) + public void encode(ReadableDuration value, OutputStream outStream) throws CoderException, IOException { if (value == null) { throw new CoderException("cannot encode a null ReadableDuration"); @@ -63,7 +63,7 @@ public void encode(ReadableDuration value, OutputStream outStream, Context conte } @Override - public ReadableDuration decode(InputStream inStream, Context context) + public ReadableDuration decode(InputStream inStream) throws CoderException, IOException { return fromLong(LONG_CODER.decode(inStream)); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java index 22b11a3ed08e..648493ea2ddc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java @@ -68,7 +68,7 @@ protected Instant doBackward(Long shiftedMillis) { } @Override - public void encode(Instant value, OutputStream outStream, Context context) + public void encode(Instant value, OutputStream outStream) throws CoderException, IOException { if (value == null) { throw new CoderException("cannot encode a null Instant"); @@ -77,7 +77,7 @@ public void encode(Instant value, OutputStream outStream, Context context) } @Override - public Instant decode(InputStream inStream, Context context) + public Instant decode(InputStream inStream) throws CoderException, IOException { return ORDER_PRESERVING_CONVERTER.reverse().convert(LONG_CODER.decode(inStream)); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java index 59d5424d52cc..248c26caed06 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java @@ -84,12 +84,11 @@ protected IterableLikeCoder(Coder elementCoder, String iterableName) { @Override public void encode( - IterableT iterable, OutputStream outStream, Context context) + IterableT iterable, OutputStream outStream) throws IOException, CoderException { if (iterable == null) { throw new CoderException("cannot encode a null " + iterableName); } - Context nestedContext = context.nested(); DataOutputStream dataOutStream = new DataOutputStream(outStream); if (iterable instanceof Collection) { // We can know the size of the Iterable. Use an encoding with a @@ -117,9 +116,8 @@ public void encode( } @Override - public IterableT decode(InputStream inStream, Context context) + public IterableT decode(InputStream inStream) throws IOException, CoderException { - Context nestedContext = context.nested(); DataInputStream dataInStream = new DataInputStream(inStream); int size = dataInStream.readInt(); if (size >= 0) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoder.java index 0bb53ecbe006..9c0188671a4a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoder.java @@ -57,6 +57,12 @@ private KvCoder(Coder keyCoder, Coder valueCoder) { this.valueCoder = valueCoder; } + @Override + public void encode(KV kv, OutputStream outStream) + throws IOException, CoderException { + encode(kv, outStream, Context.NESTED); + } + @Override public void encode(KV kv, OutputStream outStream, Context context) throws IOException, CoderException { @@ -67,6 +73,11 @@ public void encode(KV kv, OutputStream outStream, Context context) valueCoder.encode(kv.getValue(), outStream, context); } + @Override + public KV decode(InputStream inStream) throws IOException, CoderException { + return decode(inStream, Context.NESTED); + } + @Override public KV decode(InputStream inStream, Context context) throws IOException, CoderException { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/LengthPrefixCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/LengthPrefixCoder.java index 7dd2a32098a7..b24f66d3a083 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/LengthPrefixCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/LengthPrefixCoder.java @@ -53,7 +53,7 @@ private LengthPrefixCoder(Coder valueCoder) { } @Override - public void encode(T value, OutputStream outStream, Context context) + public void encode(T value, OutputStream outStream) throws CoderException, IOException { ByteArrayOutputStream bos = new ByteArrayOutputStream(); valueCoder.encode(value, bos, Context.OUTER); @@ -62,7 +62,7 @@ public void encode(T value, OutputStream outStream, Context context) } @Override - public T decode(InputStream inStream, Context context) throws CoderException, IOException { + public T decode(InputStream inStream) throws CoderException, IOException { long size = VarInt.decodeLong(inStream); return valueCoder.decode(ByteStreams.limit(inStream, size), Context.OUTER); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java index f20eb93422e0..d8b3f1cdabe8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java @@ -68,6 +68,12 @@ private MapCoder(Coder keyCoder, Coder valueCoder) { this.valueCoder = valueCoder; } + @Override + public void encode(Map map, OutputStream outStream) + throws IOException, CoderException { + encode(map, outStream, Context.NESTED); + } + @Override public void encode( Map map, @@ -99,6 +105,11 @@ public void encode( // no flush needed as DataOutputStream does not buffer } + @Override + public Map decode(InputStream inStream) throws IOException, CoderException { + return decode(inStream, Context.NESTED); + } + @Override public Map decode(InputStream inStream, Context context) throws IOException, CoderException { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java index e46591e4c799..64229e8edc50 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java @@ -60,6 +60,12 @@ public Coder getValueCoder() { return valueCoder; } + @Override + public void encode(@Nullable T value, OutputStream outStream) + throws IOException, CoderException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(@Nullable T value, OutputStream outStream, Context context) throws IOException, CoderException { @@ -71,6 +77,11 @@ public void encode(@Nullable T value, OutputStream outStream, Context context) } } + @Override + public T decode(InputStream inStream) throws IOException, CoderException { + return decode(inStream, Context.NESTED); + } + @Override @Nullable public T decode(InputStream inStream, Context context) throws IOException, CoderException { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java index e3b2959fd98b..9aa8493e5e71 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java @@ -119,7 +119,7 @@ public Class getRecordType() { } @Override - public void encode(T value, OutputStream outStream, Context context) + public void encode(T value, OutputStream outStream) throws IOException, CoderException { try { ObjectOutputStream oos = new ObjectOutputStream(outStream); @@ -131,7 +131,7 @@ public void encode(T value, OutputStream outStream, Context context) } @Override - public T decode(InputStream inStream, Context context) + public T decode(InputStream inStream) throws IOException, CoderException { try { ObjectInputStream ois = new ObjectInputStream(inStream); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java index 1f4538fc1eff..216129157a5b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java @@ -99,12 +99,23 @@ public int hashCode() { return this.clazz.hashCode(); } + @Override + public void encode(T value, OutputStream outStream) + throws CoderException, IOException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(T value, OutputStream outStream, Context context) throws CoderException, IOException { delegateCoder.encode(value, outStream, context); } + @Override + public T decode(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Context.NESTED); + } + @Override public T decode(InputStream inStream, Context context) throws CoderException, IOException { return delegateCoder.decode(inStream, context); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringUtf8Coder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringUtf8Coder.java index 44856e8a045f..3bbc983f6551 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringUtf8Coder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringUtf8Coder.java @@ -66,6 +66,12 @@ private static String readString(DataInputStream dis) throws IOException { private StringUtf8Coder() {} + @Override + public void encode(String value, OutputStream outStream) + throws IOException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(String value, OutputStream outStream, Context context) throws IOException { @@ -84,6 +90,11 @@ public void encode(String value, OutputStream outStream, Context context) } } + @Override + public String decode(InputStream inStream) throws IOException { + return decode(inStream, Context.NESTED); + } + @Override public String decode(InputStream inStream, Context context) throws IOException { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TextualIntegerCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TextualIntegerCoder.java index 718811c9061b..6078fa3c594f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TextualIntegerCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TextualIntegerCoder.java @@ -38,6 +38,12 @@ public static TextualIntegerCoder of() { protected TextualIntegerCoder() {} + @Override + public void encode(Integer value, OutputStream outStream) + throws IOException, CoderException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(Integer value, OutputStream outStream, Context context) throws IOException, CoderException { @@ -48,6 +54,11 @@ public void encode(Integer value, OutputStream outStream, Context context) StringUtf8Coder.of().encode(textualValue, outStream, context); } + @Override + public Integer decode(InputStream inStream) throws IOException, CoderException { + return decode(inStream, Context.NESTED); + } + @Override public Integer decode(InputStream inStream, Context context) throws IOException, CoderException { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarIntCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarIntCoder.java index bda66bbeb40d..3a9abe77ca99 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarIntCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarIntCoder.java @@ -44,7 +44,7 @@ public static VarIntCoder of() { private VarIntCoder() {} @Override - public void encode(Integer value, OutputStream outStream, Context context) + public void encode(Integer value, OutputStream outStream) throws IOException, CoderException { if (value == null) { throw new CoderException("cannot encode a null Integer"); @@ -53,7 +53,7 @@ public void encode(Integer value, OutputStream outStream, Context context) } @Override - public Integer decode(InputStream inStream, Context context) + public Integer decode(InputStream inStream) throws IOException, CoderException { try { return VarInt.decodeInt(inStream); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarLongCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarLongCoder.java index bf651c37bb6f..37ad8f62a872 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarLongCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarLongCoder.java @@ -45,7 +45,7 @@ public static VarLongCoder of() { private VarLongCoder() {} @Override - public void encode(Long value, OutputStream outStream, Context context) + public void encode(Long value, OutputStream outStream) throws IOException, CoderException { if (value == null) { throw new CoderException("cannot encode a null Long"); @@ -54,7 +54,7 @@ public void encode(Long value, OutputStream outStream, Context context) } @Override - public Long decode(InputStream inStream, Context context) + public Long decode(InputStream inStream) throws IOException, CoderException { try { return VarInt.decodeLong(inStream); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VoidCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VoidCoder.java index 4467faa84f62..3e1ff7f688f7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VoidCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VoidCoder.java @@ -38,12 +38,12 @@ public static VoidCoder of() { private VoidCoder() {} @Override - public void encode(Void value, OutputStream outStream, Context context) { + public void encode(Void value, OutputStream outStream) { // Nothing to write! } @Override - public Void decode(InputStream inStream, Context context) { + public Void decode(InputStream inStream) { // Nothing to read! return null; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java index d8a98cd2e2c6..3620c22737ec 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java @@ -946,6 +946,12 @@ public static FileResultCoder of() { return INSTANCE; } + @Override + public void encode(FileResult value, OutputStream outStream) + throws IOException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(FileResult value, OutputStream outStream, Context context) throws IOException { @@ -960,6 +966,11 @@ public void encode(FileResult value, OutputStream outStream, Context context) } } + @Override + public FileResult decode(InputStream inStream) throws IOException { + return decode(inStream, Context.NESTED); + } + @Override public FileResult decode(InputStream inStream, Context context) throws IOException { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java index 9b9d3f8a106f..d12d193edc00 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java @@ -679,7 +679,7 @@ public QuantileStateCoder(ComparatorT compareFn, Coder elementCoder) { @Override public void encode( - QuantileState state, OutputStream outStream, Coder.Context context) + QuantileState state, OutputStream outStream) throws CoderException, IOException { intCoder.encode(state.numQuantiles, outStream); intCoder.encode(state.bufferSize, outStream); @@ -695,7 +695,7 @@ public void encode( } @Override - public QuantileState decode(InputStream inStream, Coder.Context context) + public QuantileState decode(InputStream inStream) throws CoderException, IOException { int numQuantiles = intCoder.decode(inStream); int bufferSize = intCoder.decode(inStream); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java index b9cdbd5f407f..7e435647ed15 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java @@ -531,6 +531,12 @@ public HolderCoder(Coder valueCoder) { this.valueCoder = valueCoder; } + @Override + public void encode(Holder accumulator, OutputStream outStream) + throws CoderException, IOException { + encode(accumulator, outStream, Context.NESTED); + } + @Override public void encode(Holder accumulator, OutputStream outStream, Context context) throws CoderException, IOException { @@ -542,6 +548,11 @@ public void encode(Holder accumulator, OutputStream outStream, Context contex } } + @Override + public Holder decode(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Context.NESTED); + } + @Override public Holder decode(InputStream inStream, Context context) throws CoderException, IOException { @@ -1970,6 +1981,12 @@ public InputOrAccumCoder(Coder inputCoder, Coder accumCoder) { this.accumCoder = accumCoder; } + @Override + public void encode(InputOrAccum value, OutputStream outStream) + throws CoderException, IOException { + encode(value, outStream, Coder.Context.NESTED); + } + @Override public void encode( InputOrAccum value, OutputStream outStream, Coder.Context context) @@ -1983,6 +2000,11 @@ public void encode( } } + @Override + public InputOrAccum decode(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Coder.Context.NESTED); + } + @Override public InputOrAccum decode(InputStream inStream, Coder.Context context) throws CoderException, IOException { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java index c45df046f60f..c61978367d9e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java @@ -533,6 +533,12 @@ public ComposedAccumulatorCoder(List> coders) { this.codersCount = coders.size(); } + @Override + public void encode(Object[] value, OutputStream outStream) + throws CoderException, IOException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(Object[] value, OutputStream outStream, Context context) throws CoderException, IOException { @@ -541,13 +547,17 @@ public void encode(Object[] value, OutputStream outStream, Context context) return; } int lastIndex = codersCount - 1; - Context nestedContext = context.nested(); for (int i = 0; i < lastIndex; ++i) { coders.get(i).encode(value[i], outStream); } coders.get(lastIndex).encode(value[lastIndex], outStream, context); } + @Override + public Object[] decode(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Context.NESTED); + } + @Override public Object[] decode(InputStream inStream, Context context) throws CoderException, IOException { @@ -556,7 +566,6 @@ public Object[] decode(InputStream inStream, Context context) return ret; } int lastIndex = codersCount - 1; - Context nestedContext = context.nested(); for (int i = 0; i < lastIndex; ++i) { ret[i] = coders.get(i).decode(inStream); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java index 497d62b2f4a1..b405dd1aae30 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java @@ -169,13 +169,13 @@ public Coder getAccumulatorCoder(CoderRegistry registry, Coder inputCoder) { return new AtomicCoder() { @Override - public void encode(long[] value, OutputStream outStream, Context context) + public void encode(long[] value, OutputStream outStream) throws IOException { VarInt.encode(value[0], outStream); } @Override - public long[] decode(InputStream inStream, Context context) + public long[] decode(InputStream inStream) throws IOException, CoderException { try { return new long[] {VarInt.decodeLong(inStream)}; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java index c8e0d954d174..8932b03e5271 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java @@ -185,14 +185,14 @@ static class CountSumCoder extends AtomicCoder DOUBLE_CODER = DoubleCoder.of(); @Override - public void encode(CountSum value, OutputStream outStream, Coder.Context context) + public void encode(CountSum value, OutputStream outStream) throws CoderException, IOException { LONG_CODER.encode(value.count, outStream); DOUBLE_CODER.encode(value.sum, outStream); } @Override - public CountSum decode(InputStream inStream, Coder.Context context) + public CountSum decode(InputStream inStream) throws CoderException, IOException { return new CountSum<>( LONG_CODER.decode(inStream), diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java index 7aec66730288..dd8bc5f6f659 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java @@ -539,13 +539,13 @@ public BoundedHeapCoder(int maximumSize, ComparatorT compareFn, Coder element @Override public void encode( - BoundedHeap value, OutputStream outStream, Context context) + BoundedHeap value, OutputStream outStream) throws CoderException, IOException { listCoder.encode(value.asList(), outStream); } @Override - public BoundedHeap decode(InputStream inStream, Coder.Context context) + public BoundedHeap decode(InputStream inStream) throws CoderException, IOException { return new BoundedHeap<>(maximumSize, compareFn, listCoder.decode(inStream)); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java index 66033255184d..d42de829c789 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java @@ -242,8 +242,7 @@ public UnionCoder getUnionCoder() { @SuppressWarnings("unchecked") public void encode( CoGbkResult value, - OutputStream outStream, - Context context) throws CoderException, + OutputStream outStream) throws CoderException, IOException { if (!schema.equals(value.getSchema())) { throw new CoderException("input schema does not match coder schema"); @@ -258,8 +257,7 @@ public void encode( @Override public CoGbkResult decode( - InputStream inStream, - Context context) + InputStream inStream) throws CoderException, IOException { if (schema.size() == 0) { return new CoGbkResult(schema, ImmutableList.>of()); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java index 3194a37f4966..66959d34aa47 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java @@ -54,6 +54,12 @@ private int getIndexForEncoding(RawUnionValue union) { return index; } + @Override + public void encode(RawUnionValue union, OutputStream outStream) + throws IOException, CoderException { + encode(union, outStream, Context.NESTED); + } + @SuppressWarnings("unchecked") @Override public void encode( @@ -73,6 +79,11 @@ public void encode( context); } + @Override + public RawUnionValue decode(InputStream inStream) throws IOException, CoderException { + return decode(inStream, Context.NESTED); + } + @Override public RawUnionValue decode(InputStream inStream, Context context) throws IOException, CoderException { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java index 0bfb8750dcdc..078cbeed087f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java @@ -67,10 +67,10 @@ public static class Coder extends StructuredCoder { public static final Coder INSTANCE = new Coder(); @Override - public void encode(GlobalWindow window, OutputStream outStream, Context context) {} + public void encode(GlobalWindow window, OutputStream outStream) {} @Override - public GlobalWindow decode(InputStream inStream, Context context) { + public GlobalWindow decode(InputStream inStream) { return GlobalWindow.INSTANCE; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java index 318dc4ca97bb..f25a2086fa17 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java @@ -179,14 +179,14 @@ public static IntervalWindowCoder of() { } @Override - public void encode(IntervalWindow window, OutputStream outStream, Context context) + public void encode(IntervalWindow window, OutputStream outStream) throws IOException, CoderException { instantCoder.encode(window.end, outStream); durationCoder.encode(new Duration(window.start, window.end), outStream); } @Override - public IntervalWindow decode(InputStream inStream, Context context) + public IntervalWindow decode(InputStream inStream) throws IOException, CoderException { Instant end = instantCoder.decode(inStream); ReadableDuration duration = durationCoder.decode(inStream); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java index 79ce2f5bf24a..75df220ab873 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java @@ -347,7 +347,7 @@ public static PaneInfoCoder of() { private PaneInfoCoder() {} @Override - public void encode(PaneInfo value, final OutputStream outStream, Coder.Context context) + public void encode(PaneInfo value, final OutputStream outStream) throws CoderException, IOException { Encoding encoding = chooseEncoding(value); switch (chooseEncoding(value)) { @@ -369,7 +369,7 @@ public void encode(PaneInfo value, final OutputStream outStream, Coder.Context c } @Override - public PaneInfo decode(final InputStream inStream, Coder.Context context) + public PaneInfo decode(final InputStream inStream) throws CoderException, IOException { byte keyAndTag = (byte) inStream.read(); PaneInfo base = BYTE_TO_PANE_INFO.get((byte) (keyAndTag & 0x0F)); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java index a0896f53cfd7..b20206569bbd 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java @@ -42,12 +42,23 @@ public static BitSetCoder of() { return INSTANCE; } + @Override + public void encode(BitSet value, OutputStream outStream) + throws CoderException, IOException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(BitSet value, OutputStream outStream, Context context) throws CoderException, IOException { BYTE_ARRAY_CODER.encodeAndOwn(value.toByteArray(), outStream, context); } + @Override + public BitSet decode(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Context.NESTED); + } + @Override public BitSet decode(InputStream inStream, Context context) throws CoderException, IOException { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java index e3e61cffb1c5..963886bee9d6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java @@ -635,12 +635,17 @@ public WindowedValueCoder withValueCoder(Coder valueCoder) { return new FullWindowedValueCoder<>(valueCoder, windowCoder); } + @Override + public void encode(WindowedValue windowedElem, OutputStream outStream) + throws CoderException, IOException { + encode(windowedElem, outStream, Context.NESTED); + } + @Override public void encode(WindowedValue windowedElem, OutputStream outStream, Context context) throws CoderException, IOException { - Context nestedContext = context.nested(); InstantCoder.of().encode( windowedElem.getTimestamp(), outStream, nestedContext); windowsCoder.encode(windowedElem.getWindows(), outStream); @@ -648,10 +653,14 @@ public void encode(WindowedValue windowedElem, valueCoder.encode(windowedElem.getValue(), outStream, context); } + @Override + public WindowedValue decode(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Context.NESTED); + } + @Override public WindowedValue decode(InputStream inStream, Context context) throws CoderException, IOException { - Context nestedContext = context.nested(); Instant timestamp = InstantCoder.of().decode(inStream); Collection windows = windowsCoder.decode(inStream); @@ -709,12 +718,23 @@ public WindowedValueCoder withValueCoder(Coder valueCoder) { return new ValueOnlyWindowedValueCoder<>(valueCoder); } + @Override + public void encode(WindowedValue windowedElem, OutputStream outStream) + throws CoderException, IOException { + encode(windowedElem, outStream, Context.NESTED); + } + @Override public void encode(WindowedValue windowedElem, OutputStream outStream, Context context) throws CoderException, IOException { valueCoder.encode(windowedElem.getValue(), outStream, context); } + @Override + public WindowedValue decode(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Context.NESTED); + } + @Override public WindowedValue decode(InputStream inStream, Context context) throws CoderException, IOException { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java index 95a3152d874e..a4c8b3fa4b4b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java @@ -101,8 +101,7 @@ public static TimestampedValueCoder of(Coder valueCoder) { @Override public void encode(TimestampedValue windowedElem, - OutputStream outStream, - Context context) + OutputStream outStream) throws IOException { valueCoder.encode(windowedElem.getValue(), outStream); InstantCoder.of().encode( @@ -110,7 +109,7 @@ public void encode(TimestampedValue windowedElem, } @Override - public TimestampedValue decode(InputStream inStream, Context context) + public TimestampedValue decode(InputStream inStream) throws IOException { T value = valueCoder.decode(inStream); Instant timestamp = InstantCoder.of().decode(inStream); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java index e8a2dfdb516d..24c3c3835e41 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java @@ -74,19 +74,28 @@ public static Coder of( this.windowCoder = (org.apache.beam.sdk.coders.Coder) windowCoder; } + @Override + public void encode(ValueInSingleWindow windowedElem, OutputStream outStream) + throws IOException { + encode(windowedElem, outStream, Context.NESTED); + } + @Override public void encode(ValueInSingleWindow windowedElem, OutputStream outStream, Context context) throws IOException { - Context nestedContext = context.nested(); InstantCoder.of().encode(windowedElem.getTimestamp(), outStream); windowCoder.encode(windowedElem.getWindow(), outStream); PaneInfo.PaneInfoCoder.INSTANCE.encode(windowedElem.getPane(), outStream); valueCoder.encode(windowedElem.getValue(), outStream, context); } + @Override + public ValueInSingleWindow decode(InputStream inStream) throws IOException { + return decode(inStream, Context.NESTED); + } + @Override public ValueInSingleWindow decode(InputStream inStream, Context context) throws IOException { - Context nestedContext = context.nested(); Instant timestamp = InstantCoder.of().decode(inStream); BoundedWindow window = windowCoder.decode(inStream); PaneInfo pane = PaneInfo.PaneInfoCoder.INSTANCE.decode(inStream); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueWithRecordId.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueWithRecordId.java index f06317b38571..96a5f1db91c9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueWithRecordId.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueWithRecordId.java @@ -100,6 +100,12 @@ public List> getCoderArguments() { return Arrays.asList(valueCoder); } + @Override + public void encode(ValueWithRecordId value, OutputStream outStream) + throws IOException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(ValueWithRecordId value, OutputStream outStream, Context context) throws IOException { @@ -107,6 +113,11 @@ public void encode(ValueWithRecordId value, OutputStream outStream, Cont idCoder.encode(value.id, outStream, context); } + @Override + public ValueWithRecordId decode(InputStream inStream) throws IOException { + return decode(inStream, Context.NESTED); + } + @Override public ValueWithRecordId decode(InputStream inStream, Context context) throws IOException { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java index 7ca7fb993717..c883ca021366 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java @@ -81,12 +81,12 @@ public void testRegisterInstantiatedCoder() throws Exception { @SuppressWarnings("rawtypes") // this class exists to fail a test because of its rawtypes private class MyListCoder extends AtomicCoder { @Override - public void encode(List value, OutputStream outStream, Context context) + public void encode(List value, OutputStream outStream) throws CoderException, IOException { } @Override - public List decode(InputStream inStream, Context context) + public List decode(InputStream inStream) throws CoderException, IOException { return Collections.emptyList(); } @@ -375,12 +375,12 @@ public static MyValueCoder of() { } @Override - public void encode(MyValue value, OutputStream outStream, Context context) + public void encode(MyValue value, OutputStream outStream) throws CoderException, IOException { } @Override - public MyValue decode(InputStream inStream, Context context) + public MyValue decode(InputStream inStream) throws CoderException, IOException { return new MyValue(); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CustomCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CustomCoderTest.java index dfd4ea252e7f..13a72619f946 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CustomCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CustomCoderTest.java @@ -48,13 +48,13 @@ public MyCustomCoder(String key) { } @Override - public void encode(KV kv, OutputStream out, Context context) + public void encode(KV kv, OutputStream out) throws IOException { new DataOutputStream(out).writeLong(kv.getValue()); } @Override - public KV decode(InputStream inStream, Context context) + public KV decode(InputStream inStream) throws IOException { return KV.of(key, new DataInputStream(inStream).readLong()); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java index d6d7de8b5779..9fb0b822a7a7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java @@ -166,6 +166,12 @@ public void testEncodedTypeDescriptor() throws Exception { } private static class EntireStreamExpectingCoder extends AtomicCoder { + @Override + public void encode(String value, OutputStream outStream) + throws IOException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode( String value, OutputStream outStream, Context context) throws IOException { @@ -173,6 +179,11 @@ public void encode( StringUtf8Coder.of().encode(value, outStream, context); } + @Override + public String decode(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Context.NESTED); + } + @Override public String decode(InputStream inStream, Context context) throws CoderException, IOException { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StructuredCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StructuredCoderTest.java index af2c94e1c68b..7aa2080cf101 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StructuredCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StructuredCoderTest.java @@ -47,7 +47,7 @@ private static class NullBooleanCoder extends StructuredCoder { private static final long serialVersionUID = 0L; @Override - public void encode(@Nullable Boolean value, OutputStream outStream, Context context) + public void encode(@Nullable Boolean value, OutputStream outStream) throws CoderException, IOException { if (value == null) { outStream.write(2); @@ -61,7 +61,7 @@ public void encode(@Nullable Boolean value, OutputStream outStream, Context cont @Override @Nullable public Boolean decode( - InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context) + InputStream inStream) throws CoderException, IOException { int value = inStream.read(); if (value == 0) { @@ -110,7 +110,7 @@ private static class ObjectIdentityBooleanCoder extends StructuredCoder extends StructuredCoder { @Override - public void encode(T value, OutputStream outStream, Coder.Context context) + public void encode(T value, OutputStream outStream) throws CoderException, IOException { throw new UnsupportedOperationException(); } @Override - public T decode(InputStream inStream, Coder.Context context) + public T decode(InputStream inStream) throws CoderException, IOException { throw new UnsupportedOperationException(); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java index 83f348c2678f..37db4ef84824 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java @@ -98,12 +98,12 @@ public static NotSerializableObjectCoder of() { } @Override - public void encode(NotSerializableObject value, OutputStream outStream, Context context) + public void encode(NotSerializableObject value, OutputStream outStream) throws CoderException, IOException { } @Override - public NotSerializableObject decode(InputStream inStream, Context context) + public NotSerializableObject decode(InputStream inStream) throws CoderException, IOException { return new NotSerializableObject(); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java index db5ff2e3049b..375be33944cb 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java @@ -153,11 +153,11 @@ private static class NotSerializableClass { private static class NotSerializableClassCoder extends AtomicCoder { @Override - public void encode(NotSerializableClass value, OutputStream outStream, Coder.Context context) { + public void encode(NotSerializableClass value, OutputStream outStream) { } @Override - public NotSerializableClass decode(InputStream inStream, Coder.Context context) { + public NotSerializableClass decode(InputStream inStream) { return new NotSerializableClass(); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java index 546683b44e36..3939800f9a4c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java @@ -75,14 +75,14 @@ public void unencodableWindowFails() { private static class FailingCoder extends AtomicCoder { @Override public void encode( - BoundedWindow value, OutputStream outStream, Context context) + BoundedWindow value, OutputStream outStream) throws CoderException, IOException { throw new CoderException("Test Encode Exception"); } @Override public BoundedWindow decode( - InputStream inStream, Context context) throws CoderException, IOException { + InputStream inStream) throws CoderException, IOException { throw new CoderException("Test Decode Exception"); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java index 8a4d56365410..33c652a523af 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java @@ -335,12 +335,23 @@ public static UserStringCoder of() { private static final UserStringCoder INSTANCE = new UserStringCoder(); + @Override + public void encode(UserString value, OutputStream outStream) + throws CoderException, IOException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(UserString value, OutputStream outStream, Context context) throws CoderException, IOException { StringUtf8Coder.of().encode(value.strValue, outStream, context); } + @Override + public UserString decode(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Context.NESTED); + } + @Override public UserString decode(InputStream inStream, Context context) throws CoderException, IOException { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java index e4b016be10bb..bd8aee405bf8 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java @@ -875,6 +875,12 @@ public Coder getAccumulatorCoder( * A {@link Coder} for {@link CountSum}. */ private class CountSumCoder extends AtomicCoder { + @Override + public void encode(CountSum value, OutputStream outStream, OutputStream outStream) + throws CoderException, IOException { + encode(outStream, outStream, Context.NESTED); + } + @Override public void encode(CountSum value, OutputStream outStream, Context context) throws CoderException, IOException { @@ -882,6 +888,11 @@ public void encode(CountSum value, OutputStream outStream, DOUBLE_CODER.encode(value.sum, outStream, context); } + @Override + public CountSum decode(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Coder.Context.NESTED); + } + @Override public CountSum decode(InputStream inStream, Coder.Context context) throws CoderException, IOException { @@ -924,12 +935,23 @@ public Accumulator(String value) { public static Coder getCoder() { return new AtomicCoder() { + @Override + public void encode(Accumulator accumulator, OutputStream outStream) + throws CoderException, IOException { + encode(accumulator, outStream, Coder.Context.NESTED); + } + @Override public void encode(Accumulator accumulator, OutputStream outStream, Coder.Context context) throws CoderException, IOException { StringUtf8Coder.of().encode(accumulator.value, outStream, context); } + @Override + public Accumulator decode(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Coder.Context.NESTED); + } + @Override public Accumulator decode(InputStream inStream, Coder.Context context) throws CoderException, IOException { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java index 7e8a1dd7bfcc..a05d31c29d4d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java @@ -134,11 +134,11 @@ static class Record2 extends Record { private static class RecordCoder extends AtomicCoder { @Override - public void encode(Record value, OutputStream outStream, Context context) + public void encode(Record value, OutputStream outStream) throws CoderException, IOException {} @Override - public Record decode(InputStream inStream, Context context) throws CoderException, IOException { + public Record decode(InputStream inStream) throws CoderException, IOException { return null; } } @@ -207,15 +207,14 @@ static class UnserializableRecordCoder extends AtomicCoder @Override public void encode( UnserializableRecord value, - OutputStream outStream, - org.apache.beam.sdk.coders.Coder.Context context) + OutputStream outStream) throws CoderException, IOException { stringCoder.encode(value.myString, outStream); } @Override public UnserializableRecord decode( - InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context) + InputStream inStream) throws CoderException, IOException { return new UnserializableRecord(stringCoder.decode(inStream)); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java index aba33ebe8b5a..0cd885ca2557 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java @@ -469,13 +469,13 @@ public static DeterministicKeyCoder of() { private DeterministicKeyCoder() {} @Override - public void encode(BadEqualityKey value, OutputStream outStream, Context context) + public void encode(BadEqualityKey value, OutputStream outStream) throws IOException { new DataOutputStream(outStream).writeLong(value.key); } @Override - public BadEqualityKey decode(InputStream inStream, Context context) + public BadEqualityKey decode(InputStream inStream) throws IOException { return new BadEqualityKey(new DataInputStream(inStream).readLong()); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java index d2cb980f3a75..3697211091a4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java @@ -986,12 +986,12 @@ public static TestDummyCoder of() { } @Override - public void encode(TestDummy value, OutputStream outStream, Context context) + public void encode(TestDummy value, OutputStream outStream) throws CoderException, IOException { } @Override - public TestDummy decode(InputStream inStream, Context context) + public TestDummy decode(InputStream inStream) throws CoderException, IOException { return new TestDummy(); } @@ -1089,12 +1089,23 @@ public static MyIntegerCoder of() { return INSTANCE; } + @Override + public void encode(MyInteger value, OutputStream outStream) + throws CoderException, IOException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(MyInteger value, OutputStream outStream, Context context) throws CoderException, IOException { delegate.encode(value.getValue(), outStream, context); } + @Override + public MyInteger decode(InputStream inStream) throws CoderException { + return decode(inStream, Context.NESTED); + } + @Override public MyInteger decode(InputStream inStream, Context context) throws CoderException, IOException { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java index 84f3d69dd10a..cdd03d9714fd 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java @@ -506,12 +506,23 @@ public void processElement(ProcessContext c) { } private static class NonDeterministicStringCoder extends AtomicCoder { + @Override + public void encode(String value, OutputStream outStream) + throws CoderException, IOException { + encode(value, outStream, Coder.Context.NESTED); + } + @Override public void encode(String value, OutputStream outStream, Coder.Context context) throws CoderException, IOException { StringUtf8Coder.of().encode(value, outStream, context); } + @Override + public String decode(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Coder.Context.NESTED); + } + @Override public String decode(InputStream inStream, Coder.Context context) throws CoderException, IOException { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java index 489493ab22a8..a8cd35ed7066 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java @@ -277,10 +277,10 @@ public static SomeRestrictionCoder of() { } @Override - public void encode(SomeRestriction value, OutputStream outStream, Context context) {} + public void encode(SomeRestriction value, OutputStream outStream) {} @Override - public SomeRestriction decode(InputStream inStream, Context context) { + public SomeRestriction decode(InputStream inStream) { return null; } } @@ -400,10 +400,10 @@ public static CoderForDefaultTracker of() { @Override public void encode( - RestrictionWithDefaultTracker value, OutputStream outStream, Context context) {} + RestrictionWithDefaultTracker value, OutputStream outStream) {} @Override - public RestrictionWithDefaultTracker decode(InputStream inStream, Context context) { + public RestrictionWithDefaultTracker decode(InputStream inStream) { return null; } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java index 7230a8bbaae4..f36e5e1f6cab 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java @@ -50,12 +50,12 @@ public static TestCoder of() { } @Override - public void encode(Integer value, OutputStream outStream, Context context) { + public void encode(Integer value, OutputStream outStream) { throw new RuntimeException("not expecting to be called"); } @Override - public Integer decode(InputStream inStream, Context context) { + public Integer decode(InputStream inStream) { throw new RuntimeException("not expecting to be called"); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java index 6ba1d4a8af70..9a807305a254 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java @@ -89,12 +89,12 @@ private static class UnserializableCoderByJava extends AtomicCoder { private final Object unserializableField = new Object(); @Override - public void encode(Object value, OutputStream outStream, Context context) + public void encode(Object value, OutputStream outStream) throws CoderException, IOException { } @Override - public Object decode(InputStream inStream, Context context) + public Object decode(InputStream inStream) throws CoderException, IOException { return unserializableField; } diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ByteStringCoder.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ByteStringCoder.java index 325c69dfee8e..73c79774e03f 100644 --- a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ByteStringCoder.java +++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ByteStringCoder.java @@ -48,6 +48,12 @@ public static ByteStringCoder of() { private ByteStringCoder() {} + @Override + public void encode(ByteString value, OutputStream outStream) + throws IOException, CoderException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(ByteString value, OutputStream outStream, Context context) throws IOException, CoderException { @@ -62,6 +68,11 @@ public void encode(ByteString value, OutputStream outStream, Context context) value.writeTo(outStream); } + @Override + public ByteString decode(InputStream inStream) throws IOException { + return decode(inStream, Context.NESTED); + } + @Override public ByteString decode(InputStream inStream, Context context) throws IOException { if (context.isWholeStream) { diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java index 968a2fa06230..f73bf2b09736 100644 --- a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java +++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java @@ -167,6 +167,12 @@ public ProtoCoder withExtensionsFrom(Class... moreExtensionHosts) { return withExtensionsFrom(Arrays.asList(moreExtensionHosts)); } + @Override + public void encode(T value, OutputStream outStream) + throws IOException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(T value, OutputStream outStream, Context context) throws IOException { if (value == null) { @@ -179,6 +185,11 @@ public void encode(T value, OutputStream outStream, Context context) throws IOEx } } + @Override + public T decode(InputStream inStream) throws IOException { + return decode(inStream, Context.NESTED); + } + @Override public T decode(InputStream inStream, Context context) throws IOException { if (context.isWholeStream) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java index 33b9f779ab9d..f034a030da60 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java @@ -38,7 +38,7 @@ public static TableDestinationCoder of() { } @Override - public void encode(TableDestination value, OutputStream outStream, Context context) + public void encode(TableDestination value, OutputStream outStream) throws IOException { if (value == null) { throw new CoderException("cannot encode a null value"); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowInfoCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowInfoCoder.java index 8ae75c56581e..c4707da19b2e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowInfoCoder.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowInfoCoder.java @@ -37,6 +37,12 @@ public static TableRowInfoCoder of() { return INSTANCE; } + @Override + public void encode(TableRowInfo value, OutputStream outStream) + throws IOException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(TableRowInfo value, OutputStream outStream, Context context) throws IOException { @@ -47,6 +53,11 @@ public void encode(TableRowInfo value, OutputStream outStream, Context context) idCoder.encode(value.uniqueId, outStream, context); } + @Override + public TableRowInfo decode(InputStream inStream) throws IOException { + return decode(inStream, Context.NESTED); + } + @Override public TableRowInfo decode(InputStream inStream, Context context) throws IOException { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowJsonCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowJsonCoder.java index cfec991d7a3a..e4b6f1fde072 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowJsonCoder.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowJsonCoder.java @@ -37,6 +37,12 @@ public static TableRowJsonCoder of() { return INSTANCE; } + @Override + public void encode(TableRow value, OutputStream outStream) + throws IOException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(TableRow value, OutputStream outStream, Context context) throws IOException { @@ -44,6 +50,11 @@ public void encode(TableRow value, OutputStream outStream, Context context) StringUtf8Coder.of().encode(strValue, outStream, context); } + @Override + public TableRow decode(InputStream inStream) throws IOException { + return decode(inStream, Context.NESTED); + } + @Override public TableRow decode(InputStream inStream, Context context) throws IOException { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java index 9e83271a5865..f0140392f4cb 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java @@ -101,7 +101,7 @@ public static ResultCoder of( } @Override - public void encode(Result value, OutputStream outStream, Context context) + public void encode(Result value, OutputStream outStream) throws IOException { if (value == null) { throw new CoderException("cannot encode a null value"); @@ -112,7 +112,7 @@ public void encode(Result value, OutputStream outStream, Context c } @Override - public Result decode(InputStream inStream, Context context) throws IOException { + public Result decode(InputStream inStream) throws IOException { String filename = stringCoder.decode(inStream); long fileByteSize = longCoder.decode(inStream); DestinationT destination = destinationCoder.decode(inStream); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessagePayloadOnlyCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessagePayloadOnlyCoder.java index d120f7225fe5..5df2bcf45734 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessagePayloadOnlyCoder.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessagePayloadOnlyCoder.java @@ -33,12 +33,23 @@ public static PubsubMessagePayloadOnlyCoder of() { return new PubsubMessagePayloadOnlyCoder(); } + @Override + public void encode(PubsubMessage value, OutputStream outStream) + throws IOException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(PubsubMessage value, OutputStream outStream, Context context) throws IOException { PAYLOAD_CODER.encode(value.getPayload(), outStream, context); } + @Override + public PubsubMessage decode(InputStream inStream) throws IOException { + return decode(inStream, Context.NESTED); + } + @Override public PubsubMessage decode(InputStream inStream, Context context) throws IOException { return new PubsubMessage( diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithAttributesCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithAttributesCoder.java index 5907c9e7a214..bcf7656cbee1 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithAttributesCoder.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithAttributesCoder.java @@ -45,12 +45,23 @@ public static PubsubMessageWithAttributesCoder of() { return new PubsubMessageWithAttributesCoder(); } + @Override + public void encode(PubsubMessage value, OutputStream outStream) + throws IOException { + encode(value, outStream, Context.NESTED); + } + public void encode(PubsubMessage value, OutputStream outStream, Context context) throws IOException { PAYLOAD_CODER.encode(value.getPayload(), outStream); ATTRIBUTES_CODER.encode(value.getAttributeMap(), outStream, context); } + @Override + public PubsubMessage decode(InputStream inStream) throws IOException { + return decode(inStream, Context.NESTED); + } + @Override public PubsubMessage decode(InputStream inStream, Context context) throws IOException { byte[] payload = PAYLOAD_CODER.decode(inStream); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java index ae320c73a05b..ad38e2800f9e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java @@ -108,7 +108,7 @@ private static class OutgoingMessageCoder extends AtomicCoder { @Override public void encode( - OutgoingMessage value, OutputStream outStream, Context context) + OutgoingMessage value, OutputStream outStream) throws CoderException, IOException { ByteArrayCoder.of().encode(value.elementBytes, outStream); ATTRIBUTES_CODER.encode(value.attributes, outStream); @@ -118,7 +118,7 @@ public void encode( @Override public OutgoingMessage decode( - InputStream inStream, Context context) throws CoderException, IOException { + InputStream inStream) throws CoderException, IOException { byte[] elementBytes = ByteArrayCoder.of().decode(inStream); Map attributes = ATTRIBUTES_CODER.decode(inStream); long timestampMsSinceEpoch = BigEndianLongCoder.of().decode(inStream); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java index e53976e5ffb8..db8c1b74e66d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java @@ -368,6 +368,12 @@ public static PubsubCheckpointCoder of() { private PubsubCheckpointCoder() {} + @Override + public void encode(PubsubCheckpoint value, OutputStream outStream) + throws IOException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(PubsubCheckpoint value, OutputStream outStream, Context context) throws IOException { @@ -378,6 +384,11 @@ public void encode(PubsubCheckpoint value, OutputStream outStream, Context conte LIST_CODER.encode(value.notYetReadIds, outStream, context); } + @Override + public PubsubCheckpoint decode(InputStream inStream) throws IOException { + return decode(inStream, Context.NESTED); + } + @Override public PubsubCheckpoint decode(InputStream inStream, Context context) throws IOException { String path = SUBSCRIPTION_PATH_CODER.decode(inStream); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java index d60c7214b701..70d53779f281 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java @@ -747,12 +747,23 @@ public int hashCode() { * Coder for @link{PartitionedGlobalWindow}. */ private static class PartitionedGlobalWindowCoder extends AtomicCoder { + @Override + public void encode(PartitionedGlobalWindow window, OutputStream outStream) + throws IOException, CoderException { + encode(window, outStream, Context.NESTED); + } + @Override public void encode(PartitionedGlobalWindow window, OutputStream outStream, Context context) throws IOException, CoderException { StringUtf8Coder.of().encode(window.value, outStream, context); } + @Override + public PartitionedGlobalWindow decode(InputStream inStream) throws IOException, CoderException { + return decode(inStream, Context.NESTED); + } + @Override public PartitionedGlobalWindow decode(InputStream inStream, Context context) throws IOException, CoderException { diff --git a/sdks/java/io/hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop/WritableCoder.java b/sdks/java/io/hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop/WritableCoder.java index 8fddfe004229..8d2598ad3712 100644 --- a/sdks/java/io/hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop/WritableCoder.java +++ b/sdks/java/io/hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop/WritableCoder.java @@ -68,13 +68,13 @@ public WritableCoder(Class type) { } @Override - public void encode(T value, OutputStream outStream, Context context) throws IOException { + public void encode(T value, OutputStream outStream) throws IOException { value.write(new DataOutputStream(outStream)); } @SuppressWarnings("unchecked") @Override - public T decode(InputStream inStream, Context context) throws IOException { + public T decode(InputStream inStream) throws IOException { try { if (type == NullWritable.class) { // NullWritable has no default constructor diff --git a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseMutationCoder.java b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseMutationCoder.java index 7cc043ce34db..501fe09259c8 100644 --- a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseMutationCoder.java +++ b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseMutationCoder.java @@ -44,16 +44,14 @@ public static HBaseMutationCoder of() { } @Override - public void encode(Mutation mutation, OutputStream outStream, - Coder.Context context) throws IOException { + public void encode(Mutation mutation, OutputStream outStream) throws IOException { MutationType type = getType(mutation); MutationProto proto = ProtobufUtil.toMutation(type, mutation); proto.writeDelimitedTo(outStream); } @Override - public Mutation decode(InputStream inStream, - Coder.Context context) throws IOException { + public Mutation decode(InputStream inStream) throws IOException { return ProtobufUtil.toMutation(MutationProto.parseDelimitedFrom(inStream)); } diff --git a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseResultCoder.java b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseResultCoder.java index 24a5f7f090aa..1d066351d1aa 100644 --- a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseResultCoder.java +++ b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseResultCoder.java @@ -41,13 +41,13 @@ public static HBaseResultCoder of() { } @Override - public void encode(Result value, OutputStream outputStream, Coder.Context context) + public void encode(Result value, OutputStream outputStream) throws IOException { ProtobufUtil.toResult(value).writeDelimitedTo(outputStream); } @Override - public Result decode(InputStream inputStream, Coder.Context context) + public Result decode(InputStream inputStream) throws IOException { return ProtobufUtil.toResult(ClientProtos.Result.parseDelimitedFrom(inputStream)); } diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index ba84c2a3484a..e21945fd7f6d 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -1597,13 +1597,13 @@ public void populateDisplayData(DisplayData.Builder builder) { private static class NullOnlyCoder extends AtomicCoder { @Override - public void encode(T value, OutputStream outStream, Context context) { + public void encode(T value, OutputStream outStream) { checkArgument(value == null, "Can only encode nulls"); // Encode as no bytes. } @Override - public T decode(InputStream inStream, Context context) { + public T decode(InputStream inStream) { return null; } } diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoder.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoder.java index d838a0d7943f..197106011e1f 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoder.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoder.java @@ -49,6 +49,12 @@ public KafkaRecordCoder(Coder keyCoder, Coder valueCoder) { this.kvCoder = KvCoder.of(keyCoder, valueCoder); } + @Override + public void encode(KafkaRecord value, OutputStream outStream) + throws CoderException, IOException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(KafkaRecord value, OutputStream outStream, Context context) throws CoderException, IOException { @@ -60,6 +66,11 @@ public void encode(KafkaRecord value, OutputStream outStream, Context cont kvCoder.encode(value.getKV(), outStream, context); } + @Override + public KafkaRecord decode(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Context.NESTED); + } + @Override public KafkaRecord decode(InputStream inStream, Context context) throws CoderException, IOException { diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java index c6a017461ad4..f233e27d064f 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java @@ -43,7 +43,7 @@ public static KinesisRecordCoder of() { } @Override - public void encode(KinesisRecord value, OutputStream outStream, Context context) throws + public void encode(KinesisRecord value, OutputStream outStream) throws IOException { BYTE_ARRAY_CODER.encode(value.getData().array(), outStream); STRING_CODER.encode(value.getSequenceNumber(), outStream); @@ -56,7 +56,7 @@ public void encode(KinesisRecord value, OutputStream outStream, Context context) } @Override - public KinesisRecord decode(InputStream inStream, Context context) throws IOException { + public KinesisRecord decode(InputStream inStream) throws IOException { ByteBuffer data = ByteBuffer.wrap(BYTE_ARRAY_CODER.decode(inStream)); String sequenceNumber = STRING_CODER.decode(inStream); String partitionKey = STRING_CODER.decode(inStream); diff --git a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/JAXBCoder.java b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/JAXBCoder.java index 5b2ec0251484..d4c0440cc478 100644 --- a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/JAXBCoder.java +++ b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/JAXBCoder.java @@ -88,15 +88,8 @@ public static JAXBCoder of(Class jaxbClass) { } @Override - public void encode(T value, OutputStream outStream) throws IOException { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - try { - jaxbMarshaller.get().marshal(value, baos); - } catch (JAXBException e) { - throw new CoderException(e); - } - VarInt.encode(baos.size(), outStream); - baos.writeTo(outStream); + public void encode(T value, OutputStream outStream) throws CoderException, IOException { + encode(value, outStream, Context.NESTED); } @Override @@ -109,10 +102,22 @@ public void encode(T value, OutputStream outStream, Context context) throw new CoderException(e); } } else { - encode(value, outStream); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + try { + jaxbMarshaller.get().marshal(value, baos); + } catch (JAXBException e) { + throw new CoderException(e); + } + VarInt.encode(baos.size(), outStream); + baos.writeTo(outStream); } } + @Override + public T decode(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Context.NESTED); + } + @Override public T decode(InputStream inStream, Context context) throws CoderException, IOException { try { diff --git a/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/JAXBCoderTest.java b/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/JAXBCoderTest.java index 5386a613fcbc..c175e4a3444a 100644 --- a/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/JAXBCoderTest.java +++ b/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/JAXBCoderTest.java @@ -177,19 +177,28 @@ public TestCoder(JAXBCoder jaxbCoder) { this.jaxbCoder = jaxbCoder; } + @Override + public void encode(TestType value, OutputStream outStream) + throws CoderException, IOException { + encode(value, outStream, Context.NESTED); + } + @Override public void encode(TestType value, OutputStream outStream, Context context) throws CoderException, IOException { - Context nestedContext = context.nested(); VarIntCoder.of().encode(3, outStream); jaxbCoder.encode(value, outStream); VarLongCoder.of().encode(22L, outStream, context); } + @Override + public TestType decode(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Context.NESTED); + } + @Override public TestType decode(InputStream inStream, Context context) throws CoderException, IOException { - Context nestedContext = context.nested(); VarIntCoder.of().decode(inStream); TestType result = jaxbCoder.decode(inStream); VarLongCoder.of().decode(inStream, context); From 9de1d0d0749c25dd270949da8991a6902b20baa6 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 5 May 2017 17:27:13 -0700 Subject: [PATCH 04/11] get it compiling --- .../runners/dataflow/BatchViewOverrides.java | 4 +-- .../apache/beam/sdk/transforms/Combine.java | 3 +- .../beam/sdk/transforms/join/CoGbkResult.java | 2 +- .../sdk/transforms/windowing/PaneInfo.java | 1 - .../apache/beam/sdk/util/WindowedValue.java | 3 +- .../beam/sdk/coders/CoderRegistryTest.java | 6 ++++ .../beam/sdk/testing/CoderPropertiesTest.java | 36 +++++++++---------- .../sdk/testing/SerializableMatchersTest.java | 1 - .../beam/sdk/transforms/CombineTest.java | 19 +++------- .../apache/beam/sdk/transforms/ParDoTest.java | 17 ++------- 10 files changed, 37 insertions(+), 55 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java index 34609dfecce3..d640f6effd00 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java @@ -1351,9 +1351,9 @@ public static TransformedMapCoder of( } @Override - public void encode(TransformedMap value, OutputStream outStream, OutputStream outStream) + public void encode(TransformedMap value, OutputStream outStream) throws CoderException, IOException { - encode(outStream, outStream, Coder.Context.NESTED); + encode(value, outStream, Coder.Context.NESTED); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java index 7e435647ed15..9e1cc7113b80 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java @@ -2001,7 +2001,8 @@ public void encode( } @Override - public InputOrAccum decode(InputStream inStream) throws CoderException, IOException { + public InputOrAccum decode(InputStream inStream) + throws CoderException, IOException { return decode(inStream, Coder.Context.NESTED); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java index d42de829c789..877bb077c78c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java @@ -264,7 +264,7 @@ public CoGbkResult decode( } List> valueMap = Lists.newArrayListWithExpectedSize(schema.size()); for (int unionTag = 0; unionTag < schema.size(); unionTag++) { - valueMap.add(tagListCoder(unionTag).decode(inStream, Coder.Context.NESTED)); + valueMap.add(tagListCoder(unionTag).decode(inStream)); } return new CoGbkResult(schema, valueMap); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java index 75df220ab873..1e9a187f6b6c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java @@ -27,7 +27,6 @@ import java.io.OutputStream; import java.util.Objects; import org.apache.beam.sdk.coders.AtomicCoder; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.WindowedContext; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java index 963886bee9d6..444521adc8f7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java @@ -646,8 +646,7 @@ public void encode(WindowedValue windowedElem, OutputStream outStream, Context context) throws CoderException, IOException { - InstantCoder.of().encode( - windowedElem.getTimestamp(), outStream, nestedContext); + InstantCoder.of().encode(windowedElem.getTimestamp(), outStream); windowsCoder.encode(windowedElem.getWindows(), outStream); PaneInfoCoder.INSTANCE.encode(windowedElem.getPane(), outStream); valueCoder.encode(windowedElem.getValue(), outStream, context); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java index c883ca021366..b199a06caa59 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java @@ -438,6 +438,12 @@ private static class AutoRegistrationClass { private static class AutoRegistrationClassCoder extends CustomCoder { private static final AutoRegistrationClassCoder INSTANCE = new AutoRegistrationClassCoder(); + + public void encode(AutoRegistrationClass value, OutputStream outStream) {} + + public AutoRegistrationClass decode(InputStream inStream) { + return null; + } } @Test diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CoderPropertiesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CoderPropertiesTest.java index 164d22131446..ce78411880b6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CoderPropertiesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CoderPropertiesTest.java @@ -50,15 +50,15 @@ public void testGoodCoderIsDeterministic() throws Exception { /** A coder that says it is not deterministic but actually is. */ public static class NonDeterministicCoder extends AtomicCoder { @Override - public void encode(String value, OutputStream outStream, Context context) + public void encode(String value, OutputStream outStream) throws CoderException, IOException { - StringUtf8Coder.of().encode(value, outStream, context); + StringUtf8Coder.of().encode(value, outStream); } @Override - public String decode(InputStream inStream, Context context) + public String decode(InputStream inStream) throws CoderException, IOException { - return StringUtf8Coder.of().decode(inStream, context); + return StringUtf8Coder.of().decode(inStream); } public void verifyDeterministic() throws NonDeterministicException { @@ -96,15 +96,15 @@ public BadDeterminsticCoder() { } @Override - public void encode(String value, OutputStream outStream, Context context) + public void encode(String value, OutputStream outStream) throws IOException, CoderException { - StringUtf8Coder.of().encode(value + System.nanoTime(), outStream, context); + StringUtf8Coder.of().encode(value + System.nanoTime(), outStream); } @Override - public String decode(InputStream inStream, Context context) + public String decode(InputStream inStream) throws CoderException, IOException { - return StringUtf8Coder.of().decode(inStream, context); + return StringUtf8Coder.of().decode(inStream); } @Override @@ -136,16 +136,16 @@ public StateChangingSerializingCoder() { } @Override - public void encode(String value, OutputStream outStream, Context context) + public void encode(String value, OutputStream outStream) throws CoderException, IOException { changedState += 1; - StringUtf8Coder.of().encode(value + Strings.repeat("A", changedState), outStream, context); + StringUtf8Coder.of().encode(value + Strings.repeat("A", changedState), outStream); } @Override - public String decode(InputStream inStream, Context context) + public String decode(InputStream inStream) throws CoderException, IOException { - String decodedValue = StringUtf8Coder.of().decode(inStream, context); + String decodedValue = StringUtf8Coder.of().decode(inStream); return decodedValue.substring(0, decodedValue.length() - changedState); } @@ -180,18 +180,18 @@ public ForgetfulSerializingCoder(int lostState) { } @Override - public void encode(String value, OutputStream outStream, Context context) + public void encode(String value, OutputStream outStream) throws CoderException, IOException { if (lostState == 0) { throw new RuntimeException("I forgot something..."); } - StringUtf8Coder.of().encode(value, outStream, context); + StringUtf8Coder.of().encode(value, outStream); } @Override - public String decode(InputStream inStream, Context context) + public String decode(InputStream inStream) throws CoderException, IOException { - return StringUtf8Coder.of().decode(inStream, context); + return StringUtf8Coder.of().decode(inStream); } @Override @@ -216,12 +216,12 @@ public void testBadCoderThatDependsOnStateThatIsLost() throws Exception { /** A coder which closes the underlying stream during encoding and decoding. */ public static class ClosingCoder extends AtomicCoder { @Override - public void encode(String value, OutputStream outStream, Context context) throws IOException { + public void encode(String value, OutputStream outStream) throws IOException { outStream.close(); } @Override - public String decode(InputStream inStream, Context context) throws IOException { + public String decode(InputStream inStream) throws IOException { inStream.close(); return null; } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java index 375be33944cb..6b176965620a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java @@ -30,7 +30,6 @@ import java.io.OutputStream; import java.io.Serializable; import org.apache.beam.sdk.coders.AtomicCoder; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.KV; import org.hamcrest.Matchers; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java index bd8aee405bf8..dc9788f69a1e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java @@ -876,28 +876,17 @@ public Coder getAccumulatorCoder( */ private class CountSumCoder extends AtomicCoder { @Override - public void encode(CountSum value, OutputStream outStream, OutputStream outStream) + public void encode(CountSum value, OutputStream outStream) throws CoderException, IOException { - encode(outStream, outStream, Context.NESTED); - } - - @Override - public void encode(CountSum value, OutputStream outStream, - Context context) throws CoderException, IOException { LONG_CODER.encode(value.count, outStream); - DOUBLE_CODER.encode(value.sum, outStream, context); - } - - @Override - public CountSum decode(InputStream inStream) throws CoderException, IOException { - return decode(inStream, Coder.Context.NESTED); + DOUBLE_CODER.encode(value.sum, outStream); } @Override - public CountSum decode(InputStream inStream, Coder.Context context) + public CountSum decode(InputStream inStream) throws CoderException, IOException { long count = LONG_CODER.decode(inStream); - double sum = DOUBLE_CODER.decode(inStream, context); + double sum = DOUBLE_CODER.decode(inStream); return new CountSum(count, sum); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java index 3697211091a4..ef27f4c03446 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java @@ -1092,24 +1092,13 @@ public static MyIntegerCoder of() { @Override public void encode(MyInteger value, OutputStream outStream) throws CoderException, IOException { - encode(value, outStream, Context.NESTED); + delegate.encode(value.getValue(), outStream); } @Override - public void encode(MyInteger value, OutputStream outStream, Context context) - throws CoderException, IOException { - delegate.encode(value.getValue(), outStream, context); - } - - @Override - public MyInteger decode(InputStream inStream) throws CoderException { - return decode(inStream, Context.NESTED); - } - - @Override - public MyInteger decode(InputStream inStream, Context context) throws CoderException, + public MyInteger decode(InputStream inStream) throws CoderException, IOException { - return new MyInteger(delegate.decode(inStream, context)); + return new MyInteger(delegate.decode(inStream)); } } From 4e7d79b3bb65077e2d80f20f0733857f91574ffe Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 5 May 2017 17:35:35 -0700 Subject: [PATCH 05/11] Remove en/decodeOuter and default encode/decode methods. Now only the context-free encode() and decode() methods are abstract. --- .../org/apache/beam/sdk/coders/Coder.java | 35 ++++---------- .../apache/beam/sdk/coders/CustomCoder.java | 47 ------------------- .../beam/sdk/coders/StructuredCoder.java | 47 ------------------- 3 files changed, 8 insertions(+), 121 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java index d140e896e0a0..ec8a72d02c89 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java @@ -125,18 +125,6 @@ public String toString() { public abstract void encode(T value, OutputStream outStream) throws CoderException, IOException; - /** - * Encodes the given value of type {@code T} onto the given output stream - * in the outer context. - * - * @throws IOException if writing to the {@code OutputStream} fails - * for some reason - * @throws CoderException if the value could not be encoded for some reason - */ - @Deprecated - public abstract void encodeOuter(T value, OutputStream outStream) - throws CoderException, IOException; - /** * Encodes the given value of type {@code T} onto the given output stream * in the given context. @@ -146,8 +134,10 @@ public abstract void encodeOuter(T value, OutputStream outStream) * @throws CoderException if the value could not be encoded for some reason */ @Deprecated - public abstract void encode(T value, OutputStream outStream, Context context) - throws CoderException, IOException; + public void encode(T value, OutputStream outStream, Context context) + throws CoderException, IOException { + encode(value, outStream); + } /** * Decodes a value of type {@code T} from the given input stream in @@ -159,17 +149,6 @@ public abstract void encode(T value, OutputStream outStream, Context context) */ public abstract T decode(InputStream inStream) throws CoderException, IOException; - /** - * Decodes a value of type {@code T} from the given input stream in - * the outer context. Returns the decoded value. - * - * @throws IOException if reading from the {@code InputStream} fails - * for some reason - * @throws CoderException if the value could not be decoded for some reason - */ - @Deprecated - public abstract T decodeOuter(InputStream inStream) throws CoderException, IOException; - /** * Decodes a value of type {@code T} from the given input stream in * the given context. Returns the decoded value. @@ -179,8 +158,10 @@ public abstract void encode(T value, OutputStream outStream, Context context) * @throws CoderException if the value could not be decoded for some reason */ @Deprecated - public abstract T decode(InputStream inStream, Context context) - throws CoderException, IOException; + public T decode(InputStream inStream, Context context) + throws CoderException, IOException { + return decode(inStream); + } /** * If this is a {@code Coder} for a parameterized type, returns the diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CustomCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CustomCoder.java index edbaa7f92fc8..c58192373432 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CustomCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CustomCoder.java @@ -17,9 +17,6 @@ */ package org.apache.beam.sdk.coders; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.io.Serializable; import java.util.Collections; import java.util.List; @@ -39,50 +36,6 @@ public abstract class CustomCoder extends Coder implements Serializable { - @Override - public void encode(T value, OutputStream outStream) - throws CoderException, IOException { - encode(value, outStream, Coder.Context.NESTED); - } - - @Deprecated - @Override - public void encodeOuter(T value, OutputStream outStream) - throws CoderException, IOException { - encode(value, outStream, Coder.Context.OUTER); - } - - @Deprecated - public void encode(T value, OutputStream outStream, Coder.Context context) - throws CoderException, IOException { - if (context == Coder.Context.NESTED) { - encode(value, outStream); - } else { - encodeOuter(value, outStream); - } - } - - @Override - public T decode(InputStream inStream) throws CoderException, IOException { - return decode(inStream, Coder.Context.NESTED); - } - - @Deprecated - @Override - public T decodeOuter(InputStream inStream) throws CoderException, IOException { - return decode(inStream, Coder.Context.OUTER); - } - - @Deprecated - public T decode(InputStream inStream, Coder.Context context) - throws CoderException, IOException { - if (context == Coder.Context.NESTED) { - return decode(inStream); - } else { - return decodeOuter(inStream); - } - } - /** * {@inheritDoc}. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuredCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuredCoder.java index 437f10d0b1c0..42c0598aa985 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuredCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuredCoder.java @@ -18,9 +18,6 @@ package org.apache.beam.sdk.coders; import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -101,50 +98,6 @@ public String toString() { return builder.toString(); } - @Override - public void encode(T value, OutputStream outStream) - throws CoderException, IOException { - encode(value, outStream, Coder.Context.NESTED); - } - - @Deprecated - @Override - public void encodeOuter(T value, OutputStream outStream) - throws CoderException, IOException { - encode(value, outStream, Coder.Context.OUTER); - } - - @Deprecated - public void encode(T value, OutputStream outStream, Coder.Context context) - throws CoderException, IOException { - if (context == Coder.Context.NESTED) { - encode(value, outStream); - } else { - encodeOuter(value, outStream); - } - } - - @Override - public T decode(InputStream inStream) throws CoderException, IOException { - return decode(inStream, Coder.Context.NESTED); - } - - @Deprecated - @Override - public T decodeOuter(InputStream inStream) throws CoderException, IOException { - return decode(inStream, Coder.Context.OUTER); - } - - @Deprecated - public T decode(InputStream inStream, Coder.Context context) - throws CoderException, IOException { - if (context == Coder.Context.NESTED) { - return decode(inStream); - } else { - return decodeOuter(inStream); - } - } - protected void verifyDeterministic(String message, Iterable> coders) throws NonDeterministicException { for (Coder coder : coders) { From d3456a68b5e4995a666787f8b4f192421083b77a Mon Sep 17 00:00:00 2001 From: Lukasz Cwik Date: Sun, 7 May 2017 19:41:07 -0700 Subject: [PATCH 06/11] fixup! Swap to use encode/decode in anonymous inner class coder and @AutoValue coder --- .../core/construction/PCollectionsTest.java | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionsTest.java index 2c45cbdb94b3..a114cf5b0612 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionsTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionsTest.java @@ -29,10 +29,7 @@ import java.util.Collection; import java.util.Collections; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.AtomicCoder; -import org.apache.beam.sdk.coders.BigEndianLongCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CustomCoder; +import org.apache.beam.sdk.coders.*; import org.apache.beam.sdk.common.runner.v1.RunnerApi; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.testing.TestPipeline; @@ -130,13 +127,13 @@ public void testEncodeDecodeCycle() throws Exception { @AutoValue abstract static class CustomIntCoder extends CustomCoder { @Override - public void encode(Integer value, OutputStream outStream, Context context) throws IOException { - VarInt.encode(value, outStream); + public Integer decode(InputStream inStream) throws IOException { + return VarInt.decodeInt(inStream); } @Override - public Integer decode(InputStream inStream, Context context) throws IOException { - return VarInt.decodeInt(inStream); + public void encode(Integer value, OutputStream outStream) throws IOException { + VarInt.encode(value, outStream); } } @@ -163,13 +160,13 @@ public Coder windowCoder() { @Override public void verifyDeterministic() {} @Override - public void encode(BoundedWindow value, OutputStream outStream, Context context) + public void encode(BoundedWindow value, OutputStream outStream) throws IOException { VarInt.encode(value.maxTimestamp().getMillis(), outStream); } @Override - public BoundedWindow decode(InputStream inStream, Context context) throws IOException { + public BoundedWindow decode(InputStream inStream) throws IOException { final Instant ts = new Instant(VarInt.decodeLong(inStream)); return new BoundedWindow() { @Override From 4064048868a626fb68920e2efe0adff60ffc0950 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 8 May 2017 10:48:40 -0700 Subject: [PATCH 07/11] Reviewer comments + a couple of extra fixes. All compiles. --- .../beam/runners/direct/DirectRunnerTest.java | 4 ++-- .../runners/dataflow/BatchViewOverrides.java | 16 ++-------------- .../runners/dataflow/internal/IsmFormat.java | 16 ++-------------- .../dataflow/util/CloudObjectsTest.java | 8 ++++---- .../org/apache/beam/sdk/io/FileBasedSink.java | 18 +++--------------- .../beam/sdk/coders/CoderRegistryTest.java | 2 ++ .../io/gcp/pubsub/PubsubUnboundedSource.java | 18 +++--------------- 7 files changed, 18 insertions(+), 64 deletions(-) diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java index 85e55eb3a30c..943d27c07ad1 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java @@ -526,11 +526,11 @@ public void testUnencodableOutputFromUnboundedRead() { private static class LongNoDecodeCoder extends AtomicCoder { @Override public void encode( - Long value, OutputStream outStream, Context context) throws IOException { + Long value, OutputStream outStream) throws IOException { } @Override - public Long decode(InputStream inStream, Context context) throws IOException { + public Long decode(InputStream inStream) throws IOException { throw new CoderException("Cannot decode a long"); } } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java index d640f6effd00..32a04c096095 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java @@ -1353,27 +1353,15 @@ public static TransformedMapCoder of( @Override public void encode(TransformedMap value, OutputStream outStream) throws CoderException, IOException { - encode(value, outStream, Coder.Context.NESTED); - } - - @Override - public void encode(TransformedMap value, OutputStream outStream, - Coder.Context context) throws CoderException, IOException { transformCoder.encode(value.transform, outStream); - originalMapCoder.encode(value.originalMap, outStream, context); + originalMapCoder.encode(value.originalMap, outStream); } @Override public TransformedMap decode(InputStream inStream) throws CoderException, IOException { - return decode(inStream, Coder.Context.NESTED); - } - - @Override - public TransformedMap decode( - InputStream inStream, Coder.Context context) throws CoderException, IOException { return new TransformedMap<>( transformCoder.decode(inStream), - originalMapCoder.decode(inStream, context)); + originalMapCoder.decode(inStream)); } @Override diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java index 8cfae816d217..0796d0805b23 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java @@ -622,32 +622,20 @@ private IsmShardCoder() {} @Override public void encode(IsmShard value, OutputStream outStream) throws CoderException, IOException { - encode(value, outStream, Coder.Context.NESTED); - } - - @Override - public void encode(IsmShard value, OutputStream outStream, Coder.Context context) - throws CoderException, IOException { checkState(value.getIndexOffset() >= 0, "%s attempting to be written without index offset.", value); VarIntCoder.of().encode(value.getId(), outStream); VarLongCoder.of().encode(value.getBlockOffset(), outStream); - VarLongCoder.of().encode(value.getIndexOffset(), outStream, context); + VarLongCoder.of().encode(value.getIndexOffset(), outStream); } @Override public IsmShard decode(InputStream inStream) throws CoderException, IOException { - return decode(inStream, Coder.Context.NESTED); - } - - @Override - public IsmShard decode( - InputStream inStream, Coder.Context context) throws CoderException, IOException { return IsmShard.of( VarIntCoder.of().decode(inStream), VarLongCoder.of().decode(inStream), - VarLongCoder.of().decode(inStream, context)); + VarLongCoder.of().decode(inStream)); } @Override diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java index 64c0dbd8170f..59a54316d8d5 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java @@ -171,12 +171,12 @@ private static class Record implements Serializable {} private static class ObjectCoder extends CustomCoder { @Override - public void encode(Object value, OutputStream outStream, Context context) + public void encode(Object value, OutputStream outStream) throws CoderException, IOException { } @Override - public Object decode(InputStream inStream, Context context) + public Object decode(InputStream inStream) throws CoderException, IOException { return new Object(); } @@ -197,11 +197,11 @@ public int hashCode() { */ private static class ArbitraryCoder extends StructuredCoder { @Override - public void encode(Record value, OutputStream outStream, Context context) + public void encode(Record value, OutputStream outStream) throws CoderException, IOException {} @Override - public Record decode(InputStream inStream, Context context) throws CoderException, IOException { + public Record decode(InputStream inStream) throws CoderException, IOException { return new Record(); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java index 3620c22737ec..32aa9c388d89 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java @@ -949,34 +949,22 @@ public static FileResultCoder of() { @Override public void encode(FileResult value, OutputStream outStream) throws IOException { - encode(value, outStream, Context.NESTED); - } - - @Override - public void encode(FileResult value, OutputStream outStream, Context context) - throws IOException { if (value == null) { throw new CoderException("cannot encode a null value"); } stringCoder.encode(value.getFilename().toString(), outStream); if (value.getDestinationFilename() == null) { - stringCoder.encode(null, outStream, context); + stringCoder.encode(null, outStream); } else { - stringCoder.encode(value.getDestinationFilename().toString(), outStream, context); + stringCoder.encode(value.getDestinationFilename().toString(), outStream); } } @Override public FileResult decode(InputStream inStream) throws IOException { - return decode(inStream, Context.NESTED); - } - - @Override - public FileResult decode(InputStream inStream, Context context) - throws IOException { String filename = stringCoder.decode(inStream); assert filename != null; // fixes a compiler warning - @Nullable String destinationFilename = stringCoder.decode(inStream, context); + @Nullable String destinationFilename = stringCoder.decode(inStream); return new FileResult( FileSystems.matchNewResource(filename, false /* isDirectory */), destinationFilename == null diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java index b199a06caa59..d1113f7aa938 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java @@ -439,8 +439,10 @@ private static class AutoRegistrationClass { private static class AutoRegistrationClassCoder extends CustomCoder { private static final AutoRegistrationClassCoder INSTANCE = new AutoRegistrationClassCoder(); + @Override public void encode(AutoRegistrationClass value, OutputStream outStream) {} + @Override public AutoRegistrationClass decode(InputStream inStream) { return null; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java index db8c1b74e66d..e8fe7011c121 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java @@ -371,28 +371,16 @@ private PubsubCheckpointCoder() {} @Override public void encode(PubsubCheckpoint value, OutputStream outStream) throws IOException { - encode(value, outStream, Context.NESTED); - } - - @Override - public void encode(PubsubCheckpoint value, OutputStream outStream, Context context) - throws IOException { SUBSCRIPTION_PATH_CODER.encode( value.subscriptionPath, - outStream, - context.nested()); - LIST_CODER.encode(value.notYetReadIds, outStream, context); + outStream); + LIST_CODER.encode(value.notYetReadIds, outStream); } @Override public PubsubCheckpoint decode(InputStream inStream) throws IOException { - return decode(inStream, Context.NESTED); - } - - @Override - public PubsubCheckpoint decode(InputStream inStream, Context context) throws IOException { String path = SUBSCRIPTION_PATH_CODER.decode(inStream); - List notYetReadIds = LIST_CODER.decode(inStream, context); + List notYetReadIds = LIST_CODER.decode(inStream); return new PubsubCheckpoint(path, null, null, notYetReadIds); } } From 45aae3b11d6dafdc5e599586d8d5370aaaeb8053 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 8 May 2017 11:27:40 -0700 Subject: [PATCH 08/11] lint error --- .../beam/runners/core/construction/PCollectionsTest.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionsTest.java index a114cf5b0612..c38dbc052468 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionsTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionsTest.java @@ -29,7 +29,10 @@ import java.util.Collection; import java.util.Collections; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.*; +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.BigEndianLongCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.common.runner.v1.RunnerApi; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.testing.TestPipeline; From 5160974cf7d462e3b638c2c97c1f52590ea84226 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 8 May 2017 11:54:48 -0700 Subject: [PATCH 09/11] checkstyle --- .../wrappers/streaming/SingletonKeyedWorkItemCoder.java | 3 ++- .../wrappers/streaming/state/FlinkKeyGroupStateInternals.java | 1 - .../org/apache/beam/runners/dataflow/BatchViewOverrides.java | 3 ++- .../org/apache/beam/runners/dataflow/internal/IsmFormat.java | 3 ++- .../fn/harness/data/BeamFnDataBufferingOutboundObserver.java | 1 - .../apache/beam/fn/harness/data/BeamFnDataInboundObserver.java | 1 - .../harness/data/BeamFnDataBufferingOutboundObserverTest.java | 1 - .../beam/fn/harness/data/BeamFnDataInboundObserverTest.java | 1 - 8 files changed, 6 insertions(+), 8 deletions(-) diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java index b62fc16fbe6c..2ed2055b489f 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java @@ -81,7 +81,8 @@ public void encode(SingletonKeyedWorkItem value, } @Override - public SingletonKeyedWorkItem decode(InputStream inStream) throws CoderException, IOException { + public SingletonKeyedWorkItem decode(InputStream inStream) + throws CoderException, IOException { return decode(inStream, Context.NESTED); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java index 8d437d56d573..512e4efc85a0 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java @@ -31,7 +31,6 @@ import org.apache.beam.runners.core.StateNamespace; import org.apache.beam.runners.core.StateTag; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java index 32a04c096095..b4a6e6470a4c 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java @@ -1358,7 +1358,8 @@ public void encode(TransformedMap value, OutputStream outStream) } @Override - public TransformedMap decode(InputStream inStream) throws CoderException, IOException { + public TransformedMap decode(InputStream inStream) + throws CoderException, IOException { return new TransformedMap<>( transformCoder.decode(inStream), originalMapCoder.decode(inStream)); diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java index 0796d0805b23..81ac2a29a80f 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java @@ -231,7 +231,8 @@ public Coder getValueCoder() { } @Override - public void encode(IsmRecord value, OutputStream outStream) throws CoderException, IOException { + public void encode(IsmRecord value, OutputStream outStream) + throws CoderException, IOException { if (value.getKeyComponents().size() != keyComponentCoders.size()) { throw new CoderException(String.format( "Expected %s key component(s) but received key component(s) %s.", diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserver.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserver.java index 37745be9cb9f..7223e876c896 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserver.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserver.java @@ -27,7 +27,6 @@ import org.apache.beam.fn.v1.BeamFnApi; import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserver.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserver.java index ece87d26c8e5..ac603bde8db9 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserver.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserver.java @@ -23,7 +23,6 @@ import org.apache.beam.fn.harness.fn.ThrowingConsumer; import org.apache.beam.fn.v1.BeamFnApi; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.util.WindowedValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserverTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserverTest.java index 3f6ece71063b..c2b45424466a 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserverTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserverTest.java @@ -33,7 +33,6 @@ import org.apache.beam.fn.v1.BeamFnApi; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.coders.LengthPrefixCoder; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.util.WindowedValue; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserverTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserverTest.java index 4b0bf0ce932b..54aba8be12b2 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserverTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserverTest.java @@ -34,7 +34,6 @@ import java.util.concurrent.ExecutionException; import org.apache.beam.fn.v1.BeamFnApi; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.WindowedValue; From 8876bc9afe6f303d2dcc9b50155355d305e73b27 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 8 May 2017 14:56:25 -0700 Subject: [PATCH 10/11] Use latest dataflow worker. From dbf36db66d24134ebf3c758793c6445f252855fe Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 8 May 2017 15:04:36 -0700 Subject: [PATCH 11/11] Explicitly mark Coder context as experimental as well as deprecated. --- .../java/org/apache/beam/sdk/annotations/Experimental.java | 3 +++ .../core/src/main/java/org/apache/beam/sdk/coders/Coder.java | 3 +++ 2 files changed, 6 insertions(+) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/annotations/Experimental.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/annotations/Experimental.java index 7255a01bd183..2e3a711cddfa 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/annotations/Experimental.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/annotations/Experimental.java @@ -84,6 +84,9 @@ enum Kind { /** Metrics-related experimental APIs. */ METRICS, + /** Experimental feature related to alternative, unnested encodings for coders. */ + CODER_CONTEXT, + /** Experimental runner APIs. Should not be used by pipeline authors. */ CORE_RUNNERS_ONLY, diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java index ec8a72d02c89..2ee532dfaa6e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java @@ -64,6 +64,7 @@ public abstract class Coder implements Serializable { /** The context in which encoding or decoding is being done. */ @Deprecated + @Experimental(Kind.CODER_CONTEXT) public static class Context { /** * The outer context: the value being encoded or decoded takes @@ -134,6 +135,7 @@ public abstract void encode(T value, OutputStream outStream) * @throws CoderException if the value could not be encoded for some reason */ @Deprecated + @Experimental(Kind.CODER_CONTEXT) public void encode(T value, OutputStream outStream, Context context) throws CoderException, IOException { encode(value, outStream); @@ -158,6 +160,7 @@ public void encode(T value, OutputStream outStream, Context context) * @throws CoderException if the value could not be decoded for some reason */ @Deprecated + @Experimental(Kind.CODER_CONTEXT) public T decode(InputStream inStream, Context context) throws CoderException, IOException { return decode(inStream);