Skip to content

Commit 5e471d7

Browse files
authored
Bump Beam to 2.30.0, update transitives (spotify#3880)
* Bump Beam to 2.30.0, update transitives * add confluent resolver * fix AppliedPTransform ResourceHints breakage * exclude kafka globally * fix coder breakage * fix sparkey breakage * scalafix * fix non-unique test pipeline names
1 parent 16e3538 commit 5e471d7

File tree

6 files changed

+37
-26
lines changed

6 files changed

+37
-26
lines changed

build.sbt

+12-9
Original file line numberDiff line numberDiff line change
@@ -33,10 +33,10 @@ val autoServiceVersion = "1.0"
3333
val autoValueVersion = "1.8.1"
3434
val avroVersion = "1.8.2"
3535
val beamVendorVersion = "0.1"
36-
val beamVersion = "2.29.0"
36+
val beamVersion = "2.30.0"
3737
val bigdataossVersion = "2.1.6"
38-
val bigQueryStorageVersion = "1.12.0"
39-
val bigtableClientVersion = "1.16.0"
38+
val bigQueryStorageVersion = "1.18.1"
39+
val bigtableClientVersion = "1.19.1"
4040
val breezeVersion = "1.2"
4141
val caffeineVersion = "2.9.1"
4242
val caseappVersion = "2.0.6"
@@ -58,16 +58,16 @@ val gcsVersion = "1.8.0"
5858
val generatedGrpcBetaVersion = "1.19.2"
5959
val generatedDatastoreProtoVersion = "0.88.5"
6060
val googleClientsVersion = "1.31.1"
61-
val googleApiServicesBigQueryVersion = s"v2-rev20210219-1.31.0"
62-
val googleApiServicesDataflowVersion = s"v1b3-rev20210217-1.31.0"
63-
val googleApiServicesPubsubVersion = s"v1-rev20210208-1.31.0"
61+
val googleApiServicesBigQueryVersion = s"v2-rev20210410-1.31.0"
62+
val googleApiServicesDataflowVersion = s"v1b3-rev20210408-1.31.0"
63+
val googleApiServicesPubsubVersion = s"v1-rev20210322-1.31.0"
6464
val googleApiServicesStorageVersion = s"v1-rev20210127-1.31.0"
6565
val googleAuthVersion = "0.22.2"
6666
val googleCloudCoreVersion = "1.94.0"
6767
val googleCloudSpannerVersion = "3.2.1"
6868
val googleHttpClientsVersion = "1.38.1"
6969
val googleOauthClientVersion = "1.31.2"
70-
val grpcVersion = "1.35.0"
70+
val grpcVersion = "1.37.0"
7171
val guavaVersion = "30.1-jre"
7272
val hadoopVersion = "2.10.1"
7373
val hamcrestVersion = "2.2"
@@ -87,7 +87,7 @@ val magnoliaVersion = "0.17.0"
8787
val magnolifyVersion = "0.4.3"
8888
val metricsVersion = "3.2.6"
8989
val nettyVersion = "4.1.52.Final"
90-
val nettyTcNativeVersion = "2.0.33.Final"
90+
val nettyTcNativeVersion = "2.0.34.Final"
9191
val opencensusVersion = "0.28.0"
9292
val parquetExtraVersion = "0.4.0"
9393
val parquetVersion = "1.12.0"
@@ -149,7 +149,10 @@ val commonSettings = Def
149149
javacOptions ++= Seq("-source", "1.8", "-target", "1.8", "-Xlint:unchecked"),
150150
Compile / doc / javacOptions := Seq("-source", "1.8"),
151151
// protobuf-lite is an older subset of protobuf-java and causes issues
152-
excludeDependencies += "com.google.protobuf" % "protobuf-lite",
152+
excludeDependencies ++= Seq(
153+
"com.google.protobuf" % "protobuf-lite",
154+
"org.apache.beam" % "beam-sdks-java-io-kafka"
155+
),
153156
resolvers += Resolver.sonatypeRepo("public"),
154157
Test / javaOptions += "-Dscio.ignoreVersionWarning=true",
155158
Test / testOptions += Tests.Argument("-oD"),

scio-core/src/main/scala/com/spotify/scio/coders/BeamCoders.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -98,7 +98,7 @@ private[scio] object BeamCoders {
9898

9999
/** Get key-value coders from a `SideInput[Map[K, Iterable[V]]]`. */
100100
def getMultiMapKV[K, V](si: SideInput[Map[K, Iterable[V]]]): (Coder[K], Coder[V]) = {
101-
val coder = si.view.getPCollection.getCoder.asInstanceOf[beam.KvCoder[_, _]].getValueCoder
101+
val coder = si.view.getPCollection.getCoder
102102
val (k, v) = unwrap(coder) match {
103103
// Beam's `View.asMultiMap`
104104
case (c: beam.KvCoder[K, V] @unchecked) => (c.getKeyCoder, c.getValueCoder)

scio-core/src/main/scala/com/spotify/scio/runners/dataflow/DataflowResult.scala

+2
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@ import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions
2727
import org.apache.beam.runners.dataflow.{DataflowClient, DataflowPipelineJob}
2828
import org.apache.beam.sdk.options.PipelineOptionsFactory
2929
import org.apache.beam.sdk.runners.AppliedPTransform
30+
import org.apache.beam.sdk.transforms.resourcehints.ResourceHints
3031
import org.apache.beam.sdk.transforms.PTransform
3132
import org.apache.beam.sdk.values.{PInput, POutput}
3233
import org.apache.beam.sdk.{Pipeline, PipelineResult}
@@ -148,6 +149,7 @@ object DataflowResult {
148149
Collections.emptyMap(),
149150
Collections.emptyMap(),
150151
new EmptyPTransform,
152+
ResourceHints.create(),
151153
new EmptyPipeline
152154
)
153155

scio-extra/src/test/scala/com/spotify/scio/extra/sparkey/SparkeyTest.scala

+4-9
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,6 @@ import com.spotify.scio.testing._
2727
import com.spotify.scio.util._
2828
import com.spotify.sparkey._
2929
import org.apache.beam.sdk.io.FileSystems
30-
import org.apache.beam.sdk.values.KV
3130
import org.apache.commons.io.FileUtils
3231

3332
import scala.jdk.CollectionConverters._
@@ -567,8 +566,7 @@ class SparkeyTest extends PipelineSpec {
567566
.tap(sparkeyMaterialized)
568567
.value
569568
.next
570-
.asInstanceOf[KV[Any, SparkeyUri]]
571-
.getValue
569+
.asInstanceOf[SparkeyUri]
572570
.basePath
573571
FileUtils.deleteDirectory(new File(basePath))
574572
}
@@ -600,8 +598,7 @@ class SparkeyTest extends PipelineSpec {
600598
.tap(sparkeyMaterialized)
601599
.value
602600
.next
603-
.asInstanceOf[KV[Any, SparkeyUri]]
604-
.getValue
601+
.asInstanceOf[SparkeyUri]
605602
.basePath
606603
FileUtils.deleteDirectory(new File(basePath))
607604
}
@@ -632,8 +629,7 @@ class SparkeyTest extends PipelineSpec {
632629
.tap(sparkeyMaterialized)
633630
.value
634631
.next
635-
.asInstanceOf[KV[Any, SparkeyUri]]
636-
.getValue
632+
.asInstanceOf[SparkeyUri]
637633
.basePath
638634
FileUtils.deleteDirectory(new File(basePath))
639635
}
@@ -664,8 +660,7 @@ class SparkeyTest extends PipelineSpec {
664660
.tap(sparkeyMaterialized)
665661
.value
666662
.next
667-
.asInstanceOf[KV[Any, SparkeyUri]]
668-
.getValue
663+
.asInstanceOf[SparkeyUri]
669664
.basePath
670665
FileUtils.deleteDirectory(new File(basePath))
671666
}

scio-smb/src/test/java/org/apache/beam/sdk/extensions/smb/SortedBucketSinkTest.java

+15-5
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@
3030
import java.util.HashMap;
3131
import java.util.List;
3232
import java.util.Map;
33+
import java.util.UUID;
3334
import java.util.function.Consumer;
3435
import java.util.stream.Collectors;
3536
import java.util.stream.IntStream;
@@ -156,7 +157,9 @@ public void testCleansUpTempFiles() throws Exception {
156157
1,
157158
0);
158159

159-
pipeline.apply(Create.of(Stream.of(input).collect(Collectors.toList()))).apply(sink);
160+
pipeline
161+
.apply("CleansUpTempFiles", Create.of(Stream.of(input).collect(Collectors.toList())))
162+
.apply(sink);
160163
pipeline.run().waitUntilFinish();
161164

162165
// Assert that no files are left in the temp directory
@@ -173,7 +176,7 @@ public void testCustomFilenamePrefix() throws Exception {
173176
new SortedBucketSink<>(
174177
metadata, outputDirectory, fromFolder(temp), ".txt", new TestFileOperations(), 1);
175178

176-
pipeline.apply(Create.empty(StringUtf8Coder.of())).apply(sink);
179+
pipeline.apply("CustomFilenamePrefix", Create.empty(StringUtf8Coder.of())).apply(sink);
177180
pipeline.run().waitUntilFinish();
178181

179182
final MatchResult outputFiles =
@@ -198,7 +201,7 @@ public void testWritesEmptyBucketFiles() throws Exception {
198201
new SortedBucketSink<>(
199202
metadata, outputDirectory, fromFolder(temp), ".txt", new TestFileOperations(), 1);
200203

201-
pipeline.apply(Create.empty(StringUtf8Coder.of())).apply(sink);
204+
pipeline.apply("WritesEmptyBucketFiles", Create.empty(StringUtf8Coder.of())).apply(sink);
202205
pipeline.run().waitUntilFinish();
203206

204207
final FileAssignment dstFiles =
@@ -233,7 +236,11 @@ public void testWritesNoFilesIfPriorStepsFail() throws Exception {
233236
new ExceptionThrowingFileOperations(),
234237
1);
235238

236-
pipeline.apply(Create.of(Stream.of(input).collect(Collectors.toList()))).apply(sink);
239+
pipeline
240+
.apply(
241+
"WritesNoFilesIfPriorStepsFail",
242+
Create.of(Stream.of(input).collect(Collectors.toList())))
243+
.apply(sink);
237244

238245
try {
239246
pipeline.run();
@@ -263,7 +270,9 @@ private void test(int numBuckets, int numShards, boolean useKeyCache) throws Exc
263270

264271
check(
265272
pipeline
266-
.apply(Create.of(Stream.of(input).collect(Collectors.toList())))
273+
.apply(
274+
"test-" + UUID.randomUUID(),
275+
Create.of(Stream.of(input).collect(Collectors.toList())))
267276
.apply(reshuffle)
268277
.apply(sink),
269278
metadata,
@@ -298,6 +307,7 @@ private void testKeyedCollection(int numBuckets, int numShards, boolean useKeyCa
298307
check(
299308
pipeline
300309
.apply(
310+
"test-keyed-collection-" + UUID.randomUUID(),
301311
Create.of(keyedInput)
302312
.withCoder(
303313
KvCoder.of(NullableCoder.of(StringUtf8Coder.of()), StringUtf8Coder.of())))

scio-smb/src/test/java/org/apache/beam/sdk/extensions/smb/SortedBucketSourceTest.java

+3-2
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@
3232
import java.util.List;
3333
import java.util.Map;
3434
import java.util.Set;
35+
import java.util.UUID;
3536
import java.util.function.Function;
3637
import java.util.function.ToIntFunction;
3738
import java.util.stream.Collectors;
@@ -42,11 +43,9 @@
4243
import org.apache.beam.sdk.extensions.smb.FileOperations.Writer;
4344
import org.apache.beam.sdk.extensions.smb.SMBFilenamePolicy.FileAssignment;
4445
import org.apache.beam.sdk.extensions.smb.SortedBucketSource.Predicate;
45-
import org.apache.beam.sdk.io.BoundedSource;
4646
import org.apache.beam.sdk.io.FileSystems;
4747
import org.apache.beam.sdk.io.LocalResources;
4848
import org.apache.beam.sdk.io.Read;
49-
import org.apache.beam.sdk.io.fs.ResolveOptions;
5049
import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions;
5150
import org.apache.beam.sdk.io.fs.ResourceId;
5251
import org.apache.beam.sdk.metrics.DistributionResult;
@@ -276,6 +275,7 @@ private void testSingleSourceGbk(Predicate<String> predicate) throws Exception {
276275

277276
PCollection<KV<String, CoGbkResult>> output =
278277
pipeline.apply(
278+
"SingleSourceGbk-" + UUID.randomUUID(),
279279
Read.from(
280280
new SortedBucketSource<>(String.class, Collections.singletonList(bucketedInput))));
281281

@@ -704,6 +704,7 @@ private static void checkJoin(
704704

705705
PCollection<KV<String, CoGbkResult>> output =
706706
pipeline.apply(
707+
"CheckJoin-" + UUID.randomUUID(),
707708
Read.from(new SortedBucketSource<>(String.class, inputs, targetParallelism)));
708709

709710
Function<String, String> extractKeyFn = TestBucketMetadata.of(2, 1)::extractKey;

0 commit comments

Comments
 (0)