From e94dedc6a21cd9e5bde7c99e69cc1d23f3f9268c Mon Sep 17 00:00:00 2001 From: "wtanaka.com" Date: Sat, 29 Apr 2017 19:20:59 -1000 Subject: [PATCH] Clean up exception signatures Strengthen some "throws Exception" declarations CoderException extends IOException so it's not necessary to declare both in "throws" --- .../apache/beam/sdk/coders/BigDecimalCoder.java | 4 ++-- .../beam/sdk/coders/BigEndianIntegerCoder.java | 4 ++-- .../beam/sdk/coders/BigEndianLongCoder.java | 4 ++-- .../apache/beam/sdk/coders/BigIntegerCoder.java | 4 ++-- .../org/apache/beam/sdk/coders/BitSetCoder.java | 4 ++-- .../apache/beam/sdk/coders/ByteArrayCoder.java | 6 +++--- .../org/apache/beam/sdk/coders/ByteCoder.java | 4 ++-- .../java/org/apache/beam/sdk/coders/Coder.java | 4 ++-- .../apache/beam/sdk/coders/DelegateCoder.java | 6 +++--- .../org/apache/beam/sdk/coders/DoubleCoder.java | 4 ++-- .../apache/beam/sdk/coders/DurationCoder.java | 4 ++-- .../org/apache/beam/sdk/coders/InstantCoder.java | 4 ++-- .../beam/sdk/coders/IterableLikeCoder.java | 4 ++-- .../java/org/apache/beam/sdk/coders/KvCoder.java | 4 ++-- .../beam/sdk/coders/LengthPrefixCoder.java | 4 ++-- .../org/apache/beam/sdk/coders/MapCoder.java | 4 ++-- .../apache/beam/sdk/coders/NullableCoder.java | 4 ++-- .../beam/sdk/coders/SerializableCoder.java | 4 ++-- .../beam/sdk/coders/StringDelegateCoder.java | 4 ++-- .../beam/sdk/coders/TextualIntegerCoder.java | 4 ++-- .../org/apache/beam/sdk/coders/VarIntCoder.java | 4 ++-- .../org/apache/beam/sdk/coders/VarLongCoder.java | 4 ++-- .../beam/sdk/options/ProxyInvocationHandler.java | 4 ++-- .../apache/beam/sdk/options/ValueProvider.java | 2 +- .../apache/beam/sdk/testing/CoderProperties.java | 12 +++++------- .../beam/sdk/testing/MatcherDeserializer.java | 2 +- .../beam/sdk/testing/MatcherSerializer.java | 2 +- .../sdk/transforms/ApproximateQuantiles.java | 8 ++++---- .../beam/sdk/transforms/ApproximateUnique.java | 2 +- .../org/apache/beam/sdk/transforms/Combine.java | 8 ++++---- .../apache/beam/sdk/transforms/CombineFns.java | 4 ++-- .../org/apache/beam/sdk/transforms/Count.java | 2 +- .../org/apache/beam/sdk/transforms/Create.java | 2 +- .../org/apache/beam/sdk/transforms/Mean.java | 4 ++-- .../java/org/apache/beam/sdk/transforms/Top.java | 4 ++-- .../beam/sdk/transforms/join/CoGbkResult.java | 6 +++--- .../beam/sdk/transforms/join/UnionCoder.java | 4 ++-- .../sdk/transforms/windowing/IntervalWindow.java | 4 ++-- .../beam/sdk/transforms/windowing/PaneInfo.java | 4 ++-- .../org/apache/beam/sdk/util/BitSetCoder.java | 4 ++-- .../org/apache/beam/sdk/util/WindowedValue.java | 8 ++++---- .../apache/beam/sdk/coders/AvroCoderTest.java | 3 ++- .../beam/sdk/coders/CoderRegistryTest.java | 8 ++++---- .../beam/sdk/coders/NullableCoderTest.java | 2 +- .../beam/sdk/coders/StringDelegateCoderTest.java | 5 ++++- .../beam/sdk/coders/StructuredCoderTest.java | 12 ++++++------ .../apache/beam/sdk/io/FileBasedSinkTest.java | 10 +++++----- .../sdk/options/PipelineOptionsFactoryTest.java | 4 ++-- .../beam/sdk/testing/CoderPropertiesTest.java | 16 ++++++++-------- .../org/apache/beam/sdk/testing/PAssertTest.java | 4 ++-- .../beam/sdk/testing/WindowSupplierTest.java | 4 ++-- .../beam/sdk/transforms/CombineFnsTest.java | 4 ++-- .../apache/beam/sdk/transforms/CombineTest.java | 8 ++++---- .../apache/beam/sdk/transforms/CreateTest.java | 8 ++++---- .../apache/beam/sdk/transforms/ParDoTest.java | 10 +++++----- .../org/apache/beam/sdk/transforms/ViewTest.java | 4 ++-- .../beam/sdk/util/SerializableUtilsTest.java | 4 ++-- 57 files changed, 143 insertions(+), 141 deletions(-) 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 d628203a48c2..76a80721bcc8 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 @@ -49,7 +49,7 @@ private BigDecimalCoder() {} @Override public void encode(BigDecimal value, OutputStream outStream, Context context) - throws IOException, CoderException { + throws IOException { checkNotNull(value, String.format("cannot encode a null %s", BigDecimal.class.getSimpleName())); VAR_INT_CODER.encode(value.scale(), outStream, context.nested()); BIG_INT_CODER.encode(value.unscaledValue(), outStream, context); @@ -57,7 +57,7 @@ public void encode(BigDecimal value, OutputStream outStream, Context context) @Override public BigDecimal decode(InputStream inStream, Context context) - throws IOException, CoderException { + throws IOException { int scale = VAR_INT_CODER.decode(inStream, context.nested()); 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/BigEndianIntegerCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianIntegerCoder.java index 81c5e9471ad6..2b1d0f596ccb 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 @@ -44,7 +44,7 @@ private BigEndianIntegerCoder() {} @Override public void encode(Integer value, OutputStream outStream, Context context) - throws IOException, CoderException { + throws IOException { 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) - throws IOException, CoderException { + throws IOException { try { return new DataInputStream(inStream).readInt(); } catch (EOFException | UTFDataFormatException exn) { 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 173e91020427..10320e2ac153 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 @@ -46,7 +46,7 @@ private BigEndianLongCoder() {} @Override public void encode(Long value, OutputStream outStream, Context context) - throws IOException, CoderException { + throws IOException { if (value == null) { throw new CoderException("cannot encode a null Long"); } @@ -55,7 +55,7 @@ public void encode(Long value, OutputStream outStream, Context context) @Override public Long decode(InputStream inStream, Context context) - throws IOException, CoderException { + throws IOException { try { return new DataInputStream(inStream).readLong(); } catch (EOFException | UTFDataFormatException exn) { 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 a739da78aa17..cc725e0794c7 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 @@ -43,14 +43,14 @@ private BigIntegerCoder() {} @Override public void encode(BigInteger value, OutputStream outStream, Context context) - throws IOException, CoderException { + throws IOException { checkNotNull(value, String.format("cannot encode a null %s", BigInteger.class.getSimpleName())); BYTE_ARRAY_CODER.encode(value.toByteArray(), outStream, context); } @Override public BigInteger decode(InputStream inStream, Context context) - throws IOException, CoderException { + throws IOException { return new BigInteger(BYTE_ARRAY_CODER.decode(inStream, context)); } 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 5a4db24697c5..58b84849d18d 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 @@ -37,7 +37,7 @@ public static BitSetCoder of() { @Override public void encode(BitSet value, OutputStream outStream, Context context) - throws CoderException, IOException { + throws IOException { if (value == null) { throw new CoderException("cannot encode a null BitSet"); } @@ -46,7 +46,7 @@ public void encode(BitSet value, OutputStream outStream, Context context) @Override public BitSet decode(InputStream inStream, Context context) - throws CoderException, IOException { + throws IOException { return BitSet.valueOf(BYTE_ARRAY_CODER.decode(inStream, context)); } 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 cba8d49b4c14..068eb95cd7eb 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 @@ -63,7 +63,7 @@ private ByteArrayCoder() {} @Override public void encode(byte[] value, OutputStream outStream, Context context) - throws IOException, CoderException { + throws IOException { if (value == null) { throw new CoderException("cannot encode a null byte[]"); } @@ -82,7 +82,7 @@ public void encode(byte[] value, OutputStream outStream, Context context) *

