From e446a21a021a3d2f837d7138bd1870892f3f50aa Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Thu, 4 May 2017 16:14:34 -0700 Subject: [PATCH] [BEAM-1871] Move out test only utilities used only in examples/java out of sdks/java/core. --- examples/java/pom.xml | 12 ++++++++++++ .../apache/beam/examples/WindowedWordCountIT.java | 8 ++++---- .../java/org/apache/beam/examples/WordCountIT.java | 2 +- .../beam/examples/testing}/ExplicitShardedFile.java | 3 ++- .../beam/examples}/testing/FileChecksumMatcher.java | 5 ++--- .../examples}/testing/FileChecksumMatcherTest.java | 3 ++- .../beam/examples/testing}/NumberedShardedFile.java | 3 ++- .../examples/testing}/NumberedShardedFileTest.java | 2 +- .../apache/beam/examples/testing}/ShardedFile.java | 2 +- .../apache/beam/sdk/coders/StructuralByteArray.java | 5 ++--- .../apache/beam/sdk/testing/MatcherDeserializer.java | 4 ++-- .../apache/beam/sdk/testing/MatcherSerializer.java | 4 ++-- .../java/org/apache/beam/sdk/util/CoderUtils.java | 8 +++++--- 13 files changed, 38 insertions(+), 23 deletions(-) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util => examples/java/src/test/java/org/apache/beam/examples/testing}/ExplicitShardedFile.java (98%) rename {sdks/java/core/src/main/java/org/apache/beam/sdk => examples/java/src/test/java/org/apache/beam/examples}/testing/FileChecksumMatcher.java (97%) rename {sdks/java/core/src/test/java/org/apache/beam/sdk => examples/java/src/test/java/org/apache/beam/examples}/testing/FileChecksumMatcherTest.java (98%) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util => examples/java/src/test/java/org/apache/beam/examples/testing}/NumberedShardedFile.java (98%) rename {sdks/java/core/src/test/java/org/apache/beam/sdk/util => examples/java/src/test/java/org/apache/beam/examples/testing}/NumberedShardedFileTest.java (99%) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util => examples/java/src/test/java/org/apache/beam/examples/testing}/ShardedFile.java (97%) diff --git a/examples/java/pom.xml b/examples/java/pom.xml index d673da27fe29..09473cd48827 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -491,6 +491,11 @@ google-auth-library-credentials + + com.google.code.findbugs + jsr305 + + org.apache.avro avro @@ -552,6 +557,13 @@ For testing the example itself, use the direct runner. This is separate from the use of ValidatesRunner tests for testing a particular runner. --> + + org.apache.beam + beam-sdks-java-core + tests + test + + org.apache.beam beam-runners-direct-java diff --git a/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java index b5eddb5d17e7..01bc4027f814 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java +++ b/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java @@ -33,21 +33,21 @@ import org.apache.beam.examples.common.ExampleUtils; import org.apache.beam.examples.common.WriteOneFilePerWindow.PerWindowFiles; +import org.apache.beam.examples.testing.ExplicitShardedFile; +import org.apache.beam.examples.testing.FileChecksumMatcher; +import org.apache.beam.examples.testing.NumberedShardedFile; +import org.apache.beam.examples.testing.ShardedFile; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.StreamingOptions; -import org.apache.beam.sdk.testing.FileChecksumMatcher; import org.apache.beam.sdk.testing.SerializableMatcher; import org.apache.beam.sdk.testing.StreamingIT; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestPipelineOptions; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; -import org.apache.beam.sdk.util.ExplicitShardedFile; import org.apache.beam.sdk.util.FluentBackoff; -import org.apache.beam.sdk.util.NumberedShardedFile; -import org.apache.beam.sdk.util.ShardedFile; import org.hamcrest.Description; import org.hamcrest.TypeSafeMatcher; import org.joda.time.Duration; diff --git a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java index 1660b61d1672..236ca9cb0780 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java +++ b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java @@ -20,10 +20,10 @@ import java.util.Date; import org.apache.beam.examples.WordCount.WordCountOptions; +import org.apache.beam.examples.testing.FileChecksumMatcher; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.FileChecksumMatcher; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestPipelineOptions; import org.junit.BeforeClass; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExplicitShardedFile.java b/examples/java/src/test/java/org/apache/beam/examples/testing/ExplicitShardedFile.java similarity index 98% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExplicitShardedFile.java rename to examples/java/src/test/java/org/apache/beam/examples/testing/ExplicitShardedFile.java index 0f184de7f8f4..1dc7a62d07f2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExplicitShardedFile.java +++ b/examples/java/src/test/java/org/apache/beam/examples/testing/ExplicitShardedFile.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.examples.testing; import com.google.api.client.util.BackOff; import com.google.api.client.util.BackOffUtils; @@ -35,6 +35,7 @@ import java.util.List; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.fs.MatchResult.Metadata; +import org.apache.beam.sdk.util.FluentBackoff; import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java b/examples/java/src/test/java/org/apache/beam/examples/testing/FileChecksumMatcher.java similarity index 97% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java rename to examples/java/src/test/java/org/apache/beam/examples/testing/FileChecksumMatcher.java index 82a6b7117636..8a0af1199f1e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java +++ b/examples/java/src/test/java/org/apache/beam/examples/testing/FileChecksumMatcher.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.beam.sdk.testing; +package org.apache.beam.examples.testing; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; @@ -31,9 +31,8 @@ import java.util.regex.Pattern; import javax.annotation.Nonnull; import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.testing.SerializableMatcher; import org.apache.beam.sdk.util.FluentBackoff; -import org.apache.beam.sdk.util.NumberedShardedFile; -import org.apache.beam.sdk.util.ShardedFile; import org.hamcrest.Description; import org.hamcrest.TypeSafeMatcher; import org.joda.time.Duration; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java b/examples/java/src/test/java/org/apache/beam/examples/testing/FileChecksumMatcherTest.java similarity index 98% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java rename to examples/java/src/test/java/org/apache/beam/examples/testing/FileChecksumMatcherTest.java index 4ee67501817c..4d6eb6bbef49 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/testing/FileChecksumMatcherTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.testing; +package org.apache.beam.examples.testing; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsString; @@ -27,6 +27,7 @@ import java.nio.charset.StandardCharsets; import java.util.regex.Pattern; import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java b/examples/java/src/test/java/org/apache/beam/examples/testing/NumberedShardedFile.java similarity index 98% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java rename to examples/java/src/test/java/org/apache/beam/examples/testing/NumberedShardedFile.java index e18dd96f0046..f0b9c2d6045f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java +++ b/examples/java/src/test/java/org/apache/beam/examples/testing/NumberedShardedFile.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.examples.testing; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; @@ -44,6 +44,7 @@ import javax.annotation.Nonnull; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.fs.MatchResult.Metadata; +import org.apache.beam.sdk.util.FluentBackoff; import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/NumberedShardedFileTest.java b/examples/java/src/test/java/org/apache/beam/examples/testing/NumberedShardedFileTest.java similarity index 99% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/util/NumberedShardedFileTest.java rename to examples/java/src/test/java/org/apache/beam/examples/testing/NumberedShardedFileTest.java index 43a91665c9c3..83b8a4f136d4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/NumberedShardedFileTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/testing/NumberedShardedFileTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.examples.testing; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ShardedFile.java b/examples/java/src/test/java/org/apache/beam/examples/testing/ShardedFile.java similarity index 97% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/ShardedFile.java rename to examples/java/src/test/java/org/apache/beam/examples/testing/ShardedFile.java index ec9ed641976c..cd9537cb326d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ShardedFile.java +++ b/examples/java/src/test/java/org/apache/beam/examples/testing/ShardedFile.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.examples.testing; import com.google.api.client.util.BackOff; import com.google.api.client.util.Sleeper; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuralByteArray.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuralByteArray.java index 226f79cd429d..0ab0dea9bdab 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuralByteArray.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuralByteArray.java @@ -17,8 +17,7 @@ */ package org.apache.beam.sdk.coders; -import static com.google.api.client.util.Base64.encodeBase64String; - +import com.google.common.io.BaseEncoding; import java.util.Arrays; /** @@ -53,6 +52,6 @@ public int hashCode() { @Override public String toString() { - return "base64:" + encodeBase64String(value); + return "base64:" + BaseEncoding.base64().encode(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..e7aa5a7f4283 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 @@ -22,7 +22,7 @@ import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.JsonDeserializer; import com.fasterxml.jackson.databind.node.ObjectNode; -import com.google.api.client.util.Base64; +import com.google.common.io.BaseEncoding; import java.io.IOException; import org.apache.beam.sdk.util.SerializableUtils; @@ -36,7 +36,7 @@ public SerializableMatcher deserialize(JsonParser jsonParser, throws IOException, JsonProcessingException { ObjectNode node = jsonParser.readValueAsTree(); String matcher = node.get("matcher").asText(); - byte[] in = Base64.decodeBase64(matcher); + byte[] in = BaseEncoding.base64().decode(matcher); return (SerializableMatcher) SerializableUtils .deserializeFromByteArray(in, "SerializableMatcher"); } 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..35375f66047d 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 @@ -21,7 +21,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.JsonSerializer; import com.fasterxml.jackson.databind.SerializerProvider; -import com.google.api.client.util.Base64; +import com.google.common.io.BaseEncoding; import java.io.IOException; import org.apache.beam.sdk.util.SerializableUtils; @@ -33,7 +33,7 @@ class MatcherSerializer extends JsonSerializer> { public void serialize(SerializableMatcher matcher, JsonGenerator jsonGenerator, SerializerProvider serializerProvider) throws IOException, JsonProcessingException { byte[] out = SerializableUtils.serializeToByteArray(matcher); - String encodedString = Base64.encodeBase64String(out); + String encodedString = BaseEncoding.base64().encode(out); jsonGenerator.writeStartObject(); jsonGenerator.writeStringField("matcher", encodedString); jsonGenerator.writeEndObject(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java index 3380a102e68a..e3ae485f5d18 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java @@ -17,8 +17,8 @@ */ package org.apache.beam.sdk.util; -import com.google.api.client.util.Base64; import com.google.common.base.Throwables; +import com.google.common.io.BaseEncoding; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -163,7 +163,7 @@ public static T clone(Coder coder, T value) throws CoderException { public static String encodeToBase64(Coder coder, T value) throws CoderException { byte[] rawValue = encodeToByteArray(coder, value); - return Base64.encodeBase64URLSafeString(rawValue); + return BaseEncoding.base64Url().omitPadding().encode(rawValue); } /** @@ -171,7 +171,9 @@ public static String encodeToBase64(Coder coder, T value) */ public static T decodeFromBase64(Coder coder, String encodedValue) throws CoderException { return decodeFromSafeStream( - coder, new ByteArrayInputStream(Base64.decodeBase64(encodedValue)), Coder.Context.OUTER); + coder, + new ByteArrayInputStream(BaseEncoding.base64Url().omitPadding().decode(encodedValue)), + Coder.Context.OUTER); } /**