Once passed to this method, {@code value} should never be observed or mutated again. */ public void encodeAndOwn(byte[] value, OutputStream outStream, Context context) - throws IOException, CoderException { + throws IOException { if (!context.isWholeStream) { VarInt.encode(value.length, outStream); outStream.write(value); @@ -97,7 +97,7 @@ public void encodeAndOwn(byte[] value, OutputStream outStream, Context context) @Override public byte[] decode(InputStream inStream, Context context) - throws IOException, CoderException { + throws IOException { if (context.isWholeStream) { return StreamUtils.getBytes(inStream); } else { 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 1a1be641169a..30cfb8ef445b 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 @@ -42,7 +42,7 @@ private ByteCoder() {} @Override public void encode(Byte value, OutputStream outStream, Context context) - throws IOException, CoderException { + throws IOException { if (value == null) { throw new CoderException("cannot encode a null Byte"); } @@ -51,7 +51,7 @@ public void encode(Byte value, OutputStream outStream, Context context) @Override public Byte decode(InputStream inStream, Context context) - throws IOException, CoderException { + throws IOException { try { // value will be between 0-255, -1 for EOF int value = inStream.read(); 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 632cf8900661..295a48163413 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 @@ -120,7 +120,7 @@ public String toString() { * @throws CoderException if the value could not be encoded for some reason */ void encode(T value, OutputStream outStream, Context context) - throws CoderException, IOException; + throws IOException; /** * Decodes a value of type {@code T} from the given input stream in @@ -131,7 +131,7 @@ void encode(T value, OutputStream outStream, Context context) * @throws CoderException if the value could not be decoded for some reason */ T decode(InputStream inStream, Context context) - throws CoderException, IOException; + throws IOException; /** * 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/DelegateCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DelegateCoder.java index 86077eb120ce..a5df9bfe4085 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 @@ -67,12 +67,12 @@ public static DelegateCoder of( @Override public void encode(T value, OutputStream outStream, Context context) - throws CoderException, IOException { + throws IOException { coder.encode(applyAndWrapExceptions(toFn, value), outStream, context); } @Override - public T decode(InputStream inStream, Context context) throws CoderException, IOException { + public T decode(InputStream inStream, Context context) throws IOException { return applyAndWrapExceptions(fromFn, coder.decode(inStream, context)); } @@ -155,7 +155,7 @@ private String delegateEncodingId(Class delegateClass, String encodingId) { private OutputT applyAndWrapExceptions( CodingFunction fn, - InputT input) throws CoderException, IOException { + InputT input) throws IOException { try { return fn.apply(input); } catch (IOException exc) { 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 06e7daef2712..bb62fb73b509 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 @@ -44,7 +44,7 @@ private DoubleCoder() {} @Override public void encode(Double value, OutputStream outStream, Context context) - throws IOException, CoderException { + throws IOException { if (value == null) { throw new CoderException("cannot encode a null Double"); } @@ -53,7 +53,7 @@ public void encode(Double value, OutputStream outStream, Context context) @Override public Double decode(InputStream inStream, Context context) - throws IOException, CoderException { + throws IOException { try { return new DataInputStream(inStream).readDouble(); } catch (EOFException | UTFDataFormatException exn) { 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 10a83ef94c34..e101bafaa185 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 @@ -55,7 +55,7 @@ private ReadableDuration fromLong(Long decoded) { @Override public void encode(ReadableDuration value, OutputStream outStream, Context context) - throws CoderException, IOException { + throws IOException { if (value == null) { throw new CoderException("cannot encode a null ReadableDuration"); } @@ -64,7 +64,7 @@ public void encode(ReadableDuration value, OutputStream outStream, Context conte @Override public ReadableDuration decode(InputStream inStream, Context context) - throws CoderException, IOException { + throws IOException { return fromLong(LONG_CODER.decode(inStream, context)); } 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 cfd1979545a3..aa6e6a96c5c7 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 @@ -69,7 +69,7 @@ protected Instant doBackward(Long shiftedMillis) { @Override public void encode(Instant value, OutputStream outStream, Context context) - throws CoderException, IOException { + throws IOException { if (value == null) { throw new CoderException("cannot encode a null Instant"); } @@ -78,7 +78,7 @@ public void encode(Instant value, OutputStream outStream, Context context) @Override public Instant decode(InputStream inStream, Context context) - throws CoderException, IOException { + throws IOException { return ORDER_PRESERVING_CONVERTER.reverse().convert(LONG_CODER.decode(inStream, context)); } 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 8e10ca25cdd7..8d6c857f9041 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 @@ protected IterableLikeCoder(Coder elementCoder, String iterableName) { @Override public void encode( IterableT iterable, OutputStream outStream, Context context) - throws IOException, CoderException { + throws IOException { if (iterable == null) { throw new CoderException("cannot encode a null " + iterableName); } @@ -130,7 +130,7 @@ public void encode( @Override public IterableT decode(InputStream inStream, Context context) - throws IOException, CoderException { + throws IOException { Context nestedContext = context.nested(); DataInputStream dataInStream = new DataInputStream(inStream); int size = dataInStream.readInt(); 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 3d813b6a556a..99d05b698f94 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 @@ -81,7 +81,7 @@ private KvCoder(Coder keyCoder, Coder valueCoder) { @Override public void encode(KV kv, OutputStream outStream, Context context) - throws IOException, CoderException { + throws IOException { if (kv == null) { throw new CoderException("cannot encode a null KV"); } @@ -91,7 +91,7 @@ public void encode(KV kv, OutputStream outStream, Context context) @Override public KV decode(InputStream inStream, Context context) - throws IOException, CoderException { + throws IOException { K key = keyCoder.decode(inStream, context.nested()); 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/LengthPrefixCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/LengthPrefixCoder.java index 0972b1e31994..cdf11e7e9412 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 @@ -73,7 +73,7 @@ protected CloudObject initializeCloudObject() { @Override public void encode(T value, OutputStream outStream, Context context) - throws CoderException, IOException { + throws IOException { ByteArrayOutputStream bos = new ByteArrayOutputStream(); valueCoder.encode(value, bos, Context.OUTER); VarInt.encode(bos.size(), outStream); @@ -81,7 +81,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, Context context) throws 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 e2c4e28c1e52..b3b3c9aff2d8 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 @@ -85,7 +85,7 @@ public void encode( Map map, OutputStream outStream, Context context) - throws IOException, CoderException { + throws IOException { if (map == null) { throw new CoderException("cannot encode a null Map"); } @@ -113,7 +113,7 @@ public void encode( @Override public Map decode(InputStream inStream, Context context) - throws IOException, CoderException { + throws IOException { 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/NullableCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java index 747d91c441c4..d7ae7252f69e 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 @@ -62,7 +62,7 @@ public Coder getValueCoder() { @Override public void encode(@Nullable T value, OutputStream outStream, Context context) - throws IOException, CoderException { + throws IOException { if (value == null) { outStream.write(ENCODE_NULL); } else { @@ -73,7 +73,7 @@ public void encode(@Nullable T value, OutputStream outStream, Context context) @Override @Nullable - public T decode(InputStream inStream, Context context) throws IOException, CoderException { + public T decode(InputStream inStream, Context context) throws IOException { int b = inStream.read(); if (b == ENCODE_NULL) { return null; 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 b52b9dba114b..99141ce9b0ee 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 @@ -101,7 +101,7 @@ public Class getRecordType() { @Override public void encode(T value, OutputStream outStream, Context context) - throws IOException, CoderException { + throws IOException { try { ObjectOutputStream oos = new ObjectOutputStream(outStream); oos.writeObject(value); @@ -113,7 +113,7 @@ public void encode(T value, OutputStream outStream, Context context) @Override public T decode(InputStream inStream, Context context) - throws IOException, CoderException { + throws IOException { try { ObjectInputStream ois = new ObjectInputStream(inStream); return type.cast(ois.readObject()); 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 39a16587cc94..227278d51df3 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 @@ -101,12 +101,12 @@ public int hashCode() { @Override public void encode(T value, OutputStream outStream, Context context) - throws CoderException, IOException { + throws IOException { delegateCoder.encode(value, outStream, context); } @Override - public T decode(InputStream inStream, Context context) throws CoderException, IOException { + public T decode(InputStream inStream, Context context) throws IOException { return delegateCoder.decode(inStream, context); } 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 91a46ea69093..9b3971b431c9 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 @@ -40,7 +40,7 @@ protected TextualIntegerCoder() {} @Override public void encode(Integer value, OutputStream outStream, Context context) - throws IOException, CoderException { + throws IOException { if (value == null) { throw new CoderException("cannot encode a null Integer"); } @@ -50,7 +50,7 @@ public void encode(Integer value, OutputStream outStream, Context context) @Override public Integer decode(InputStream inStream, Context context) - throws IOException, CoderException { + throws IOException { String textualValue = StringUtf8Coder.of().decode(inStream, context); try { return Integer.valueOf(textualValue); 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 fcc0335b1fcf..a45c66862ad7 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 @@ -45,7 +45,7 @@ private VarIntCoder() {} @Override public void encode(Integer value, OutputStream outStream, Context context) - throws IOException, CoderException { + throws IOException { if (value == null) { throw new CoderException("cannot encode a null Integer"); } @@ -54,7 +54,7 @@ public void encode(Integer value, OutputStream outStream, Context context) @Override public Integer decode(InputStream inStream, Context context) - throws IOException, CoderException { + throws IOException { try { return VarInt.decodeInt(inStream); } catch (EOFException | UTFDataFormatException exn) { 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 7fc094f755b9..ea49e5849eed 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 @@ -53,7 +53,7 @@ private VarLongCoder() {} @Override public void encode(Long value, OutputStream outStream, Context context) - throws IOException, CoderException { + throws IOException { if (value == null) { throw new CoderException("cannot encode a null Long"); } @@ -62,7 +62,7 @@ public void encode(Long value, OutputStream outStream, Context context) @Override public Long decode(InputStream inStream, Context context) - throws IOException, CoderException { + throws IOException { try { return VarInt.decodeLong(inStream); } catch (EOFException | UTFDataFormatException exn) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java index eda21a8aadb3..80d4a09f6288 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java @@ -618,7 +618,7 @@ private static Map generateSettersToPropertyNames( static class Serializer extends JsonSerializer { @Override public void serialize(PipelineOptions value, JsonGenerator jgen, SerializerProvider provider) - throws IOException, JsonProcessingException { + throws IOException { ProxyInvocationHandler handler = (ProxyInvocationHandler) Proxy.getInvocationHandler(value); synchronized (handler) { // We first filter out any properties that have been modified since @@ -718,7 +718,7 @@ private void ensureSerializable(Set> interfaces static class Deserializer extends JsonDeserializer { @Override public PipelineOptions deserialize(JsonParser jp, DeserializationContext ctxt) - throws IOException, JsonProcessingException { + throws IOException { ObjectNode objectNode = (ObjectNode) jp.readValueAsTree(); ObjectNode optionsNode = (ObjectNode) objectNode.get("options"); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java index 030eed595939..114f7310dbf9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java @@ -312,7 +312,7 @@ public JsonDeserializer createContextual(DeserializationContext ctxt, @Override public ValueProvider deserialize(JsonParser jp, DeserializationContext ctxt) - throws IOException, JsonProcessingException { + throws IOException { JsonDeserializer dser = ctxt.findRootValueDeserializer( checkNotNull(innerType, "Invalid %s: innerType is null. Serialization error?", getClass())); Object o = dser.deserialize(jp, ctxt); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java index bd6d86a86ccb..0d7b98d9e82f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java @@ -69,8 +69,7 @@ public class CoderProperties { * {@code Coder.Context}. */ public static void coderDeterministic( - Coder coder, T value1, T value2) - throws Exception { + Coder coder, T value1, T value2) throws IOException { for (Coder.Context context : ALL_CONTEXTS) { coderDeterministicInContext(coder, context, value1, value2); } @@ -81,8 +80,7 @@ public static void coderDeterministic( * type {@code T}, if the values are equal then the encoded bytes are equal. */ public static void coderDeterministicInContext( - Coder coder, Coder.Context context, T value1, T value2) - throws Exception { + Coder coder, Coder.Context context, T value1, T value2) throws IOException { try { coder.verifyDeterministic(); } catch (NonDeterministicException e) { @@ -350,7 +348,7 @@ public static > void coderDecodesBase64Contents @VisibleForTesting static byte[] encode( - Coder coder, Coder.Context context, T value) throws CoderException, IOException { + Coder coder, Coder.Context context, T value) throws IOException { @SuppressWarnings("unchecked") Coder deserializedCoder = Serializer.deserialize(coder.asCloudObject(), Coder.class); @@ -361,7 +359,7 @@ static byte[] encode( @VisibleForTesting static T decode( - Coder coder, Coder.Context context, byte[] bytes) throws CoderException, IOException { + Coder coder, Coder.Context context, byte[] bytes) throws IOException { @SuppressWarnings("unchecked") Coder deserializedCoder = Serializer.deserialize(coder.asCloudObject(), Coder.class); @@ -382,7 +380,7 @@ static T decode( } private static T decodeEncode(Coder coder, Coder.Context context, T value) - throws CoderException, IOException { + throws IOException { return decode(coder, context, encode(coder, context, value)); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java index 6ca07badac92..8fdefaf69ba6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java @@ -33,7 +33,7 @@ class MatcherDeserializer extends JsonDeserializer> { @Override public SerializableMatcher deserialize(JsonParser jsonParser, DeserializationContext deserializationContext) - throws IOException, JsonProcessingException { + throws IOException { ObjectNode node = jsonParser.readValueAsTree(); String matcher = node.get("matcher").asText(); byte[] in = Base64.decodeBase64(matcher); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java index 2b4584c8cdac..cc6983264569 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java @@ -31,7 +31,7 @@ class MatcherSerializer extends JsonSerializer> { @Override public void serialize(SerializableMatcher matcher, JsonGenerator jsonGenerator, - SerializerProvider serializerProvider) throws IOException, JsonProcessingException { + SerializerProvider serializerProvider) throws IOException { byte[] out = SerializableUtils.serializeToByteArray(matcher); String encodedString = Base64.encodeBase64String(out); jsonGenerator.writeStartObject(); 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 ed3a253e1016..913b28f4a237 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 @@ -683,7 +683,7 @@ public QuantileStateCoder(ComparatorT compareFn, Coder elementCoder) { @Override public void encode( QuantileState state, OutputStream outStream, Coder.Context context) - throws CoderException, IOException { + throws IOException { Coder.Context nestedContext = context.nested(); intCoder.encode(state.numQuantiles, outStream, nestedContext); intCoder.encode(state.bufferSize, outStream, nestedContext); @@ -700,7 +700,7 @@ public void encode( @Override public QuantileState decode(InputStream inStream, Coder.Context context) - throws CoderException, IOException { + throws IOException { Coder.Context nestedContext = context.nested(); int numQuantiles = intCoder.decode(inStream, nestedContext); int bufferSize = intCoder.decode(inStream, nestedContext); @@ -720,7 +720,7 @@ public QuantileState decode(InputStream inStream, Coder.Context private void encodeBuffer( QuantileBuffer buffer, OutputStream outStream, Coder.Context context) - throws CoderException, IOException { + throws IOException { DataOutputStream outData = new DataOutputStream(outStream); outData.writeInt(buffer.level); outData.writeLong(buffer.weight); @@ -729,7 +729,7 @@ private void encodeBuffer( private QuantileBuffer decodeBuffer( InputStream inStream, Coder.Context context) - throws IOException, CoderException { + throws IOException { DataInputStream inData = new DataInputStream(inStream); return new QuantileBuffer<>( inData.readInt(), diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java index 33820e05d540..107b6e05e08f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java @@ -436,7 +436,7 @@ public Coder getAccumulatorCoder(CoderRegistry registry, /** * Encodes the given element using the given coder and hashes the encoding. */ - static long hash(T element, Coder coder) throws CoderException, IOException { + static long hash(T element, Coder coder) throws IOException { try (HashingOutputStream stream = new HashingOutputStream(Hashing.murmur3_128(), ByteStreams.nullOutputStream())) { coder.encode(element, stream, Context.OUTER); 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 5ffaef83b742..8f67e91008ac 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 @@ -656,7 +656,7 @@ public HolderCoder(Coder valueCoder) { @Override public void encode(Holder accumulator, OutputStream outStream, Context context) - throws CoderException, IOException { + throws IOException { if (accumulator.present) { outStream.write(1); valueCoder.encode(accumulator.value, outStream, context); @@ -667,7 +667,7 @@ public void encode(Holder accumulator, OutputStream outStream, Context contex @Override public Holder decode(InputStream inStream, Context context) - throws CoderException, IOException { + throws IOException { if (inStream.read() == 1) { return new Holder<>(valueCoder.decode(inStream, context)); } else { @@ -2287,7 +2287,7 @@ public InputOrAccumCoder(Coder inputCoder, Coder accumCoder) { @Override public void encode( InputOrAccum value, OutputStream outStream, Coder.Context context) - throws CoderException, IOException { + throws IOException { if (value.input != null) { outStream.write(0); inputCoder.encode(value.input, outStream, context); @@ -2299,7 +2299,7 @@ public void encode( @Override public InputOrAccum decode(InputStream inStream, Coder.Context context) - throws CoderException, IOException { + throws IOException { if (inStream.read() == 0) { return InputOrAccum.input(inputCoder.decode(inStream, context)); } else { 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 ca939c154041..f6a8462db56b 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 @@ -981,7 +981,7 @@ public ComposedAccumulatorCoder(List> coders) { @Override public void encode(Object[] value, OutputStream outStream, Context context) - throws CoderException, IOException { + throws IOException { checkArgument(value.length == codersCount); if (value.length == 0) { return; @@ -996,7 +996,7 @@ public void encode(Object[] value, OutputStream outStream, Context context) @Override public Object[] decode(InputStream inStream, Context context) - throws CoderException, IOException { + throws IOException { Object[] ret = new Object[codersCount]; if (codersCount == 0) { return ret; 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 78a6cd13fe72..3813e3a51e3a 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 @@ -176,7 +176,7 @@ public void encode(long[] value, OutputStream outStream, Context context) @Override public long[] decode(InputStream inStream, Context context) - throws IOException, CoderException { + throws IOException { try { return new long[] {VarInt.decodeLong(inStream)}; } catch (EOFException | UTFDataFormatException exn) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java index 5624f2f2cbe4..7de0bddf5296 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java @@ -356,7 +356,7 @@ static class CreateSource extends OffsetBasedSource { private final Coder coder; public static CreateSource fromIterable(Iterable elements, Coder elemCoder) - throws CoderException, IOException { + throws IOException { ImmutableList.Builder allElementsBytes = ImmutableList.builder(); long totalSize = 0L; for (T element : elements) { 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 a6808cf15013..47e5ac68fe03 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,14 +187,14 @@ static class CountSumCoder @Override public void encode(CountSum value, OutputStream outStream, Coder.Context context) - throws CoderException, IOException { + throws IOException { LONG_CODER.encode(value.count, outStream, context.nested()); DOUBLE_CODER.encode(value.sum, outStream, context); } @Override public CountSum decode(InputStream inStream, Coder.Context context) - throws CoderException, IOException { + throws IOException { return new CountSum<>( LONG_CODER.decode(inStream, context.nested()), DOUBLE_CODER.decode(inStream, context)); 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 47be9b9c7a3b..a069cbac4694 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) - throws CoderException, IOException { + throws IOException { listCoder.encode(value.asList(), outStream, context); } @Override public BoundedHeap decode(InputStream inStream, Coder.Context context) - throws CoderException, IOException { + throws IOException { return new BoundedHeap<>(maximumSize, compareFn, listCoder.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 6c62cbe73cc0..bff8250585fe 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 @@ -235,8 +235,8 @@ public List> getCoderArguments() { public void encode( CoGbkResult value, OutputStream outStream, - Context context) throws CoderException, - IOException { + Context context) throws + IOException { if (!schema.equals(value.getSchema())) { throw new CoderException("input schema does not match coder schema"); } @@ -254,7 +254,7 @@ public void encode( public CoGbkResult decode( InputStream inStream, Context context) - throws CoderException, IOException { + throws 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 f411cd1333b4..2588e8f0884f 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 @@ -60,7 +60,7 @@ public void encode( RawUnionValue union, OutputStream outStream, Context context) - throws IOException, CoderException { + throws IOException { int index = getIndexForEncoding(union); // Write out the union tag. VarInt.encode(index, outStream); @@ -75,7 +75,7 @@ public void encode( @Override public RawUnionValue decode(InputStream inStream, Context context) - throws IOException, CoderException { + throws IOException { int index = VarInt.decodeInt(inStream); Object value = elementCoders.get(index).decode(inStream, context); return new RawUnionValue(index, value); 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 55bf585ae149..ac8364371f94 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 @@ -188,14 +188,14 @@ public static List getInstanceComponents(T value) { @Override public void encode(IntervalWindow window, OutputStream outStream, Context context) - throws IOException, CoderException { + throws IOException { instantCoder.encode(window.end, outStream, context.nested()); durationCoder.encode(new Duration(window.start, window.end), outStream, context); } @Override public IntervalWindow decode(InputStream inStream, Context context) - throws IOException, CoderException { + throws IOException { Instant end = instantCoder.decode(inStream, context.nested()); 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/transforms/windowing/PaneInfo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java index b43c7a3a7836..8eb34419fd41 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 @@ -341,7 +341,7 @@ private Encoding chooseEncoding(PaneInfo value) { @Override public void encode(PaneInfo value, final OutputStream outStream, Coder.Context context) - throws CoderException, IOException { + throws IOException { Encoding encoding = chooseEncoding(value); switch (chooseEncoding(value)) { case FIRST: @@ -363,7 +363,7 @@ public void encode(PaneInfo value, final OutputStream outStream, Coder.Context c @Override public PaneInfo decode(final InputStream inStream, Coder.Context context) - throws CoderException, IOException { + throws IOException { byte keyAndTag = (byte) inStream.read(); PaneInfo base = BYTE_TO_PANE_INFO.get((byte) (keyAndTag & 0x0F)); long index, onTimeIndex; 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 b0e9b5c50254..59935c380764 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,13 +42,13 @@ public static BitSetCoder of() { @Override public void encode(BitSet value, OutputStream outStream, Context context) - throws CoderException, IOException { + throws IOException { BYTE_ARRAY_CODER.encodeAndOwn(value.toByteArray(), outStream, context); } @Override public BitSet decode(InputStream inStream, Context context) - throws CoderException, IOException { + throws IOException { return BitSet.valueOf(BYTE_ARRAY_CODER.decode(inStream, context)); } 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 fc9a404c435e..1cc0050e7f98 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 @@ -654,7 +654,7 @@ public WindowedValueCoder withValueCoder(Coder valueCoder) { public void encode(WindowedValue windowedElem, OutputStream outStream, Context context) - throws CoderException, IOException { + throws IOException { Context nestedContext = context.nested(); InstantCoder.of().encode( windowedElem.getTimestamp(), outStream, nestedContext); @@ -665,7 +665,7 @@ public void encode(WindowedValue windowedElem, @Override public WindowedValue decode(InputStream inStream, Context context) - throws CoderException, IOException { + throws IOException { Context nestedContext = context.nested(); Instant timestamp = InstantCoder.of().decode(inStream, nestedContext); Collection windows = @@ -744,13 +744,13 @@ public WindowedValueCoder withValueCoder(Coder valueCoder) { @Override public void encode(WindowedValue windowedElem, OutputStream outStream, Context context) - throws CoderException, IOException { + throws IOException { valueCoder.encode(windowedElem.getValue(), outStream, context); } @Override public WindowedValue decode(InputStream inStream, Context context) - throws CoderException, IOException { + throws IOException { T value = valueCoder.decode(inStream, context); return WindowedValue.valueInGlobalWindow(value); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java index cbc4d24a0ad8..20f5ea7ee6c1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java @@ -28,6 +28,7 @@ import com.esotericsoftware.kryo.io.Output; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; +import java.io.IOException; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; import java.util.ArrayList; @@ -635,7 +636,7 @@ private static class HasCustomSchema { @Test public void testAvroCoderTreeMapDeterminism() - throws Exception, NonDeterministicException { + throws NonDeterministicException, IOException { TreeMapField size1 = new TreeMapField(); TreeMapField size2 = new TreeMapField(); 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 fe21a1c2bca5..e2b22f2a1041 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 @@ -102,12 +102,12 @@ public void testRegisterInstantiatedCoder() throws Exception { private class MyListCoder extends CustomCoder { @Override public void encode(List value, OutputStream outStream, Context context) - throws CoderException, IOException { + throws IOException { } @Override public List decode(InputStream inStream, Context context) - throws CoderException, IOException { + throws IOException { return Collections.emptyList(); } @@ -458,12 +458,12 @@ public static List getInstanceComponents( @Override public void encode(MyValue value, OutputStream outStream, Context context) - throws CoderException, IOException { + throws IOException { } @Override public MyValue decode(InputStream inStream, Context context) - throws CoderException, IOException { + throws IOException { return new MyValue(); } 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 894d2d194236..1054234860d7 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 @@ -179,7 +179,7 @@ public void encode( @Override public String decode(InputStream inStream, Context context) - throws CoderException, IOException { + throws IOException { checkArgument(context.isWholeStream, "Expected to get entire stream"); return StringUtf8Coder.of().decode(inStream, context); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringDelegateCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringDelegateCoderTest.java index e972f0786839..b203c029ce44 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringDelegateCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringDelegateCoderTest.java @@ -20,7 +20,9 @@ import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; +import java.io.IOException; import java.net.URI; +import java.net.URISyntaxException; import java.util.Arrays; import java.util.List; import org.apache.beam.sdk.coders.Coder.NonDeterministicException; @@ -48,7 +50,8 @@ public class StringDelegateCoderTest { // Tests @Test - public void testDeterministic() throws Exception, NonDeterministicException { + public void testDeterministic() + throws NonDeterministicException, URISyntaxException, IOException { uriCoder.verifyDeterministic(); for (String uriString : TEST_URI_STRINGS) { CoderProperties.coderDeterministic(uriCoder, new URI(uriString), new URI(uriString)); 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..86becea09d8d 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 @@ -48,7 +48,7 @@ private static class NullBooleanCoder extends StructuredCoder { @Override public void encode(@Nullable Boolean value, OutputStream outStream, Context context) - throws CoderException, IOException { + throws IOException { if (value == null) { outStream.write(2); } else if (value) { @@ -62,7 +62,7 @@ public void encode(@Nullable Boolean value, OutputStream outStream, Context cont @Nullable public Boolean decode( InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context) - throws CoderException, IOException { + throws IOException { int value = inStream.read(); if (value == 0) { return false; @@ -111,7 +111,7 @@ private static class ObjectIdentityBooleanCoder extends StructuredCoder extends StructuredCoder { @Override public void encode(T value, OutputStream outStream, Coder.Context context) - throws CoderException, IOException { + throws IOException { throw new UnsupportedOperationException(); } @Override public T decode(InputStream inStream, Coder.Context context) - throws CoderException, IOException { + throws IOException { throw new UnsupportedOperationException(); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java index 7efe47c70e0a..635172d535ac 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java @@ -455,7 +455,7 @@ public void testGenerateOutputFilenamesWithoutExtension() { * {@link CompressionType#BZIP2} correctly writes BZip2 data. */ @Test - public void testCompressionTypeBZIP2() throws FileNotFoundException, IOException { + public void testCompressionTypeBZIP2() throws IOException { final File file = writeValuesWithWritableByteChannelFactory(CompressionType.BZIP2, "abc", "123"); // Read Bzip2ed data back in using Apache commons API (de facto standard). @@ -468,7 +468,7 @@ public void testCompressionTypeBZIP2() throws FileNotFoundException, IOException * {@link CompressionType#GZIP} correctly writes Gzipped data. */ @Test - public void testCompressionTypeGZIP() throws FileNotFoundException, IOException { + public void testCompressionTypeGZIP() throws IOException { final File file = writeValuesWithWritableByteChannelFactory(CompressionType.GZIP, "abc", "123"); // Read Gzipped data back in using standard API. assertReadValues(new BufferedReader(new InputStreamReader( @@ -480,7 +480,7 @@ public void testCompressionTypeGZIP() throws FileNotFoundException, IOException * {@link CompressionType#DEFLATE} correctly writes deflate data. */ @Test - public void testCompressionTypeDEFLATE() throws FileNotFoundException, IOException { + public void testCompressionTypeDEFLATE() throws IOException { final File file = writeValuesWithWritableByteChannelFactory( CompressionType.DEFLATE, "abc", "123"); // Read Gzipped data back in using standard API. @@ -492,7 +492,7 @@ public void testCompressionTypeDEFLATE() throws FileNotFoundException, IOExcepti * {@link CompressionType#UNCOMPRESSED} correctly writes uncompressed data. */ @Test - public void testCompressionTypeUNCOMPRESSED() throws FileNotFoundException, IOException { + public void testCompressionTypeUNCOMPRESSED() throws IOException { final File file = writeValuesWithWritableByteChannelFactory(CompressionType.UNCOMPRESSED, "abc", "123"); // Read uncompressed data back in using standard API. @@ -511,7 +511,7 @@ private void assertReadValues(final BufferedReader br, String... values) throws private File writeValuesWithWritableByteChannelFactory(final WritableByteChannelFactory factory, String... values) - throws IOException, FileNotFoundException { + throws IOException { final File file = tmpFolder.newFile("test.gz"); final WritableByteChannel channel = factory.create(Channels.newChannel(new FileOutputStream(file))); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java index 11bd7b90d3f2..4a0ca56d9849 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java @@ -1722,7 +1722,7 @@ public static class JacksonIncompatibleDeserializer extends @Override public JacksonIncompatible deserialize(JsonParser jsonParser, - DeserializationContext deserializationContext) throws IOException, JsonProcessingException { + DeserializationContext deserializationContext) throws IOException { return new JacksonIncompatible(jsonParser.readValueAs(String.class)); } } @@ -1732,7 +1732,7 @@ public static class JacksonIncompatibleSerializer extends JsonSerializer { @Override public void encode(String value, OutputStream outStream, Context context) - throws CoderException, IOException { + throws IOException { StringUtf8Coder.of().encode(value, outStream, context); } @Override public String decode(InputStream inStream, Context context) - throws CoderException, IOException { + throws IOException { return StringUtf8Coder.of().decode(inStream, context); } } @@ -90,13 +90,13 @@ public BadDeterminsticCoder() { @Override public void encode(String value, OutputStream outStream, Context context) - throws IOException, CoderException { + throws IOException { StringUtf8Coder.of().encode(value + System.nanoTime(), outStream, context); } @Override public String decode(InputStream inStream, Context context) - throws CoderException, IOException { + throws IOException { return StringUtf8Coder.of().decode(inStream, context); } @@ -130,14 +130,14 @@ public StateChangingSerializingCoder() { @Override public void encode(String value, OutputStream outStream, Context context) - throws CoderException, IOException { + throws IOException { changedState += 1; StringUtf8Coder.of().encode(value + Strings.repeat("A", changedState), outStream, context); } @Override public String decode(InputStream inStream, Context context) - throws CoderException, IOException { + throws IOException { String decodedValue = StringUtf8Coder.of().decode(inStream, context); return decodedValue.substring(0, decodedValue.length() - changedState); } @@ -163,7 +163,7 @@ public ForgetfulSerializingCoder(int lostState) { @Override public void encode(String value, OutputStream outStream, Context context) - throws CoderException, IOException { + throws IOException { if (lostState == 0) { throw new RuntimeException("I forgot something..."); } @@ -172,7 +172,7 @@ public void encode(String value, OutputStream outStream, Context context) @Override public String decode(InputStream inStream, Context context) - throws CoderException, IOException { + throws IOException { return StringUtf8Coder.of().decode(inStream, context); } } 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 cfe743664a9b..7d4517e3f2c3 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 @@ -99,12 +99,12 @@ public static NotSerializableObjectCoder of() { @Override public void encode(NotSerializableObject value, OutputStream outStream, Context context) - throws CoderException, IOException { + throws IOException { } @Override public NotSerializableObject decode(InputStream inStream, Context context) - throws CoderException, IOException { + throws IOException { return new NotSerializableObject(); } 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 38a2fa2375bc..feb732e1ed58 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 @@ -76,13 +76,13 @@ private static class FailingCoder extends CustomCoder { @Override public void encode( BoundedWindow value, OutputStream outStream, Context context) - throws CoderException, IOException { + throws IOException { throw new CoderException("Test Enccode Exception"); } @Override public BoundedWindow decode( - InputStream inStream, Context context) throws CoderException, IOException { + InputStream inStream, Context context) throws 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 13c5f1693cc9..5a29806faddb 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 @@ -354,13 +354,13 @@ public static UserStringCoder of() { @Override public void encode(UserString value, OutputStream outStream, Context context) - throws CoderException, IOException { + throws IOException { StringUtf8Coder.of().encode(value.strValue, outStream, context); } @Override public UserString decode(InputStream inStream, Context context) - throws CoderException, IOException { + throws IOException { return UserString.of(StringUtf8Coder.of().decode(inStream, context)); } 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 a5f3df2dee95..49cd941e5e1b 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,14 +875,14 @@ public Coder getAccumulatorCoder( private class CountSumCoder extends CustomCoder { @Override public void encode(CountSum value, OutputStream outStream, - Context context) throws CoderException, IOException { + Context context) throws IOException { LONG_CODER.encode(value.count, outStream, context.nested()); DOUBLE_CODER.encode(value.sum, outStream, context); } @Override public CountSum decode(InputStream inStream, Coder.Context context) - throws CoderException, IOException { + throws IOException { long count = LONG_CODER.decode(inStream, context.nested()); double sum = DOUBLE_CODER.decode(inStream, context); return new CountSum(count, sum); @@ -928,13 +928,13 @@ public static Coder getCoder() { return new CustomCoder() { @Override public void encode(Accumulator accumulator, OutputStream outStream, Coder.Context context) - throws CoderException, IOException { + throws IOException { StringUtf8Coder.of().encode(accumulator.value, outStream, context); } @Override public Accumulator decode(InputStream inStream, Coder.Context context) - throws CoderException, IOException { + throws IOException { return new Accumulator(StringUtf8Coder.of().decode(inStream, context)); } }; 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 89a1f332704a..669498674cef 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 @@ -135,10 +135,10 @@ static class Record2 extends Record { private static class RecordCoder extends CustomCoder { @Override public void encode(Record value, OutputStream outStream, Context context) - throws CoderException, IOException {} + throws IOException {} @Override - public Record decode(InputStream inStream, Context context) throws CoderException, IOException { + public Record decode(InputStream inStream, Context context) throws IOException { return null; } } @@ -209,14 +209,14 @@ public void encode( UnserializableRecord value, OutputStream outStream, org.apache.beam.sdk.coders.Coder.Context context) - throws CoderException, IOException { + throws IOException { stringCoder.encode(value.myString, outStream, context.nested()); } @Override public UnserializableRecord decode( InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context) - throws CoderException, IOException { + throws IOException { return new UnserializableRecord(stringCoder.decode(inStream, context.nested())); } } 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 1a976f22d088..091269b8bc8a 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 @@ -994,12 +994,12 @@ public static List getInstanceComponents(TestDummy exampleValue) { @Override public void encode(TestDummy value, OutputStream outStream, Context context) - throws CoderException, IOException { + throws IOException { } @Override public TestDummy decode(InputStream inStream, Context context) - throws CoderException, IOException { + throws IOException { return new TestDummy(); } @@ -1098,13 +1098,13 @@ public static MyIntegerCoder of() { @Override public void encode(MyInteger value, OutputStream outStream, Context context) - throws CoderException, IOException { + throws IOException { delegate.encode(value.getValue(), outStream, context); } @Override - public MyInteger decode(InputStream inStream, Context context) throws CoderException, - IOException { + public MyInteger decode(InputStream inStream, Context context) throws + IOException { return new MyInteger(delegate.decode(inStream, context)); } } 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 867fe0a6445f..dcfcac8655d3 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 @@ -508,13 +508,13 @@ public void processElement(ProcessContext c) { private static class NonDeterministicStringCoder extends CustomCoder { @Override public void encode(String value, OutputStream outStream, Coder.Context context) - throws CoderException, IOException { + throws IOException { StringUtf8Coder.of().encode(value, outStream, context); } @Override public String decode(InputStream inStream, Coder.Context context) - throws CoderException, IOException { + throws IOException { return StringUtf8Coder.of().decode(inStream, context); } 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 2d5baf24a16b..6c7086ed7780 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 @@ -90,12 +90,12 @@ private static class UnserializableCoderByJava extends CustomCoder { @Override public void encode(Object value, OutputStream outStream, Context context) - throws CoderException, IOException { + throws IOException { } @Override public Object decode(InputStream inStream, Context context) - throws CoderException, IOException { + throws IOException { return unserializableField; }