From 6c51a74b3f620cc2965ac911f3432993b565d321 Mon Sep 17 00:00:00 2001 From: Nancy Date: Tue, 1 Mar 2022 09:46:11 -0800 Subject: [PATCH 01/68] Added integration test for transaction boundaries and transaction ID ordering. Made small fixes in ordered by key integration test. --- ...mOrderedByTimestampAndTransactionIdIT.java | 581 ++++++++++++++++++ ...hangeStreamOrderedWithinKeyGloballyIT.java | 121 +--- ...erChangeStreamTransactionBoundariesIT.java | 400 ++++++++++++ 3 files changed, 995 insertions(+), 107 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.java create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamTransactionBoundariesIT.java diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.java new file mode 100644 index 000000000000..01ae1f055add --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.java @@ -0,0 +1,581 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.gcp.spanner.changestreams.it; + +import com.google.cloud.Timestamp; +import com.google.cloud.spanner.DatabaseClient; +import com.google.cloud.spanner.Key; +import com.google.cloud.spanner.KeySet; +import com.google.cloud.spanner.Mutation; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; +import javax.annotation.Nullable; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.coders.BooleanCoder; +import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; +import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.DataChangeRecord; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.Mod; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.TimeDomain; +import org.apache.beam.sdk.state.Timer; +import org.apache.beam.sdk.state.TimerSpec; +import org.apache.beam.sdk.state.TimerSpecs; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * End-to-end test of Cloud Spanner Change Streams with strict commit timestamp and transaction + * ordering. + */ +@RunWith(JUnit4.class) +public class SpannerChangeStreamOrderedByTimestampAndTransactionIdIT { + + private static final Logger LOG = + LoggerFactory.getLogger(SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.class); + + @ClassRule public static final IntegrationTestEnv ENV = new IntegrationTestEnv(); + @Rule public final transient TestPipeline pipeline = TestPipeline.create(); + + private static String projectId; + private static String instanceId; + private static String databaseId; + private static String tableName; + private static String changeStreamName; + private static DatabaseClient databaseClient; + + @BeforeClass + public static void setup() throws InterruptedException, ExecutionException, TimeoutException { + projectId = ENV.getProjectId(); + instanceId = ENV.getInstanceId(); + databaseId = ENV.getDatabaseId(); + tableName = ENV.createSingersTable(); + changeStreamName = ENV.createChangeStreamFor(tableName); + databaseClient = ENV.getDatabaseClient(); + } + + @Test + public void testTransactionBoundaries() { + final SpannerConfig spannerConfig = + SpannerConfig.create() + .withProjectId(projectId) + .withInstanceId(instanceId) + .withDatabaseId(databaseId); + // Commit a initial transaction to get the timestamp to start reading from. + List mutations = new ArrayList<>(); + mutations.add(insertRecordMutation(0, "FirstName0", "LastName0")); + final long timeIncrementInSeconds = 2; + final Timestamp startTimestamp = databaseClient.write(mutations); + writeTransactionsToDatabase(); + + // Sleep the time increment interval. + try { + Thread.sleep(timeIncrementInSeconds * 1000); + } catch (InterruptedException e) { + LOG.error(e.toString(), e); + } + + // This will be the second batch of transactions that will have strict timestamp ordering + // per key. + writeTransactionsToDatabase(); + + // Sleep the time increment interval. + try { + Thread.sleep(timeIncrementInSeconds * 1000); + } catch (InterruptedException e) { + LOG.error(e.toString(), e); + } + + // This will be the final batch of transactions that will have strict timestamp ordering + // per key. + com.google.cloud.Timestamp endTimestamp = writeTransactionsToDatabase(); + + final PCollection tokens = + pipeline + .apply( + SpannerIO.readChangeStream() + .withSpannerConfig(spannerConfig) + .withChangeStreamName(changeStreamName) + .withMetadataDatabase(databaseId) + .withInclusiveStartAt(startTimestamp) + .withInclusiveEndAt(endTimestamp)) + .apply( + ParDo.of( + new SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.KeyBySortKeyFn())) + .apply( + ParDo.of( + new SpannerChangeStreamOrderedByTimestampAndTransactionIdIT + .CreateArtificialKeyFn())) + .apply( + ParDo.of( + new BufferRecordsUntilOutputTimestamp(endTimestamp, timeIncrementInSeconds))) + .apply( + ParDo.of(new SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.ToStringFn())); + + // Assert that the returned PCollection contains all six transactions (in string representation) + // and that each transaction contains, in order, the list of mutations added. + PAssert.that(tokens) + .containsInAnyOrder( + // Insert Singer 0 into the table. + "{\"SingerId\":\"0\"},INSERT\n" + + // Insert Singer 1 and 2 into the table, + + "{\"SingerId\":\"1\"}{\"SingerId\":\"2\"},INSERT\n" + + // Delete Singer 1 and Insert Singer 3 into the table. + + "{\"SingerId\":\"1\"},DELETE\n" + + "{\"SingerId\":\"3\"},INSERT\n" + + // Delete Singers 2 and 3. + + "{\"SingerId\":\"2\"}{\"SingerId\":\"3\"},DELETE\n" + + // Delete Singer 0. + + "{\"SingerId\":\"0\"},DELETE\n", + + // Second batch of transactions. + // Insert Singer 1 and 2 into the table, + "{\"SingerId\":\"1\"}{\"SingerId\":\"2\"},INSERT\n" + + // Delete Singer 1 and Insert Singer 3 into the table. + + "{\"SingerId\":\"1\"},DELETE\n" + + "{\"SingerId\":\"3\"},INSERT\n" + + // Delete Singers 2 and 3. + + "{\"SingerId\":\"2\"}{\"SingerId\":\"3\"},DELETE\n", + + // Third batch of transactions. + // Insert Singer 1 and 2 into the table, + "{\"SingerId\":\"1\"}{\"SingerId\":\"2\"},INSERT\n" + + // Delete Singer 1 and Insert Singer 3 into the table. + + "{\"SingerId\":\"1\"},DELETE\n" + + "{\"SingerId\":\"3\"},INSERT\n" + + // Delete Singers 2 and 3. + + "{\"SingerId\":\"2\"}{\"SingerId\":\"3\"},DELETE\n"); + + final PipelineResult pipelineResult = pipeline.run(); + pipelineResult.waitUntilFinish(); + } + + // KeyByTransactionIdFn takes in a DataChangeRecord and outputs a key-value pair of + // {SortKey, DataChangeRecord} + private static class KeyBySortKeyFn + extends DoFn< + DataChangeRecord, + KV> { + + private static final long serialVersionUID = 1270485392415293532L; + + @ProcessElement + public void processElement( + @Element DataChangeRecord record, + OutputReceiver< + KV< + SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.SortKey, + DataChangeRecord>> + outputReceiver) { + outputReceiver.output( + KV.of( + new SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.SortKey( + record.getCommitTimestamp(), record.getServerTransactionId()), + record)); + } + } + + // CreateArtificialKeyFn keys each input element by an artifical byte key. This is because buffers + // and timers are per key and window, and we want to buffer all data change records in a time + // interval, rather than buffer per key. + private static class CreateArtificialKeyFn + extends DoFn< + KV, + KV< + byte[], + KV< + SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.SortKey, + DataChangeRecord>>> { + private static final long serialVersionUID = -3363057370822294686L; + + @ProcessElement + public void processElement( + @Element + KV + element, + OutputReceiver< + KV< + byte[], + KV< + SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.SortKey, + DataChangeRecord>>> + outputReceiver) { + outputReceiver.output(KV.of(new byte[0], element)); + } + } + + // Timers and buffers are per-key. + // Buffer each data change record until the watermark passes the timestamp at which we want + // to output the buffered data change records. + // We utilize a looping timer to determine when to flush the buffer: + // + // 1. When we see a data change record for the first time (i.e. no data change records in + // the buffer), we will set the timer to fire at an interval after the data change record's + // timestamp. + // 2. Then, when the timer fires, if the current timer's expiration time is before the pipeline + // end time, if set, we still have data left to process. We will set the next timer to the + // current timer's expiration time plus incrementIntervalInSeconds. + // 3. Otherwise, we will not set a timer. + // + private static class BufferRecordsUntilOutputTimestamp + extends DoFn< + KV< + byte[], + KV< + SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.SortKey, + DataChangeRecord>>, + Iterable< + KV< + SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.SortKey, + DataChangeRecord>>> { + private static final long serialVersionUID = 5050535558953049259L; + + private final long incrementIntervalInSeconds; + private final @Nullable Instant pipelineEndTime; + + private BufferRecordsUntilOutputTimestamp( + @Nullable com.google.cloud.Timestamp endTimestamp, long incrementIntervalInSeconds) { + this.incrementIntervalInSeconds = incrementIntervalInSeconds; + if (endTimestamp != null) { + this.pipelineEndTime = new Instant(endTimestamp.toSqlTimestamp()); + } else { + pipelineEndTime = null; + } + } + + @SuppressWarnings("unused") + @TimerId("timer") + private final TimerSpec timerSpec = TimerSpecs.timer(TimeDomain.EVENT_TIME); + + @SuppressWarnings("unused") + @StateId("buffer") + private final StateSpec< + BagState< + KV< + SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.SortKey, + DataChangeRecord>>> + buffer = StateSpecs.bag(); + + @SuppressWarnings("unused") + @StateId("keySeen") + private final StateSpec> keySeen = StateSpecs.value(BooleanCoder.of()); + + @ProcessElement + public void process( + @Element + KV< + byte[], + KV< + SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.SortKey, + DataChangeRecord>> + element, + @StateId("buffer") + BagState< + KV< + SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.SortKey, + DataChangeRecord>> + buffer, + @TimerId("timer") Timer timer, + @StateId("keySeen") ValueState keySeen) { + buffer.add(element.getValue()); + + // Only set the timer if this is the first time we are receiving a data change record + // with this key. + Boolean hasKeyBeenSeen = keySeen.read(); + if (hasKeyBeenSeen == null) { + Instant commitTimestamp = + new Instant(element.getValue().getKey().getCommitTimestamp().toSqlTimestamp()); + Instant outputTimestamp = + commitTimestamp.plus(Duration.standardSeconds(incrementIntervalInSeconds)); + LOG.debug("Setting timer at {} for key {}", outputTimestamp.toString(), element.getKey()); + timer.set(outputTimestamp); + keySeen.write(true); + } + } + + @OnTimer("timer") + public void onExpiry( + OnTimerContext context, + @StateId("buffer") + BagState< + KV< + SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.SortKey, + DataChangeRecord>> + buffer, + @TimerId("timer") Timer timer) { + if (!buffer.isEmpty().read()) { + final List< + KV< + SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.SortKey, + DataChangeRecord>> + records = + StreamSupport.stream(buffer.read().spliterator(), false) + .collect(Collectors.toList()); + buffer.clear(); + + List> + recordsToOutput = new ArrayList<>(); + for (KV + record : records) { + Instant recordCommitTimestamp = + new Instant(record.getKey().getCommitTimestamp().toSqlTimestamp()); + // When the watermark passes time T, this means that all records with event time < T + // have been processed and successfully committed. Since the timer fires when the + // watermark passes the expiration time, we should only output records with event time + // < expiration time. + final String recordString = getRecordString(record.getValue()); + if (recordCommitTimestamp.isBefore(context.timestamp())) { + LOG.debug( + "Outputting transactions {} with id {} at expiration timestamp {}", + recordString, + record.getKey().toString(), + context.timestamp().toString()); + recordsToOutput.add(record); + } else { + LOG.debug( + "Expired at {} but adding transaction {} back to buffer " + + "due to commit timestamp {}", + context.timestamp().toString(), + recordString, + recordCommitTimestamp.toString()); + buffer.add(record); + } + } + + // Output records, if there are any to output. + if (!recordsToOutput.isEmpty()) { + context.outputWithTimestamp(recordsToOutput, context.timestamp()); + LOG.debug( + "Expired at {}, outputting records for key {}", + context.timestamp().toString(), + recordsToOutput.get(0).getKey().toString()); + } else { + LOG.debug("Expired at {} with no records", context.timestamp().toString()); + } + } + + Instant nextTimer = + context.timestamp().plus(Duration.standardSeconds(incrementIntervalInSeconds)); + if (pipelineEndTime == null || context.timestamp().isBefore(pipelineEndTime)) { + // If the current timer's timestamp is before the pipeline end time, or there is no + // pipeline end time, we still have data left to process. + LOG.debug("Setting next timer to {}", nextTimer.toString()); + timer.set(nextTimer); + } else { + LOG.debug( + "Timer not being set as exceeded pipeline end time: " + pipelineEndTime.toString()); + } + } + } + + // ToStringFn takes in a list of key-value pairs of SortKey, Iterable and + // outputs a string representation. + private static class ToStringFn + extends DoFn< + Iterable< + KV< + SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.SortKey, + DataChangeRecord>>, + String> { + + private static final long serialVersionUID = 2307936669684679038L; + + @ProcessElement + public void processElement( + @Element + Iterable< + KV< + SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.SortKey, + DataChangeRecord>> + element, + OutputReceiver outputReceiver) { + final StringBuilder builder = new StringBuilder(); + + List> + sortedTransactions = + StreamSupport.stream(element.spliterator(), false) + .sorted((kv1, kv2) -> kv1.getKey().compareTo(kv2.getKey())) + .collect(Collectors.toList()); + + sortedTransactions.forEach( + record -> { + builder.append(getRecordString(record.getValue())); + }); + outputReceiver.output(builder.toString()); + } + } + + // Get a string representation of the mods and the mod type in the data change record. + private static String getRecordString(DataChangeRecord record) { + final StringBuilder builder = new StringBuilder(); + String modString = ""; + for (Mod mod : record.getMods()) { + modString += mod.getKeysJson(); + } + builder.append(String.join(",", modString, record.getModType().toString())); + builder.append("\n"); + return builder.toString(); + } + + private Timestamp writeTransactionsToDatabase() { + List mutations = new ArrayList<>(); + + // 1. Commit a transaction to insert Singer 1 and Singer 2 into the table. + mutations.add(insertRecordMutation(1, "FirstName1", "LastName2")); + mutations.add(insertRecordMutation(2, "FirstName2", "LastName2")); + Timestamp t1 = databaseClient.write(mutations); + LOG.debug("The first transaction committed with timestamp: " + t1.toString()); + mutations.clear(); + + // 2. Commmit a transaction to insert Singer 3 and remove Singer 1 from the table. + mutations.add(insertRecordMutation(3, "FirstName3", "LastName3")); + mutations.add(deleteRecordMutation(1)); + Timestamp t2 = databaseClient.write(mutations); + LOG.debug("The second transaction committed with timestamp: " + t2.toString()); + mutations.clear(); + + // 3. Commit a transaction to delete Singer 2 and Singer 3 from the table. + mutations.add(deleteRecordMutation(2)); + mutations.add(deleteRecordMutation(3)); + Timestamp t3 = databaseClient.write(mutations); + LOG.debug("The third transaction committed with timestamp: " + t3.toString()); + mutations.clear(); + + // 4. Commit a transaction to delete Singer 0. + mutations.add(deleteRecordMutation(0)); + Timestamp t4 = databaseClient.write(mutations); + LOG.debug("The fourth transaction committed with timestamp: " + t4.toString()); + return t4; + } + + // Create an insert mutation. + private static Mutation insertRecordMutation(long singerId, String firstName, String lastName) { + return Mutation.newInsertBuilder(tableName) + .set("SingerId") + .to(singerId) + .set("FirstName") + .to(firstName) + .set("LastName") + .to(lastName) + .build(); + } + + // Create a delete mutation. + private static Mutation deleteRecordMutation(long singerId) { + return Mutation.delete(tableName, KeySet.newBuilder().addKey(Key.of(singerId)).build()); + } + + private static class SortKey + implements Serializable, + Comparable { + + private static final long serialVersionUID = 2105939115467195036L; + + private Timestamp commitTimestamp; + private String transactionId; + + public SortKey() {} + + public SortKey(Timestamp commitTimestamp, String transactionId) { + this.commitTimestamp = commitTimestamp; + this.transactionId = transactionId; + } + + public static long getSerialVersionUID() { + return serialVersionUID; + } + + public Timestamp getCommitTimestamp() { + return commitTimestamp; + } + + public void setCommitTimestamp(Timestamp commitTimestamp) { + this.commitTimestamp = commitTimestamp; + } + + public String getTransactionId() { + return transactionId; + } + + public void setTransactionId(String transactionId) { + this.transactionId = transactionId; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.SortKey sortKey = + (SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.SortKey) o; + return Objects.equals(commitTimestamp, sortKey.commitTimestamp) + && Objects.equals(transactionId, sortKey.transactionId); + } + + @Override + public int hashCode() { + return Objects.hash(commitTimestamp, transactionId); + } + + @Override + public int compareTo(SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.SortKey other) { + return Comparator + .comparingDouble( + sortKey -> + sortKey.getCommitTimestamp().getSeconds() + + sortKey.getCommitTimestamp().getNanos() / 1000000000.0) + .thenComparing(sortKey -> sortKey.getTransactionId()) + .compare(this, other); + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java index fc7ff7118a46..f50b1b6e7e59 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java @@ -35,7 +35,6 @@ import org.apache.beam.sdk.coders.BooleanCoder; import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; -import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.ChangeStreamRecordMetadata; import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.DataChangeRecord; import org.apache.beam.sdk.state.BagState; import org.apache.beam.sdk.state.StateSpec; @@ -98,7 +97,7 @@ public void testOrderedWithinKey() { .withDatabaseId(databaseId); // Get the time increment interval at which to flush data changes ordered by key. - final long timeIncrementInSeconds = 70; + final long timeIncrementInSeconds = 2; // Commit a initial transaction to get the timestamp to start reading from. List mutations = new ArrayList<>(); @@ -113,7 +112,7 @@ public void testOrderedWithinKey() { try { Thread.sleep(timeIncrementInSeconds * 1000); } catch (InterruptedException e) { - System.out.println(e); + LOG.error(e.toString(), e); } // This will be the second batch of transactions that will have strict timestamp ordering @@ -124,7 +123,7 @@ public void testOrderedWithinKey() { try { Thread.sleep(timeIncrementInSeconds * 1000); } catch (InterruptedException e) { - System.out.println(e); + LOG.error(e.toString(), e); } // This will be the final batch of transactions that will have strict timestamp ordering @@ -163,71 +162,35 @@ public void testOrderedWithinKey() { + "{\"FirstName\":\"Inserting mutation 0\",\"LastName\":null,\"SingerInfo\":null};" + "Deleted record;", "{\"SingerId\":\"1\"}\n" - + "{\"FirstName\":\"Inserting mutation 1\",\"LastName\":null,\"SingerInfo\":null};" - + "{\"FirstName\":\"Updating mutation 1\"};" - + "Deleted record;" + "{\"FirstName\":\"Inserting mutation 1\",\"LastName\":null,\"SingerInfo\":null};" + "Deleted record;", "{\"SingerId\":\"2\"}\n" + "{\"FirstName\":\"Inserting mutation 2\",\"LastName\":null,\"SingerInfo\":null};" - + "{\"FirstName\":\"Updating mutation 2\"};" + "Deleted record;", "{\"SingerId\":\"3\"}\n" + "{\"FirstName\":\"Inserting mutation 3\",\"LastName\":null,\"SingerInfo\":null};" - + "{\"FirstName\":\"Updating mutation 3\"};" - + "Deleted record;", - "{\"SingerId\":\"4\"}\n" - + "{\"FirstName\":\"Inserting mutation 4\",\"LastName\":null,\"SingerInfo\":null};" - + "Deleted record;", - "{\"SingerId\":\"5\"}\n" - + "{\"FirstName\":\"Updating mutation 5\",\"LastName\":null,\"SingerInfo\":null};" - + "{\"FirstName\":\"Updating mutation 5\"};" + "Deleted record;", // Second batch of records ordered within key. "{\"SingerId\":\"1\"}\n" - + "{\"FirstName\":\"Inserting mutation 1\",\"LastName\":null,\"SingerInfo\":null};" - + "{\"FirstName\":\"Updating mutation 1\"};" - + "Deleted record;" + "{\"FirstName\":\"Inserting mutation 1\",\"LastName\":null,\"SingerInfo\":null};" + "Deleted record;", "{\"SingerId\":\"2\"}\n" + "{\"FirstName\":\"Inserting mutation 2\",\"LastName\":null,\"SingerInfo\":null};" - + "{\"FirstName\":\"Updating mutation 2\"};" + "Deleted record;", "{\"SingerId\":\"3\"}\n" + "{\"FirstName\":\"Inserting mutation 3\",\"LastName\":null,\"SingerInfo\":null};" - + "{\"FirstName\":\"Updating mutation 3\"};" - + "Deleted record;", - "{\"SingerId\":\"4\"}\n" - + "{\"FirstName\":\"Inserting mutation 4\",\"LastName\":null,\"SingerInfo\":null};" - + "Deleted record;", - "{\"SingerId\":\"5\"}\n" - + "{\"FirstName\":\"Updating mutation 5\",\"LastName\":null,\"SingerInfo\":null};" - + "{\"FirstName\":\"Updating mutation 5\"};" + "Deleted record;", // Third batch of records ordered within key. "{\"SingerId\":\"1\"}\n" - + "{\"FirstName\":\"Inserting mutation 1\",\"LastName\":null,\"SingerInfo\":null};" - + "{\"FirstName\":\"Updating mutation 1\"};" - + "Deleted record;" + "{\"FirstName\":\"Inserting mutation 1\",\"LastName\":null,\"SingerInfo\":null};" + "Deleted record;", "{\"SingerId\":\"2\"}\n" + "{\"FirstName\":\"Inserting mutation 2\",\"LastName\":null,\"SingerInfo\":null};" - + "{\"FirstName\":\"Updating mutation 2\"};" + "Deleted record;", "{\"SingerId\":\"3\"}\n" + "{\"FirstName\":\"Inserting mutation 3\",\"LastName\":null,\"SingerInfo\":null};" - + "{\"FirstName\":\"Updating mutation 3\"};" - + "Deleted record;", - "{\"SingerId\":\"4\"}\n" - + "{\"FirstName\":\"Inserting mutation 4\",\"LastName\":null,\"SingerInfo\":null};" - + "Deleted record;", - "{\"SingerId\":\"5\"}\n" - + "{\"FirstName\":\"Updating mutation 5\",\"LastName\":null,\"SingerInfo\":null};" - + "{\"FirstName\":\"Updating mutation 5\"};" + "Deleted record;"); pipeline.run().waitUntilFinish(); @@ -241,22 +204,6 @@ private static class BreakRecordByModFn extends DoFn outputReceiver) { - final ChangeStreamRecordMetadata fakeChangeStreamMetadata = - ChangeStreamRecordMetadata.newBuilder() - .withPartitionToken("1") - .withRecordTimestamp(com.google.cloud.Timestamp.ofTimeMicroseconds(2L)) - .withPartitionStartTimestamp(com.google.cloud.Timestamp.ofTimeMicroseconds(3L)) - .withPartitionEndTimestamp(com.google.cloud.Timestamp.ofTimeMicroseconds(4L)) - .withPartitionCreatedAt(com.google.cloud.Timestamp.ofTimeMicroseconds(5L)) - .withPartitionScheduledAt(com.google.cloud.Timestamp.ofTimeMicroseconds(6L)) - .withPartitionRunningAt(com.google.cloud.Timestamp.ofTimeMicroseconds(7L)) - .withQueryStartedAt(com.google.cloud.Timestamp.ofTimeMicroseconds(8L)) - .withRecordStreamStartedAt(com.google.cloud.Timestamp.ofTimeMicroseconds(9L)) - .withRecordStreamEndedAt(com.google.cloud.Timestamp.ofTimeMicroseconds(10L)) - .withRecordReadAt(com.google.cloud.Timestamp.ofTimeMicroseconds(11L)) - .withTotalStreamTimeMillis(12L) - .withNumberOfRecordsRead(13L) - .build(); record.getMods().stream() .map( mod -> @@ -273,7 +220,7 @@ public void processElement( record.getValueCaptureType(), record.getNumberOfRecordsInTransaction(), record.getNumberOfPartitionsInTransaction(), - fakeChangeStreamMetadata)) + record.getMetadata())) .forEach(outputReceiver::output); } } @@ -546,7 +493,7 @@ public int compareTo(SortKey other) { } } - private static com.google.cloud.Timestamp writeTransactionsToDatabase() { + private com.google.cloud.Timestamp writeTransactionsToDatabase() { List mutations = new ArrayList<>(); // 1. Commit a transaction to insert Singer 1 and Singer 2 into the table. @@ -556,65 +503,25 @@ private static com.google.cloud.Timestamp writeTransactionsToDatabase() { LOG.debug("The first transaction committed with timestamp: " + t1.toString()); mutations.clear(); - // 2. Commmit a transaction to insert Singer 4 and remove Singer 1 from the table. - mutations.add(updateRecordMutation(1)); - mutations.add(insertRecordMutation(4)); + // 2. Commmit a transaction to insert Singer 3 and remove Singer 1 from the table. + mutations.add(insertRecordMutation(3)); + mutations.add(deleteRecordMutation(1)); com.google.cloud.Timestamp t2 = databaseClient.write(mutations); LOG.debug("The second transaction committed with timestamp: " + t2.toString()); mutations.clear(); - // 3. Commit a transaction to insert Singer 3 and Singer 5. - mutations.add(deleteRecordMutation(1)); - mutations.add(insertRecordMutation(3)); - mutations.add(insertRecordMutation(5)); - mutations.add(updateRecordMutation(5)); + // 3. Commit a transaction to delete Singer 2 and Singer 3 from the table. + mutations.add(deleteRecordMutation(2)); + mutations.add(deleteRecordMutation(3)); com.google.cloud.Timestamp t3 = databaseClient.write(mutations); LOG.debug("The third transaction committed with timestamp: " + t3.toString()); mutations.clear(); - // 4. Commit a transaction to update Singer 3 and Singer 2 in the table. - mutations.add(updateRecordMutation(3)); - mutations.add(updateRecordMutation(2)); + // 4. Commit a transaction to delete Singer 0. + mutations.add(deleteRecordMutation(0)); com.google.cloud.Timestamp t4 = databaseClient.write(mutations); LOG.debug("The fourth transaction committed with timestamp: " + t4.toString()); - mutations.clear(); - - // 5. Commit a transaction to delete 4, insert 1, delete 3, update 5. - mutations.add(deleteRecordMutation(4)); - mutations.add(insertRecordMutation(1)); - mutations.add(deleteRecordMutation(3)); - mutations.add(updateRecordMutation(5)); - com.google.cloud.Timestamp t5 = databaseClient.write(mutations); - - LOG.debug("The fifth transaction committed with timestamp: " + t5.toString()); - mutations.clear(); - - // 6. Commit a transaction to delete Singers 5, insert singers 6. - mutations.add(deleteRecordMutation(5)); - mutations.add(insertRecordMutation(6)); - mutations.add(deleteRecordMutation(6)); - com.google.cloud.Timestamp t6 = databaseClient.write(mutations); - LOG.debug("The sixth transaction committed with timestamp: " + t6.toString()); - mutations.clear(); - - // 7. Delete remaining rows from database. - mutations.add(deleteRecordMutation(1)); - mutations.add(deleteRecordMutation(2)); - mutations.add(deleteRecordMutation(0)); - com.google.cloud.Timestamp t7 = databaseClient.write(mutations); - LOG.debug("The seventh transaction committed with timestamp: " + t7.toString()); - - return t7; - } - - // Create an update mutation. - private static Mutation updateRecordMutation(long singerId) { - return Mutation.newUpdateBuilder(tableName) - .set("SingerId") - .to(singerId) - .set("FirstName") - .to("Updating mutation " + singerId) - .build(); + return t4; } // Create an insert mutation. diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamTransactionBoundariesIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamTransactionBoundariesIT.java new file mode 100644 index 000000000000..71b233e4daea --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamTransactionBoundariesIT.java @@ -0,0 +1,400 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.gcp.spanner.changestreams.it; + +import com.google.cloud.Timestamp; +import com.google.cloud.spanner.DatabaseClient; +import com.google.cloud.spanner.Key; +import com.google.cloud.spanner.KeySet; +import com.google.cloud.spanner.Mutation; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; +import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.DataChangeRecord; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.Mod; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.joda.time.Instant; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** End-to-end test of Cloud Spanner Change Streams Transaction Boundaries. */ +@RunWith(JUnit4.class) +public class SpannerChangeStreamTransactionBoundariesIT { + + private static final Logger LOG = + LoggerFactory.getLogger(SpannerChangeStreamTransactionBoundariesIT.class); + + @ClassRule public static final IntegrationTestEnv ENV = new IntegrationTestEnv(); + @Rule public final transient TestPipeline pipeline = TestPipeline.create(); + + private static String projectId; + private static String instanceId; + private static String databaseId; + private static String tableName; + private static String changeStreamName; + private static DatabaseClient databaseClient; + + @BeforeClass + public static void setup() throws InterruptedException, ExecutionException, TimeoutException { + projectId = ENV.getProjectId(); + instanceId = ENV.getInstanceId(); + databaseId = ENV.getDatabaseId(); + tableName = ENV.createSingersTable(); + changeStreamName = ENV.createChangeStreamFor(tableName); + databaseClient = ENV.getDatabaseClient(); + } + + @Test + public void testTransactionBoundaries() { + final SpannerConfig spannerConfig = + SpannerConfig.create() + .withProjectId(projectId) + .withInstanceId(instanceId) + .withDatabaseId(databaseId); + // Commit a initial transaction to get the timestamp to start reading from. + List mutations = new ArrayList<>(); + mutations.add(insertRecordMutation(0, "FirstName0", "LastName0")); + final Timestamp startTimestamp = databaseClient.write(mutations); + + // Get the timestamp of the last committed transaction to get the end timestamp. + final Timestamp endTimestamp = writeTransactionsToDatabase(); + + final PCollection tokens = + pipeline + .apply( + SpannerIO.readChangeStream() + .withSpannerConfig(spannerConfig) + .withChangeStreamName(changeStreamName) + .withMetadataDatabase(databaseId) + .withInclusiveStartAt(startTimestamp) + .withInclusiveEndAt(endTimestamp)) + .apply(ParDo.of(new SpannerChangeStreamTransactionBoundariesIT.KeyByTransactionIdFn())) + .apply(ParDo.of(new SpannerChangeStreamTransactionBoundariesIT.TransactionBoundaryFn())) + .apply(ParDo.of(new SpannerChangeStreamTransactionBoundariesIT.ToStringFn())); + + // Assert that the returned PCollection contains all six transactions (in string representation) + // and that each transaction contains, in order, the list of mutations added. + PAssert.that(tokens) + .containsInAnyOrder( + // Insert Singer 0 into the table. + "{\"SingerId\":\"0\"},INSERT\n", + + // Insert Singer 1 and 2 into the table, + "{\"SingerId\":\"1\"}{\"SingerId\":\"2\"},INSERT\n", + + // Delete Singer 1 and Insert Singer 3 into the table. + "{\"SingerId\":\"1\"},DELETE\n" + "{\"SingerId\":\"3\"},INSERT\n", + + // Insert Singers 4, 5, 6 into the table. + "{\"SingerId\":\"4\"}{\"SingerId\":\"5\"}{\"SingerId\":\"6\"},INSERT\n", + + // Update Singer 6 and Insert Singer 7 + "{\"SingerId\":\"6\"},UPDATE\n" + "{\"SingerId\":\"7\"},INSERT\n", + + // Update Singers 4 and 5 in the table. + "{\"SingerId\":\"4\"}{\"SingerId\":\"5\"},UPDATE\n", + + // Delete Singers 3, 4, 5 from the table. + "{\"SingerId\":\"3\"}{\"SingerId\":\"4\"}{\"SingerId\":\"5\"},DELETE\n", + + // Delete Singers 0, 2, 6, 7; + "{\"SingerId\":\"0\"}{\"SingerId\":\"2\"}{\"SingerId\":\"6\"}" + + "{\"SingerId\":\"7\"},DELETE\n"); + + final PipelineResult pipelineResult = pipeline.run(); + pipelineResult.waitUntilFinish(); + } + + // KeyByTransactionIdFn takes in a DataChangeRecord and outputs a key-value pair of + // {TransactionId, DataChangeRecord} + private static class KeyByTransactionIdFn + extends DoFn> { + + private static final long serialVersionUID = 1270485392415293532L; + + @ProcessElement + public void processElement( + @Element DataChangeRecord record, + OutputReceiver> outputReceiver) { + outputReceiver.output(KV.of(record.getServerTransactionId(), record)); + } + } + + // TransactionBoundaryFn buffers received key-value pairs of {TransactionId, DataChangeRecord} + // from KeyByTransactionIdFn and buffers them in groups based on TransactionId. + // When the number of records buffered is equal to the number of records contained in the + // entire transaction, this function sorts the DataChangeRecords in the group by record sequence + // and outputs a key-value pair of SortKey(CommitTimestamp, TransactionId), + // Iterable. + private static class TransactionBoundaryFn + extends DoFn< + KV, + KV>> { + + private static final long serialVersionUID = 5050535558953049259L; + + @SuppressWarnings("UnusedVariable") + @StateId("buffer") + private final StateSpec> buffer = StateSpecs.bag(); + + @SuppressWarnings("UnusedVariable") + @StateId("count") + private final StateSpec> countState = StateSpecs.value(); + + @ProcessElement + public void process( + ProcessContext context, + @StateId("buffer") BagState buffer, + @StateId("count") ValueState countState) { + final KV element = context.element(); + final DataChangeRecord record = element.getValue(); + + buffer.add(record); + int count = (countState.read() != null ? countState.read() : 0); + count = count + 1; + countState.write(count); + + if (count == record.getNumberOfRecordsInTransaction()) { + final List sortedRecords = + StreamSupport.stream(buffer.read().spliterator(), false) + .sorted(Comparator.comparing(DataChangeRecord::getRecordSequence)) + .collect(Collectors.toList()); + + final Instant commitInstant = + new Instant(sortedRecords.get(0).getCommitTimestamp().toSqlTimestamp().getTime()); + context.outputWithTimestamp( + KV.of( + new SpannerChangeStreamTransactionBoundariesIT.SortKey( + sortedRecords.get(0).getCommitTimestamp(), + sortedRecords.get(0).getServerTransactionId()), + sortedRecords), + commitInstant); + buffer.clear(); + countState.clear(); + } + } + } + + // ToStringFn takes in a key-value pair of SortKey, Iterable and outputs + // a string representation. + private static class ToStringFn + extends DoFn< + KV>, + String> { + + private static final long serialVersionUID = 2307936669684679038L; + + @ProcessElement + public void processElement( + @Element + KV> + element, + OutputReceiver outputReceiver) { + final StringBuilder builder = new StringBuilder(); + final Iterable sortedRecords = element.getValue(); + sortedRecords.forEach( + record -> { + // Output the string representation of the mods and the mod type for each data change + // record. + String modString = ""; + for (Mod mod : record.getMods()) { + modString += mod.getKeysJson(); + } + builder.append(String.join(",", modString, record.getModType().toString())); + builder.append("\n"); + }); + outputReceiver.output(builder.toString()); + } + } + + private static class SortKey + implements Serializable, Comparable { + + private static final long serialVersionUID = 2105939115467195036L; + + private Timestamp commitTimestamp; + private String transactionId; + + public SortKey() {} + + public SortKey(Timestamp commitTimestamp, String transactionId) { + this.commitTimestamp = commitTimestamp; + this.transactionId = transactionId; + } + + public static long getSerialVersionUID() { + return serialVersionUID; + } + + public Timestamp getCommitTimestamp() { + return commitTimestamp; + } + + public void setCommitTimestamp(Timestamp commitTimestamp) { + this.commitTimestamp = commitTimestamp; + } + + public String getTransactionId() { + return transactionId; + } + + public void setTransactionId(String transactionId) { + this.transactionId = transactionId; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SpannerChangeStreamTransactionBoundariesIT.SortKey sortKey = + (SpannerChangeStreamTransactionBoundariesIT.SortKey) o; + return Objects.equals(commitTimestamp, sortKey.commitTimestamp) + && Objects.equals(transactionId, sortKey.transactionId); + } + + @Override + public int hashCode() { + return Objects.hash(commitTimestamp, transactionId); + } + + @Override + public int compareTo(SpannerChangeStreamTransactionBoundariesIT.SortKey other) { + return Comparator.comparingDouble( + sortKey -> + sortKey.getCommitTimestamp().getSeconds() + + sortKey.getCommitTimestamp().getNanos() / 1000000000.0) + .thenComparing(sortKey -> sortKey.getTransactionId()) + .compare(this, other); + } + } + + private Timestamp writeTransactionsToDatabase() { + List mutations = new ArrayList<>(); + + // 1. Commit a transaction to insert Singer 1 and Singer 2 into the table. + mutations.add(insertRecordMutation(1, "FirstName1", "LastName2")); + mutations.add(insertRecordMutation(2, "FirstName2", "LastName2")); + Timestamp t1 = databaseClient.write(mutations); + LOG.debug("The first transaction committed with timestamp: " + t1.toString()); + mutations.clear(); + + // 2. Commmit a transaction to insert Singer 3 and remove Singer 1 from the table. + mutations.add(insertRecordMutation(3, "FirstName3", "LastName3")); + mutations.add(deleteRecordMutation(1)); + Timestamp t2 = databaseClient.write(mutations); + LOG.debug("The second transaction committed with timestamp: " + t2.toString()); + mutations.clear(); + + // 3. Commit a transaction to insert Singer 4 and Singer 5 and Singer 6 into the table. + mutations.add(insertRecordMutation(4, "FirstName4", "LastName4")); + mutations.add(insertRecordMutation(5, "FirstName5", "LastName5")); + mutations.add(insertRecordMutation(6, "FirstName6", "LastName6")); + Timestamp t3 = databaseClient.write(mutations); + LOG.debug("The third transaction committed with timestamp: " + t3.toString()); + mutations.clear(); + + // 4. Commit a transaction to insert Singer 7 and update Singer 6 in the table. + mutations.add(insertRecordMutation(7, "FirstName7", "LastName7")); + mutations.add(updateRecordMutation(6, "FirstName5", "LastName5")); + Timestamp t4 = databaseClient.write(mutations); + LOG.debug("The fourth transaction committed with timestamp: " + t4.toString()); + mutations.clear(); + + // 5. Commit a transaction to update Singer 4 and Singer 5 in the table. + mutations.add(updateRecordMutation(4, "FirstName9", "LastName9")); + mutations.add(updateRecordMutation(5, "FirstName9", "LastName9")); + Timestamp t5 = databaseClient.write(mutations); + LOG.debug("The fifth transaction committed with timestamp: " + t5.toString()); + mutations.clear(); + + // 6. Commit a transaction to delete Singers 3, 4, 5. + mutations.add(deleteRecordMutation(3)); + mutations.add(deleteRecordMutation(4)); + mutations.add(deleteRecordMutation(5)); + Timestamp t6 = databaseClient.write(mutations); + mutations.clear(); + LOG.debug("The sixth transaction committed with timestamp: " + t6.toString()); + + // 7. Commit a transaction to delete Singers 0, 2, 6, 7. + mutations.add(deleteRecordMutation(0)); + mutations.add(deleteRecordMutation(2)); + mutations.add(deleteRecordMutation(6)); + mutations.add(deleteRecordMutation(7)); + Timestamp t7 = databaseClient.write(mutations); + LOG.debug("The seventh transaction committed with timestamp: " + t7.toString()); + + return t7; + } + + // Create an update mutation. + private static Mutation updateRecordMutation(long singerId, String firstName, String lastName) { + return Mutation.newUpdateBuilder(tableName) + .set("SingerId") + .to(singerId) + .set("FirstName") + .to(firstName) + .set("LastName") + .to(lastName) + .build(); + } + + // Create an insert mutation. + private static Mutation insertRecordMutation(long singerId, String firstName, String lastName) { + return Mutation.newInsertBuilder(tableName) + .set("SingerId") + .to(singerId) + .set("FirstName") + .to(firstName) + .set("LastName") + .to(lastName) + .build(); + } + + // Create a delete mutation. + private static Mutation deleteRecordMutation(long singerId) { + return Mutation.delete(tableName, KeySet.newBuilder().addKey(Key.of(singerId)).build()); + } +} From 1615c2435b5ca7f83dcdc46a5a284c8d43225398 Mon Sep 17 00:00:00 2001 From: emily Date: Mon, 28 Feb 2022 23:46:44 -0800 Subject: [PATCH 02/68] [BEAM-9150] Fix beam_PostRelease_Python_Candidate (python RC validation scripts) (#16955) --- .../python_release_automation_utils.sh | 35 ++++++++++++++++--- ..._release_candidate_python_mobile_gaming.sh | 10 ++++-- ...run_release_candidate_python_quickstart.sh | 4 +-- 3 files changed, 41 insertions(+), 8 deletions(-) diff --git a/release/src/main/python-release/python_release_automation_utils.sh b/release/src/main/python-release/python_release_automation_utils.sh index 5d0e1c6eff76..b328c428a797 100644 --- a/release/src/main/python-release/python_release_automation_utils.sh +++ b/release/src/main/python-release/python_release_automation_utils.sh @@ -228,7 +228,7 @@ function cleanup_pubsub() { # $2 - pid: the pid of running pipeline # $3 - running_job (DataflowRunner only): the job id of streaming pipeline running on DataflowRunner ####################################### -function verify_steaming_result() { +function verify_streaming_result() { retry=3 should_see="Python: " while(( $retry > 0 )); do @@ -295,9 +295,11 @@ function verify_user_score() { function verify_hourly_team_score() { retry=3 should_see='AntiqueBrassPlatypus' + runner=$1 + while(( $retry >= 0 )); do if [[ $retry > 0 ]]; then - bq_pull_result=$(bq head -n 500 $DATASET.hourly_team_score_python_$1) + bq_pull_result=$(bq head -n 500 ${DATASET}.${HOURLY_TEAM_SCORE_TABLE_PREFIX}_${runner}) if [[ $bq_pull_result = *"$should_see"* ]]; then echo "SUCCEED: hourly_team_score example successful run on $1-runner" break @@ -307,14 +309,38 @@ function verify_hourly_team_score() { sleep 15 fi else - echo "FAILED: HourlyTeamScore example failed running on $1-runner. \ - Did not found scores of team $should_see in $DATASET.leader_board" + echo "FAILED: HourlyTeamScore example failed running on $runner runner. \ + Did not found scores of team $should_see in ${DATASET}.${HOURLY_TEAM_SCORE_TABLE_PREFIX}_${runner}" complete "FAILED" exit 1 fi done } +function cleanup_hourly_team_score() { + retry=3 + runner=$1 + + echo "Removing previously created table ${DATASET}.${HOURLY_TEAM_SCORE_TABLE_PREFIX}_${runner}..." + bq rm -q -f -t "${DATASET}.${HOURLY_TEAM_SCORE_TABLE_PREFIX}_${runner}" + + while(( $retry >= 0 )); do + if [[ $retry > 0 ]]; then + bq_ls_result=$(bq ls $DATASET) + if [[ $bq_ls_result = *"${HOURLY_TEAM_SCORE_TABLE_PREFIX}_${runner}"* ]]; then + retry=$(($retry-1)) + echo "${DATASET}.${HOURLY_TEAM_SCORE_TABLE_PREFIX}_${runner} not cleaned up yet, waiting" + sleep 1000 + else + echo "Confirmed ${DATASET}.${HOURLY_TEAM_SCORE_TABLE_PREFIX}_${runner} removed before running new test." + break + fi + else + echo "WARNING: Unable to clean up table ${DATASET}.${HOURLY_TEAM_SCORE_TABLE_PREFIX}_${runner}. \ + You may need to manually run 'bq rm -r -t ${DATASET}.${HOURLY_TEAM_SCORE_TABLE_PREFIX}_${runner}'." + fi + done +} # Python RC configurations VERSION=$(get_version) @@ -336,4 +362,5 @@ PUBSUB_SUBSCRIPTION='wordstream-python-sub2' # Mobile Gaming Configurations DATASET='beam_postrelease_mobile_gaming' USERSCORE_OUTPUT_PREFIX='python-userscore_result' +HOURLY_TEAM_SCORE_TABLE_PREFIX='hourly_team_score_python' GAME_INPUT_DATA='gs://dataflow-samples/game/5000_gaming_data.csv' diff --git a/release/src/main/python-release/run_release_candidate_python_mobile_gaming.sh b/release/src/main/python-release/run_release_candidate_python_mobile_gaming.sh index 30d8cea2bd95..efd903da48b8 100755 --- a/release/src/main/python-release/run_release_candidate_python_mobile_gaming.sh +++ b/release/src/main/python-release/run_release_candidate_python_mobile_gaming.sh @@ -100,12 +100,15 @@ function verify_userscore_dataflow() { ####################################### function verify_hourlyteamscore_direct() { print_separator "Running HourlyTeamScore example with DirectRunner" + # Clean up old bq tables + cleanup_hourly_team_score "direct" + python -m apache_beam.examples.complete.game.hourly_team_score \ --project=$PROJECT_ID \ --dataset=$DATASET \ --input=$GAME_INPUT_DATA \ --temp_location=gs://$BUCKET_NAME/temp/ \ - --table="hourly_team_score_python_direct" + --table="${HOURLY_TEAM_SCORE_TABLE_PREFIX}_direct" verify_hourly_team_score "direct" } @@ -121,6 +124,9 @@ function verify_hourlyteamscore_direct() { ####################################### function verify_hourlyteamscore_dataflow() { print_separator "Running HourlyTeamScore example with DataflowRunner" + # Clean up old bq tables + cleanup_hourly_team_score "dataflow" + python -m apache_beam.examples.complete.game.hourly_team_score \ --project=$PROJECT_ID \ --region=$REGION_ID \ @@ -129,7 +135,7 @@ function verify_hourlyteamscore_dataflow() { --temp_location=gs://$BUCKET_NAME/temp/ \ --sdk_location $BEAM_PYTHON_SDK \ --input=$GAME_INPUT_DATA \ - --table="hourly_team_score_python_dataflow" + --table="${HOURLY_TEAM_SCORE_TABLE_PREFIX}_dataflow" verify_hourly_team_score "dataflow" } diff --git a/release/src/main/python-release/run_release_candidate_python_quickstart.sh b/release/src/main/python-release/run_release_candidate_python_quickstart.sh index 6f1e10da33b3..3af527acfa2e 100755 --- a/release/src/main/python-release/run_release_candidate_python_quickstart.sh +++ b/release/src/main/python-release/run_release_candidate_python_quickstart.sh @@ -155,7 +155,7 @@ function verify_streaming_wordcount_direct() { # verify result run_pubsub_publish - verify_steaming_result "DirectRunner" $pid + verify_streaming_result "DirectRunner" $pid kill -9 $pid sleep 10 @@ -194,7 +194,7 @@ function verify_streaming_wordcount_dataflow() { # verify result run_pubsub_publish sleep 420 - verify_steaming_result "DataflowRunner" $pid $running_job + verify_streaming_result "DataflowRunner" $pid $running_job kill -9 $pid gcloud dataflow jobs cancel $running_job From f3c504c937f8e0358fcb22c52f50420a7f91ee84 Mon Sep 17 00:00:00 2001 From: egalpin Date: Fri, 4 Feb 2022 15:57:52 -0500 Subject: [PATCH 03/68] Use default context output rather than outputWithTimestamp for ElasticsearchIO --- .../apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java b/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java index 273443a4fdbd..dc29ac6074b0 100644 --- a/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java +++ b/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java @@ -2359,9 +2359,10 @@ private ProcessContextAdapter(DoFn.ProcessContext context) { @Override public void output( - TupleTag tag, Document document, Instant timestamp, BoundedWindow ignored) { - // Note: window is intentionally unused, but required as a param to fit the interface - context.outputWithTimestamp(tag, document, timestamp); + TupleTag tag, Document document, Instant ignored1, BoundedWindow ignored2) { + // Note: window and timestamp are intentionally unused, but required as params to fit the + // interface + context.output(tag, document); } } From b5ee042a244a2ae44d42c93b85a4bbff51385b9d Mon Sep 17 00:00:00 2001 From: Aydar Farrakhov Date: Thu, 24 Feb 2022 17:19:05 +0300 Subject: [PATCH 04/68] Palo Alto case study - fix link --- website/www/site/content/en/case-studies/paloalto.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/website/www/site/content/en/case-studies/paloalto.md b/website/www/site/content/en/case-studies/paloalto.md index 0fdba878fbcd..e751bd833f78 100644 --- a/website/www/site/content/en/case-studies/paloalto.md +++ b/website/www/site/content/en/case-studies/paloalto.md @@ -60,9 +60,9 @@ and flexibility to [over 85K customers](https://www.paloaltonetworks.com/about-u Palo Alto Networks’ integrated security operations platform - [Cortex™](https://www.paloaltonetworks.com/cortex) - applies AI and machine learning to enable security automation, advanced threat intelligence, and effective rapid -security responses for Palo Alto Networks’ customers. (Cortex™ Data -Lake)[https://www.paloaltonetworks.com/cortex/cortex-data-lake] infrastructure collects, integrates, and normalizes -enterprises’ security data combined with trillions of multi-source artifacts. +security responses for Palo Alto Networks’ +customers. [Cortex™ Data Lake](https://www.paloaltonetworks.com/cortex/cortex-data-lake) infrastructure collects, +integrates, and normalizes enterprises’ security data combined with trillions of multi-source artifacts. Cortex™ data infrastructure processes ~10 millions of security log events per second currently, at ~3 PB per day, which are on the high end of real-time streaming processing scale in the industry. Palo Alto Networks’ Sr Principal Software From 7d6a316c2beec3941d19dadf6d320db7d0cff001 Mon Sep 17 00:00:00 2001 From: Miguel Hernandez Date: Fri, 25 Feb 2022 08:53:00 -0600 Subject: [PATCH 05/68] [BEAM-12777] Removed current docs version redirect --- website/www/site/static/.htaccess | 2 -- 1 file changed, 2 deletions(-) diff --git a/website/www/site/static/.htaccess b/website/www/site/static/.htaccess index 45c44bcf57ff..a2ef056a262c 100644 --- a/website/www/site/static/.htaccess +++ b/website/www/site/static/.htaccess @@ -20,7 +20,5 @@ RewriteRule ^(.*)$ https://beam.apache.org/$1 [L,R=301] # path /documentation/sdks/(javadoc|pydoc)/.. # The following redirect maintains the previously supported URLs. RedirectMatch permanent "/documentation/sdks/(javadoc|pydoc)(.*)" "https://beam.apache.org/releases/$1$2" -# Keep this updated to point to the current release. -RedirectMatch "/releases/([^/]+)/current(.*)" "https://beam.apache.org/releases/$1/2.36.0$2" RedirectMatch "/contribute/design-documents" "https://cwiki.apache.org/confluence/display/BEAM/Design+Documents" From 32cfde8b11a86966165dea0c9ca3a16dc9b6cee3 Mon Sep 17 00:00:00 2001 From: Benjamin Gonzalez <74670721+benWize@users.noreply.github.com> Date: Tue, 1 Mar 2022 12:35:17 -0600 Subject: [PATCH 06/68] Merge pull request #16850: [BEAM-11205] Upgrade Libraries BOM dependencies to 24.3.0 * Update GCP Libraries BOM version to 24.3.0 * Update associated dependencies --- .../org/apache/beam/gradle/BeamModulePlugin.groovy | 12 ++++++------ .../container/license_scripts/dep_urls_java.yaml | 2 +- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index b11b590ede86..cb5d97953615 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -446,7 +446,7 @@ class BeamModulePlugin implements Plugin { // a dependency version which should match across multiple // Maven artifacts. def activemq_version = "5.14.5" - def autovalue_version = "1.8.2" + def autovalue_version = "1.9" def autoservice_version = "1.0.1" def aws_java_sdk_version = "1.12.135" def aws_java_sdk2_version = "2.17.106" @@ -463,7 +463,7 @@ class BeamModulePlugin implements Plugin { def google_code_gson_version = "2.8.9" def google_oauth_clients_version = "1.32.1" // Try to keep grpc_version consistent with gRPC version in google_cloud_platform_libraries_bom - def grpc_version = "1.43.2" + def grpc_version = "1.44.0" def guava_version = "31.0.1-jre" def hadoop_version = "2.10.1" def hamcrest_version = "2.1" @@ -479,7 +479,7 @@ class BeamModulePlugin implements Plugin { def postgres_version = "42.2.16" def powermock_version = "2.0.9" // Try to keep protobuf_version consistent with the protobuf version in google_cloud_platform_libraries_bom - def protobuf_version = "3.19.2" + def protobuf_version = "3.19.3" def quickcheck_version = "0.8" def slf4j_version = "1.7.30" def spark2_version = "2.4.8" @@ -575,9 +575,9 @@ class BeamModulePlugin implements Plugin { google_cloud_pubsub : "com.google.cloud:google-cloud-pubsub", // google_cloud_platform_libraries_bom sets version google_cloud_pubsublite : "com.google.cloud:google-cloud-pubsublite", // google_cloud_platform_libraries_bom sets version // The GCP Libraries BOM dashboard shows the versions set by the BOM: - // https://storage.googleapis.com/cloud-opensource-java-dashboard/com.google.cloud/libraries-bom/24.2.0/artifact_details.html + // https://storage.googleapis.com/cloud-opensource-java-dashboard/com.google.cloud/libraries-bom/24.3.0/artifact_details.html // Update libraries-bom version on sdks/java/container/license_scripts/dep_urls_java.yaml - google_cloud_platform_libraries_bom : "com.google.cloud:libraries-bom:24.2.0", + google_cloud_platform_libraries_bom : "com.google.cloud:libraries-bom:24.3.0", google_cloud_spanner : "com.google.cloud:google-cloud-spanner", // google_cloud_platform_libraries_bom sets version google_cloud_spanner_test : "com.google.cloud:google-cloud-spanner:$google_cloud_spanner_version:tests", google_code_gson : "com.google.code.gson:gson:$google_code_gson_version", @@ -648,7 +648,7 @@ class BeamModulePlugin implements Plugin { nemo_compiler_frontend_beam : "org.apache.nemo:nemo-compiler-frontend-beam:$nemo_version", netty_all : "io.netty:netty-all:$netty_version", netty_handler : "io.netty:netty-handler:$netty_version", - netty_tcnative_boringssl_static : "io.netty:netty-tcnative-boringssl-static:2.0.33.Final", + netty_tcnative_boringssl_static : "io.netty:netty-tcnative-boringssl-static:2.0.46.Final", netty_transport_native_epoll : "io.netty:netty-transport-native-epoll:$netty_version", postgres : "org.postgresql:postgresql:$postgres_version", powermock : "org.powermock:powermock-module-junit4:$powermock_version", diff --git a/sdks/java/container/license_scripts/dep_urls_java.yaml b/sdks/java/container/license_scripts/dep_urls_java.yaml index 03a882303586..57745777c26f 100644 --- a/sdks/java/container/license_scripts/dep_urls_java.yaml +++ b/sdks/java/container/license_scripts/dep_urls_java.yaml @@ -42,7 +42,7 @@ jaxen: '1.1.6': type: "3-Clause BSD" libraries-bom: - '24.2.0': + '24.3.0': license: "https://raw.githubusercontent.com/GoogleCloudPlatform/cloud-opensource-java/master/LICENSE" type: "Apache License 2.0" paranamer: From 05eff5ec8c011052722ef44a211450c3f0b2ed85 Mon Sep 17 00:00:00 2001 From: Pavel Avilov Date: Tue, 1 Mar 2022 22:26:14 +0300 Subject: [PATCH 07/68] Merge pull request #16484 from [BEAM-13633] [Playground] Implement method to get a default example for each SDKs * Implement method to get a default example for each SDKs * Add error handling * Added saving of precompiled objects catalog to cache at the server startup * Added caching of the catalog only in case of unspecified SDK * Update regarding comments * Update regarding comments * Simplified logging regarding comment * Get defaultExamplePath from the corresponding config * Refactoring code * Add the `link` field to response * Remove gjson; Resolve conflicts; * Refactoring code * Getting default precompiled object from cache * Refactoring code * Added saving of precompiled objects catalog to cache at the server startup * Added caching of the catalog only in case of unspecified SDK * Update regarding comments * Update regarding comments * Simplified logging regarding comment * Updates regarding comments * Update for environment_service_test.go * Get default example from catalog * GetCatalogFromCacheOrStorage method * Update licenses * Update licenses; Resolve conflicts; * [BEAM-13633][Playground] Change saving default precompiled objects to the cache * [BEAM-13633][Playground] Change logic of saving and receiving info about default precompiled objects * [BEAM-13633][Playground] Separate for each sdk * [BEAM-13633][Playground] regenerate proto files * Add code of the default example to response * Revert "Add code of the default example to response" This reverts commit da6baa0aaa272190d4a035568a5e4db0b093dfd9. * Refactoring code * Refactoring code; Add test; * Edit commentaries * Refactoring code * Add bucket name to methods Co-authored-by: Artur Khanin Co-authored-by: AydarZaynutdinov Co-authored-by: Pavel Avilov --- playground/api/v1/api.proto | 1 + playground/backend/cmd/server/controller.go | 30 +- playground/backend/cmd/server/server.go | 13 + playground/backend/internal/api/v1/api.pb.go | 398 +++++++++--------- playground/backend/internal/cache/cache.go | 11 +- .../internal/cache/local/local_cache.go | 36 +- .../internal/cache/local/local_cache_test.go | 3 +- .../internal/cache/redis/redis_cache.go | 41 ++ .../cloud_bucket/precompiled_objects.go | 106 ++++- .../cloud_bucket/precompiled_objects_test.go | 4 +- .../utils/precompiled_objects_utils.go | 44 ++ .../utils/precompiled_objects_utils_test.go | 70 ++- playground/frontend/lib/api/v1/api.pb.dart | 14 + .../frontend/lib/api/v1/api.pbjson.dart | 3 +- playground/infrastructure/api/v1/api_pb2.py | 83 ++-- playground/infrastructure/cd_helper.py | 35 ++ playground/infrastructure/config.py | 1 + playground/infrastructure/proxy/allow_list.py | 1 + playground/infrastructure/test_cd_helper.py | 20 +- playground/infrastructure/test_helper.py | 9 +- 20 files changed, 651 insertions(+), 272 deletions(-) diff --git a/playground/api/v1/api.proto b/playground/api/v1/api.proto index 4756518ea38b..bddb56bda5c7 100644 --- a/playground/api/v1/api.proto +++ b/playground/api/v1/api.proto @@ -165,6 +165,7 @@ message PrecompiledObject{ string link = 6; bool multifile = 7; int32 context_line = 8; + bool default_example = 9; } // Categories represent the array of messages with sdk and categories at this sdk diff --git a/playground/backend/cmd/server/controller.go b/playground/backend/cmd/server/controller.go index 9397523be0f4..303fea8b0547 100644 --- a/playground/backend/cmd/server/controller.go +++ b/playground/backend/cmd/server/controller.go @@ -260,16 +260,10 @@ func (controller *playgroundController) Cancel(ctx context.Context, info *pb.Can // - If there is no catalog in the cache, gets the catalog from the Storage and saves it to the cache // - If SDK or category is specified in the request, gets the catalog from the cache and filters it by SDK and category func (controller *playgroundController) GetPrecompiledObjects(ctx context.Context, info *pb.GetPrecompiledObjectsRequest) (*pb.GetPrecompiledObjectsResponse, error) { - catalog, err := controller.cacheService.GetCatalog(ctx) + catalog, err := utils.GetCatalogFromCacheOrStorage(ctx, controller.cacheService, controller.env.ApplicationEnvs.BucketName()) if err != nil { - logger.Errorf("GetPrecompiledObjects(): cache error: %s", err.Error()) - catalog, err = utils.GetCatalogFromStorage(ctx, controller.env.ApplicationEnvs.BucketName()) - if err != nil { - return nil, errors.InternalError("Error during getting Precompiled Objects", "Error with cloud connection") - } - if err = controller.cacheService.SetCatalog(ctx, catalog); err != nil { - logger.Errorf("GetPrecompiledObjects(): cache error: %s", err.Error()) - } + logger.Errorf("GetPrecompiledObjects(): error during getting catalog: %s", err.Error()) + return nil, errors.InternalError("Error during getting Precompiled Objects", "Error with cloud connection") } return &pb.GetPrecompiledObjectsResponse{ SdkCategories: utils.FilterCatalog(catalog, info.Sdk, info.Category), @@ -279,7 +273,7 @@ func (controller *playgroundController) GetPrecompiledObjects(ctx context.Contex // GetPrecompiledObjectCode returns the code of the specific example func (controller *playgroundController) GetPrecompiledObjectCode(ctx context.Context, info *pb.GetPrecompiledObjectCodeRequest) (*pb.GetPrecompiledObjectCodeResponse, error) { cd := cloud_bucket.New() - codeString, err := cd.GetPrecompiledObject(ctx, info.GetCloudPath(), controller.env.ApplicationEnvs.BucketName()) + codeString, err := cd.GetPrecompiledObjectCode(ctx, info.GetCloudPath(), controller.env.ApplicationEnvs.BucketName()) if err != nil { logger.Errorf("GetPrecompiledObjectCode(): cloud storage error: %s", err.Error()) return nil, errors.InternalError("Error during getting Precompiled Object's code", "Error with cloud connection") @@ -323,3 +317,19 @@ func (controller *playgroundController) GetPrecompiledObjectGraph(ctx context.Co response := pb.GetPrecompiledObjectGraphResponse{Graph: logs} return &response, nil } + +// GetDefaultPrecompiledObject returns the default precompile object for sdk. +func (controller *playgroundController) GetDefaultPrecompiledObject(ctx context.Context, info *pb.GetDefaultPrecompiledObjectRequest) (*pb.GetDefaultPrecompiledObjectResponse, error) { + switch info.Sdk { + case pb.Sdk_SDK_UNSPECIFIED: + logger.Errorf("GetDefaultPrecompiledObject(): unimplemented sdk: %s\n", info.Sdk) + return nil, errors.InvalidArgumentError("Error during preparing", "Sdk is not implemented yet: %s", info.Sdk.String()) + } + precompiledObject, err := utils.GetDefaultPrecompiledObject(ctx, info.Sdk, controller.cacheService, controller.env.ApplicationEnvs.BucketName()) + if err != nil { + logger.Errorf("GetDefaultPrecompiledObject(): error during getting catalog: %s", err.Error()) + return nil, errors.InternalError("Error during getting Precompiled Objects", "Error with cloud connection") + } + response := pb.GetDefaultPrecompiledObjectResponse{PrecompiledObject: precompiledObject} + return &response, nil +} diff --git a/playground/backend/cmd/server/server.go b/playground/backend/cmd/server/server.go index 03cc61f1516c..b78aa7b06272 100644 --- a/playground/backend/cmd/server/server.go +++ b/playground/backend/cmd/server/server.go @@ -20,6 +20,7 @@ import ( "beam.apache.org/playground/backend/internal/cache" "beam.apache.org/playground/backend/internal/cache/local" "beam.apache.org/playground/backend/internal/cache/redis" + "beam.apache.org/playground/backend/internal/cloud_bucket" "beam.apache.org/playground/backend/internal/environment" "beam.apache.org/playground/backend/internal/logger" "beam.apache.org/playground/backend/internal/utils" @@ -127,6 +128,18 @@ func setupExamplesCatalog(ctx context.Context, cacheService cache.Cache, bucketN if err = cacheService.SetCatalog(ctx, catalog); err != nil { logger.Errorf("GetPrecompiledObjects(): cache error: %s", err.Error()) } + + bucket := cloud_bucket.New() + defaultPrecompiledObjects, err := bucket.GetDefaultPrecompiledObjects(ctx, bucketName) + if err != nil { + return err + } + for sdk, precompiledObject := range defaultPrecompiledObjects { + if err := cacheService.SetDefaultPrecompiledObject(ctx, sdk, precompiledObject); err != nil { + logger.Errorf("GetPrecompiledObjects(): cache error: %s", err.Error()) + return err + } + } return nil } diff --git a/playground/backend/internal/api/v1/api.pb.go b/playground/backend/internal/api/v1/api.pb.go index 09cf94822dab..95023091aaca 100644 --- a/playground/backend/internal/api/v1/api.pb.go +++ b/playground/backend/internal/api/v1/api.pb.go @@ -1203,9 +1203,10 @@ type PrecompiledObject struct { Type PrecompiledObjectType `protobuf:"varint,4,opt,name=type,proto3,enum=api.v1.PrecompiledObjectType" json:"type,omitempty"` PipelineOptions string `protobuf:"bytes,5,opt,name=pipeline_options,json=pipelineOptions,proto3" json:"pipeline_options,omitempty"` // Link to the example in the Beam repository - Link string `protobuf:"bytes,6,opt,name=link,proto3" json:"link,omitempty"` - Multifile bool `protobuf:"varint,7,opt,name=multifile,proto3" json:"multifile,omitempty"` - ContextLine int32 `protobuf:"varint,8,opt,name=context_line,json=contextLine,proto3" json:"context_line,omitempty"` + Link string `protobuf:"bytes,6,opt,name=link,proto3" json:"link,omitempty"` + Multifile bool `protobuf:"varint,7,opt,name=multifile,proto3" json:"multifile,omitempty"` + ContextLine int32 `protobuf:"varint,8,opt,name=context_line,json=contextLine,proto3" json:"context_line,omitempty"` + DefaultExample bool `protobuf:"varint,9,opt,name=default_example,json=defaultExample,proto3" json:"default_example,omitempty"` } func (x *PrecompiledObject) Reset() { @@ -1296,6 +1297,13 @@ func (x *PrecompiledObject) GetContextLine() int32 { return 0 } +func (x *PrecompiledObject) GetDefaultExample() bool { + if x != nil { + return x.DefaultExample + } + return false +} + // Categories represent the array of messages with sdk and categories at this sdk type Categories struct { state protoimpl.MessageState @@ -2066,7 +2074,7 @@ var file_api_v1_api_proto_rawDesc = []byte{ 0x74, 0x12, 0x23, 0x0a, 0x0d, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x75, 0x75, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x55, 0x75, 0x69, 0x64, 0x22, 0x10, 0x0a, 0x0e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x9b, 0x02, 0x0a, 0x11, 0x50, 0x72, 0x65, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xc4, 0x02, 0x0a, 0x11, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x50, 0x61, 0x74, 0x68, 0x12, 0x12, 0x0a, @@ -2084,206 +2092,208 @@ var file_api_v1_api_proto_rawDesc = []byte{ 0x6c, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x66, 0x69, 0x6c, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x5f, 0x6c, 0x69, 0x6e, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, 0x63, 0x6f, 0x6e, 0x74, 0x65, - 0x78, 0x74, 0x4c, 0x69, 0x6e, 0x65, 0x22, 0xe5, 0x01, 0x0a, 0x0a, 0x43, 0x61, 0x74, 0x65, 0x67, - 0x6f, 0x72, 0x69, 0x65, 0x73, 0x12, 0x1d, 0x0a, 0x03, 0x73, 0x64, 0x6b, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0e, 0x32, 0x0b, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x64, 0x6b, 0x52, - 0x03, 0x73, 0x64, 0x6b, 0x12, 0x3b, 0x0a, 0x0a, 0x63, 0x61, 0x74, 0x65, 0x67, 0x6f, 0x72, 0x69, - 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, - 0x31, 0x2e, 0x43, 0x61, 0x74, 0x65, 0x67, 0x6f, 0x72, 0x69, 0x65, 0x73, 0x2e, 0x43, 0x61, 0x74, - 0x65, 0x67, 0x6f, 0x72, 0x79, 0x52, 0x0a, 0x63, 0x61, 0x74, 0x65, 0x67, 0x6f, 0x72, 0x69, 0x65, - 0x73, 0x1a, 0x7b, 0x0a, 0x08, 0x43, 0x61, 0x74, 0x65, 0x67, 0x6f, 0x72, 0x79, 0x12, 0x23, 0x0a, - 0x0d, 0x63, 0x61, 0x74, 0x65, 0x67, 0x6f, 0x72, 0x79, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x61, 0x74, 0x65, 0x67, 0x6f, 0x72, 0x79, 0x4e, 0x61, - 0x6d, 0x65, 0x12, 0x4a, 0x0a, 0x13, 0x70, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, - 0x64, 0x5f, 0x6f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x19, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, - 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x52, 0x12, 0x70, 0x72, 0x65, 0x63, - 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x22, 0x59, - 0x0a, 0x1c, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, - 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, + 0x78, 0x74, 0x4c, 0x69, 0x6e, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, + 0x74, 0x5f, 0x65, 0x78, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x0e, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x45, 0x78, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x22, + 0xe5, 0x01, 0x0a, 0x0a, 0x43, 0x61, 0x74, 0x65, 0x67, 0x6f, 0x72, 0x69, 0x65, 0x73, 0x12, 0x1d, 0x0a, 0x03, 0x73, 0x64, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0b, 0x2e, 0x61, 0x70, - 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x64, 0x6b, 0x52, 0x03, 0x73, 0x64, 0x6b, 0x12, 0x1a, 0x0a, - 0x08, 0x63, 0x61, 0x74, 0x65, 0x67, 0x6f, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x08, 0x63, 0x61, 0x74, 0x65, 0x67, 0x6f, 0x72, 0x79, 0x22, 0x40, 0x0a, 0x1f, 0x47, 0x65, 0x74, - 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, - 0x74, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, - 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x09, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x50, 0x61, 0x74, 0x68, 0x22, 0x42, 0x0a, 0x21, 0x47, - 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, - 0x65, 0x63, 0x74, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x1d, 0x0a, 0x0a, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x50, 0x61, 0x74, 0x68, 0x22, - 0x40, 0x0a, 0x1f, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, - 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x5f, 0x70, 0x61, 0x74, 0x68, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x50, 0x61, 0x74, - 0x68, 0x22, 0x41, 0x0a, 0x20, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, - 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x47, 0x72, 0x61, 0x70, 0x68, 0x52, 0x65, + 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x64, 0x6b, 0x52, 0x03, 0x73, 0x64, 0x6b, 0x12, 0x3b, 0x0a, + 0x0a, 0x63, 0x61, 0x74, 0x65, 0x67, 0x6f, 0x72, 0x69, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x1b, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x74, 0x65, 0x67, + 0x6f, 0x72, 0x69, 0x65, 0x73, 0x2e, 0x43, 0x61, 0x74, 0x65, 0x67, 0x6f, 0x72, 0x79, 0x52, 0x0a, + 0x63, 0x61, 0x74, 0x65, 0x67, 0x6f, 0x72, 0x69, 0x65, 0x73, 0x1a, 0x7b, 0x0a, 0x08, 0x43, 0x61, + 0x74, 0x65, 0x67, 0x6f, 0x72, 0x79, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x61, 0x74, 0x65, 0x67, 0x6f, + 0x72, 0x79, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, + 0x61, 0x74, 0x65, 0x67, 0x6f, 0x72, 0x79, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x4a, 0x0a, 0x13, 0x70, + 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x5f, 0x6f, 0x62, 0x6a, 0x65, 0x63, + 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, + 0x31, 0x2e, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, + 0x65, 0x63, 0x74, 0x52, 0x12, 0x70, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, + 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x22, 0x59, 0x0a, 0x1c, 0x47, 0x65, 0x74, 0x50, 0x72, + 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x03, 0x73, 0x64, 0x6b, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0b, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x64, + 0x6b, 0x52, 0x03, 0x73, 0x64, 0x6b, 0x12, 0x1a, 0x0a, 0x08, 0x63, 0x61, 0x74, 0x65, 0x67, 0x6f, + 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x63, 0x61, 0x74, 0x65, 0x67, 0x6f, + 0x72, 0x79, 0x22, 0x40, 0x0a, 0x1f, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, + 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x6f, 0x75, 0x64, - 0x50, 0x61, 0x74, 0x68, 0x22, 0x43, 0x0a, 0x22, 0x47, 0x65, 0x74, 0x44, 0x65, 0x66, 0x61, 0x75, - 0x6c, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, - 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x03, 0x73, 0x64, - 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0b, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, - 0x2e, 0x53, 0x64, 0x6b, 0x52, 0x03, 0x73, 0x64, 0x6b, 0x22, 0x5a, 0x0a, 0x1d, 0x47, 0x65, 0x74, - 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, - 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x39, 0x0a, 0x0e, 0x73, 0x64, - 0x6b, 0x5f, 0x63, 0x61, 0x74, 0x65, 0x67, 0x6f, 0x72, 0x69, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x74, 0x65, - 0x67, 0x6f, 0x72, 0x69, 0x65, 0x73, 0x52, 0x0d, 0x73, 0x64, 0x6b, 0x43, 0x61, 0x74, 0x65, 0x67, - 0x6f, 0x72, 0x69, 0x65, 0x73, 0x22, 0x36, 0x0a, 0x20, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, - 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x43, 0x6f, 0x64, - 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x6f, 0x64, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3c, 0x0a, - 0x22, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, - 0x62, 0x6a, 0x65, 0x63, 0x74, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x06, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x22, 0x3a, 0x0a, 0x20, 0x47, - 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, - 0x65, 0x63, 0x74, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, - 0x16, 0x0a, 0x06, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x06, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x22, 0x39, 0x0a, 0x21, 0x47, 0x65, 0x74, 0x50, 0x72, - 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x47, - 0x72, 0x61, 0x70, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, - 0x67, 0x72, 0x61, 0x70, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x67, 0x72, 0x61, - 0x70, 0x68, 0x22, 0x6f, 0x0a, 0x23, 0x47, 0x65, 0x74, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, - 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, - 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48, 0x0a, 0x12, 0x70, 0x72, 0x65, - 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x5f, 0x6f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, + 0x50, 0x61, 0x74, 0x68, 0x22, 0x42, 0x0a, 0x21, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, + 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x4f, 0x75, 0x74, 0x70, + 0x75, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x50, 0x61, 0x74, 0x68, 0x22, 0x40, 0x0a, 0x1f, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, - 0x52, 0x11, 0x70, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, - 0x65, 0x63, 0x74, 0x2a, 0x52, 0x0a, 0x03, 0x53, 0x64, 0x6b, 0x12, 0x13, 0x0a, 0x0f, 0x53, 0x44, - 0x4b, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, - 0x0c, 0x0a, 0x08, 0x53, 0x44, 0x4b, 0x5f, 0x4a, 0x41, 0x56, 0x41, 0x10, 0x01, 0x12, 0x0a, 0x0a, - 0x06, 0x53, 0x44, 0x4b, 0x5f, 0x47, 0x4f, 0x10, 0x02, 0x12, 0x0e, 0x0a, 0x0a, 0x53, 0x44, 0x4b, - 0x5f, 0x50, 0x59, 0x54, 0x48, 0x4f, 0x4e, 0x10, 0x03, 0x12, 0x0c, 0x0a, 0x08, 0x53, 0x44, 0x4b, - 0x5f, 0x53, 0x43, 0x49, 0x4f, 0x10, 0x04, 0x2a, 0xb8, 0x02, 0x0a, 0x06, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x12, 0x16, 0x0a, 0x12, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x55, 0x4e, 0x53, - 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x15, 0x0a, 0x11, 0x53, 0x54, - 0x41, 0x54, 0x55, 0x53, 0x5f, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x41, 0x54, 0x49, 0x4e, 0x47, 0x10, - 0x01, 0x12, 0x1b, 0x0a, 0x17, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x56, 0x41, 0x4c, 0x49, - 0x44, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x02, 0x12, 0x14, - 0x0a, 0x10, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x50, 0x52, 0x45, 0x50, 0x41, 0x52, 0x49, - 0x4e, 0x47, 0x10, 0x03, 0x12, 0x1c, 0x0a, 0x18, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x50, - 0x52, 0x45, 0x50, 0x41, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x45, 0x52, 0x52, 0x4f, 0x52, - 0x10, 0x04, 0x12, 0x14, 0x0a, 0x10, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x43, 0x4f, 0x4d, - 0x50, 0x49, 0x4c, 0x49, 0x4e, 0x47, 0x10, 0x05, 0x12, 0x18, 0x0a, 0x14, 0x53, 0x54, 0x41, 0x54, - 0x55, 0x53, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x49, 0x4c, 0x45, 0x5f, 0x45, 0x52, 0x52, 0x4f, 0x52, - 0x10, 0x06, 0x12, 0x14, 0x0a, 0x10, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x45, 0x58, 0x45, - 0x43, 0x55, 0x54, 0x49, 0x4e, 0x47, 0x10, 0x07, 0x12, 0x13, 0x0a, 0x0f, 0x53, 0x54, 0x41, 0x54, - 0x55, 0x53, 0x5f, 0x46, 0x49, 0x4e, 0x49, 0x53, 0x48, 0x45, 0x44, 0x10, 0x08, 0x12, 0x14, 0x0a, - 0x10, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x52, 0x55, 0x4e, 0x5f, 0x45, 0x52, 0x52, 0x4f, - 0x52, 0x10, 0x09, 0x12, 0x10, 0x0a, 0x0c, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x45, 0x52, - 0x52, 0x4f, 0x52, 0x10, 0x0a, 0x12, 0x16, 0x0a, 0x12, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, - 0x52, 0x55, 0x4e, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x4f, 0x55, 0x54, 0x10, 0x0b, 0x12, 0x13, 0x0a, - 0x0f, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x45, 0x44, - 0x10, 0x0c, 0x2a, 0xae, 0x01, 0x0a, 0x15, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, - 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x27, 0x0a, 0x23, - 0x50, 0x52, 0x45, 0x43, 0x4f, 0x4d, 0x50, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x4f, 0x42, 0x4a, 0x45, - 0x43, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, - 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x23, 0x0a, 0x1f, 0x50, 0x52, 0x45, 0x43, 0x4f, 0x4d, 0x50, + 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x50, 0x61, 0x74, 0x68, 0x22, 0x41, 0x0a, 0x20, 0x47, 0x65, + 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, + 0x63, 0x74, 0x47, 0x72, 0x61, 0x70, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, + 0x0a, 0x0a, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x50, 0x61, 0x74, 0x68, 0x22, 0x43, 0x0a, + 0x22, 0x47, 0x65, 0x74, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, + 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x03, 0x73, 0x64, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x0b, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x64, 0x6b, 0x52, 0x03, 0x73, + 0x64, 0x6b, 0x22, 0x5a, 0x0a, 0x1d, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, + 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x39, 0x0a, 0x0e, 0x73, 0x64, 0x6b, 0x5f, 0x63, 0x61, 0x74, 0x65, 0x67, + 0x6f, 0x72, 0x69, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x74, 0x65, 0x67, 0x6f, 0x72, 0x69, 0x65, 0x73, 0x52, + 0x0d, 0x73, 0x64, 0x6b, 0x43, 0x61, 0x74, 0x65, 0x67, 0x6f, 0x72, 0x69, 0x65, 0x73, 0x22, 0x36, + 0x0a, 0x20, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, + 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3c, 0x0a, 0x22, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, + 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x4f, 0x75, + 0x74, 0x70, 0x75, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x16, 0x0a, 0x06, + 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x6f, 0x75, + 0x74, 0x70, 0x75, 0x74, 0x22, 0x3a, 0x0a, 0x20, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, + 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x4c, 0x6f, 0x67, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x6f, 0x75, 0x74, 0x70, + 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, + 0x22, 0x39, 0x0a, 0x21, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, + 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x47, 0x72, 0x61, 0x70, 0x68, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x67, 0x72, 0x61, 0x70, 0x68, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x67, 0x72, 0x61, 0x70, 0x68, 0x22, 0x6f, 0x0a, 0x23, 0x47, + 0x65, 0x74, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, + 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x48, 0x0a, 0x12, 0x70, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, + 0x64, 0x5f, 0x6f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, + 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x52, 0x11, 0x70, 0x72, 0x65, 0x63, 0x6f, + 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x2a, 0x52, 0x0a, 0x03, + 0x53, 0x64, 0x6b, 0x12, 0x13, 0x0a, 0x0f, 0x53, 0x44, 0x4b, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, + 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x53, 0x44, 0x4b, 0x5f, + 0x4a, 0x41, 0x56, 0x41, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x44, 0x4b, 0x5f, 0x47, 0x4f, + 0x10, 0x02, 0x12, 0x0e, 0x0a, 0x0a, 0x53, 0x44, 0x4b, 0x5f, 0x50, 0x59, 0x54, 0x48, 0x4f, 0x4e, + 0x10, 0x03, 0x12, 0x0c, 0x0a, 0x08, 0x53, 0x44, 0x4b, 0x5f, 0x53, 0x43, 0x49, 0x4f, 0x10, 0x04, + 0x2a, 0xb8, 0x02, 0x0a, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x16, 0x0a, 0x12, 0x53, + 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, + 0x44, 0x10, 0x00, 0x12, 0x15, 0x0a, 0x11, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x56, 0x41, + 0x4c, 0x49, 0x44, 0x41, 0x54, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x1b, 0x0a, 0x17, 0x53, 0x54, + 0x41, 0x54, 0x55, 0x53, 0x5f, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, + 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x02, 0x12, 0x14, 0x0a, 0x10, 0x53, 0x54, 0x41, 0x54, 0x55, + 0x53, 0x5f, 0x50, 0x52, 0x45, 0x50, 0x41, 0x52, 0x49, 0x4e, 0x47, 0x10, 0x03, 0x12, 0x1c, 0x0a, + 0x18, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x50, 0x52, 0x45, 0x50, 0x41, 0x52, 0x41, 0x54, + 0x49, 0x4f, 0x4e, 0x5f, 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x04, 0x12, 0x14, 0x0a, 0x10, 0x53, + 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x49, 0x4c, 0x49, 0x4e, 0x47, 0x10, + 0x05, 0x12, 0x18, 0x0a, 0x14, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x43, 0x4f, 0x4d, 0x50, + 0x49, 0x4c, 0x45, 0x5f, 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x06, 0x12, 0x14, 0x0a, 0x10, 0x53, + 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4e, 0x47, 0x10, + 0x07, 0x12, 0x13, 0x0a, 0x0f, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x46, 0x49, 0x4e, 0x49, + 0x53, 0x48, 0x45, 0x44, 0x10, 0x08, 0x12, 0x14, 0x0a, 0x10, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, + 0x5f, 0x52, 0x55, 0x4e, 0x5f, 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x09, 0x12, 0x10, 0x0a, 0x0c, + 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x0a, 0x12, 0x16, + 0x0a, 0x12, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x52, 0x55, 0x4e, 0x5f, 0x54, 0x49, 0x4d, + 0x45, 0x4f, 0x55, 0x54, 0x10, 0x0b, 0x12, 0x13, 0x0a, 0x0f, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, + 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x45, 0x44, 0x10, 0x0c, 0x2a, 0xae, 0x01, 0x0a, 0x15, + 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, + 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x27, 0x0a, 0x23, 0x50, 0x52, 0x45, 0x43, 0x4f, 0x4d, 0x50, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x4f, 0x42, 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, - 0x5f, 0x45, 0x58, 0x41, 0x4d, 0x50, 0x4c, 0x45, 0x10, 0x01, 0x12, 0x20, 0x0a, 0x1c, 0x50, 0x52, - 0x45, 0x43, 0x4f, 0x4d, 0x50, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x4f, 0x42, 0x4a, 0x45, 0x43, 0x54, - 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4b, 0x41, 0x54, 0x41, 0x10, 0x02, 0x12, 0x25, 0x0a, 0x21, - 0x50, 0x52, 0x45, 0x43, 0x4f, 0x4d, 0x50, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x4f, 0x42, 0x4a, 0x45, - 0x43, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x49, 0x54, 0x5f, 0x54, 0x45, 0x53, - 0x54, 0x10, 0x03, 0x32, 0x9b, 0x0b, 0x0a, 0x11, 0x50, 0x6c, 0x61, 0x79, 0x67, 0x72, 0x6f, 0x75, - 0x6e, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x3a, 0x0a, 0x07, 0x52, 0x75, 0x6e, - 0x43, 0x6f, 0x64, 0x65, 0x12, 0x16, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x75, - 0x6e, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x17, 0x2e, 0x61, - 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x75, 0x6e, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x46, 0x0a, 0x0b, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x53, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x12, 0x1a, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x68, - 0x65, 0x63, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x1b, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x53, - 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x49, 0x0a, - 0x0c, 0x47, 0x65, 0x74, 0x52, 0x75, 0x6e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x1b, 0x2e, - 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x75, 0x6e, 0x4f, 0x75, 0x74, - 0x70, 0x75, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x61, 0x70, 0x69, - 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x75, 0x6e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3a, 0x0a, 0x07, 0x47, 0x65, 0x74, 0x4c, - 0x6f, 0x67, 0x73, 0x12, 0x16, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, - 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x17, 0x2e, 0x61, 0x70, - 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3d, 0x0a, 0x08, 0x47, 0x65, 0x74, 0x47, 0x72, 0x61, 0x70, 0x68, - 0x12, 0x17, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x47, 0x72, 0x61, - 0x70, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x18, 0x2e, 0x61, 0x70, 0x69, 0x2e, - 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x47, 0x72, 0x61, 0x70, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x46, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x52, 0x75, 0x6e, 0x45, 0x72, 0x72, - 0x6f, 0x72, 0x12, 0x1a, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x52, - 0x75, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, - 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x75, 0x6e, 0x45, 0x72, - 0x72, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5e, 0x0a, 0x13, 0x47, - 0x65, 0x74, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x75, 0x74, 0x70, - 0x75, 0x74, 0x12, 0x22, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x56, - 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, - 0x47, 0x65, 0x74, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x75, 0x74, - 0x70, 0x75, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x61, 0x0a, 0x14, 0x47, - 0x65, 0x74, 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x75, 0x74, - 0x70, 0x75, 0x74, 0x12, 0x23, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, - 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x75, 0x74, 0x70, 0x75, - 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, - 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x55, - 0x0a, 0x10, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x4f, 0x75, 0x74, 0x70, - 0x75, 0x74, 0x12, 0x1f, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x43, - 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, - 0x43, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x37, 0x0a, 0x06, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x12, - 0x15, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, - 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x64, - 0x0a, 0x15, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, - 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x12, 0x24, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, - 0x2e, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, - 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, - 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, - 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x6d, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, - 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x43, 0x6f, 0x64, 0x65, - 0x12, 0x27, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, - 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x43, 0x6f, - 0x64, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x23, + 0x0a, 0x1f, 0x50, 0x52, 0x45, 0x43, 0x4f, 0x4d, 0x50, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x4f, 0x42, + 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x45, 0x58, 0x41, 0x4d, 0x50, 0x4c, + 0x45, 0x10, 0x01, 0x12, 0x20, 0x0a, 0x1c, 0x50, 0x52, 0x45, 0x43, 0x4f, 0x4d, 0x50, 0x49, 0x4c, + 0x45, 0x44, 0x5f, 0x4f, 0x42, 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4b, + 0x41, 0x54, 0x41, 0x10, 0x02, 0x12, 0x25, 0x0a, 0x21, 0x50, 0x52, 0x45, 0x43, 0x4f, 0x4d, 0x50, + 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x4f, 0x42, 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, + 0x5f, 0x55, 0x4e, 0x49, 0x54, 0x5f, 0x54, 0x45, 0x53, 0x54, 0x10, 0x03, 0x32, 0x9b, 0x0b, 0x0a, + 0x11, 0x50, 0x6c, 0x61, 0x79, 0x67, 0x72, 0x6f, 0x75, 0x6e, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x12, 0x3a, 0x0a, 0x07, 0x52, 0x75, 0x6e, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x16, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x75, 0x6e, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x17, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x52, + 0x75, 0x6e, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x46, + 0x0a, 0x0b, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1a, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x76, 0x31, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x49, 0x0a, 0x0c, 0x47, 0x65, 0x74, 0x52, 0x75, 0x6e, + 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x1b, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, + 0x47, 0x65, 0x74, 0x52, 0x75, 0x6e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, + 0x52, 0x75, 0x6e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x3a, 0x0a, 0x07, 0x47, 0x65, 0x74, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x16, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x17, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, + 0x74, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3d, 0x0a, + 0x08, 0x47, 0x65, 0x74, 0x47, 0x72, 0x61, 0x70, 0x68, 0x12, 0x17, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x47, 0x72, 0x61, 0x70, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x18, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x47, + 0x72, 0x61, 0x70, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x46, 0x0a, 0x0b, + 0x47, 0x65, 0x74, 0x52, 0x75, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x1a, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x75, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, + 0x2e, 0x47, 0x65, 0x74, 0x52, 0x75, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5e, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x56, 0x61, 0x6c, 0x69, 0x64, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x22, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x23, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x56, 0x61, 0x6c, 0x69, + 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x61, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x70, 0x61, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x23, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x24, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x72, + 0x65, 0x70, 0x61, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x55, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x43, 0x6f, + 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x1f, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x4f, + 0x75, 0x74, 0x70, 0x75, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, + 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x37, + 0x0a, 0x06, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x12, 0x15, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, + 0x31, 0x2e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x16, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x64, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x50, 0x72, + 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, + 0x12, 0x24, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, + 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, + 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, + 0x6a, 0x65, 0x63, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x6d, 0x0a, + 0x18, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, + 0x62, 0x6a, 0x65, 0x63, 0x74, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x27, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, - 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x73, 0x0a, 0x1a, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, - 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x4f, 0x75, 0x74, 0x70, 0x75, - 0x74, 0x12, 0x29, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x72, - 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x4f, - 0x75, 0x74, 0x70, 0x75, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x61, - 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, - 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x6d, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x50, + 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, - 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x27, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, - 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, - 0x63, 0x74, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, + 0x43, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x73, 0x0a, 0x1a, + 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, + 0x6a, 0x65, 0x63, 0x74, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x29, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, + 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, + 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, + 0x65, 0x63, 0x74, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x6d, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, + 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x27, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x4c, 0x6f, 0x67, 0x73, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x70, 0x0a, 0x19, 0x47, 0x65, 0x74, 0x50, 0x72, - 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x47, - 0x72, 0x61, 0x70, 0x68, 0x12, 0x28, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, + 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, + 0x6a, 0x65, 0x63, 0x74, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x70, 0x0a, 0x19, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, + 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x47, 0x72, 0x61, 0x70, 0x68, 0x12, 0x28, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, + 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x47, 0x72, 0x61, 0x70, 0x68, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, + 0x2e, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, + 0x62, 0x6a, 0x65, 0x63, 0x74, 0x47, 0x72, 0x61, 0x70, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x76, 0x0a, 0x1b, 0x47, 0x65, 0x74, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, + 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, + 0x74, 0x12, 0x2a, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x44, 0x65, + 0x66, 0x61, 0x75, 0x6c, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, + 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, - 0x63, 0x74, 0x47, 0x72, 0x61, 0x70, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, - 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, - 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x47, 0x72, 0x61, 0x70, - 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x76, 0x0a, 0x1b, 0x47, 0x65, 0x74, - 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, 0x6c, - 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x12, 0x2a, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, - 0x31, 0x2e, 0x47, 0x65, 0x74, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x50, 0x72, 0x65, 0x63, - 0x6f, 0x6d, 0x70, 0x69, 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, - 0x74, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x50, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x70, 0x69, - 0x6c, 0x65, 0x64, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x42, 0x38, 0x5a, 0x36, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, - 0x2e, 0x6f, 0x72, 0x67, 0x2f, 0x70, 0x6c, 0x61, 0x79, 0x67, 0x72, 0x6f, 0x75, 0x6e, 0x64, 0x2f, - 0x62, 0x61, 0x63, 0x6b, 0x65, 0x6e, 0x64, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, - 0x3b, 0x70, 0x6c, 0x61, 0x79, 0x67, 0x72, 0x6f, 0x75, 0x6e, 0x64, 0x62, 0x06, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x33, + 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x42, 0x38, 0x5a, 0x36, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x6f, 0x72, 0x67, 0x2f, 0x70, 0x6c, + 0x61, 0x79, 0x67, 0x72, 0x6f, 0x75, 0x6e, 0x64, 0x2f, 0x62, 0x61, 0x63, 0x6b, 0x65, 0x6e, 0x64, + 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x3b, 0x70, 0x6c, 0x61, 0x79, 0x67, 0x72, + 0x6f, 0x75, 0x6e, 0x64, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( diff --git a/playground/backend/internal/cache/cache.go b/playground/backend/internal/cache/cache.go index e207f07f75b9..2b30f145378b 100644 --- a/playground/backend/internal/cache/cache.go +++ b/playground/backend/internal/cache/cache.go @@ -62,6 +62,9 @@ const ( // ExamplesCatalog is catalog of examples available in Playground ExamplesCatalog string = "EXAMPLES_CATALOG" + + // DefaultPrecompiledExamples is used to keep default examples + DefaultPrecompiledExamples string = "DEFAULT_PRECOMPILED_OBJECTS" ) // Cache is used to store states and outputs for Apache Beam pipelines that running in Playground @@ -86,6 +89,12 @@ type Cache interface { // SetCatalog adds the given catalog to cache by ExamplesCatalog key. SetCatalog(ctx context.Context, catalog []*pb.Categories) error - // GetCatalog return catalog from cache by ExamplesCatalog key. + // GetCatalog returns catalog from cache by ExamplesCatalog key. GetCatalog(ctx context.Context) ([]*pb.Categories, error) + + // SetDefaultPrecompiledObject adds default precompiled object for SDK into cache. + SetDefaultPrecompiledObject(ctx context.Context, sdk pb.Sdk, precompiledObject *pb.PrecompiledObject) error + + // GetDefaultPrecompiledObject returns default precompiled object for SDK from cache. + GetDefaultPrecompiledObject(ctx context.Context, sdk pb.Sdk) (*pb.PrecompiledObject, error) } diff --git a/playground/backend/internal/cache/local/local_cache.go b/playground/backend/internal/cache/local/local_cache.go index bdc149f6a3a1..64c39b6863ad 100644 --- a/playground/backend/internal/cache/local/local_cache.go +++ b/playground/backend/internal/cache/local/local_cache.go @@ -31,20 +31,23 @@ const ( type Cache struct { sync.RWMutex - cleanupInterval time.Duration - items map[uuid.UUID]map[cache.SubKey]interface{} - pipelinesExpiration map[uuid.UUID]time.Time - catalog []*pb.Categories + cleanupInterval time.Duration + items map[uuid.UUID]map[cache.SubKey]interface{} + pipelinesExpiration map[uuid.UUID]time.Time + catalog []*pb.Categories + defaultPrecompiledObjects map[pb.Sdk]*pb.PrecompiledObject } func New(ctx context.Context) *Cache { items := make(map[uuid.UUID]map[cache.SubKey]interface{}) pipelinesExpiration := make(map[uuid.UUID]time.Time) + defaultPrecompiledObjects := make(map[pb.Sdk]*pb.PrecompiledObject) ls := &Cache{ - cleanupInterval: cleanupInterval, - items: items, - pipelinesExpiration: pipelinesExpiration, - catalog: nil, + cleanupInterval: cleanupInterval, + items: items, + pipelinesExpiration: pipelinesExpiration, + catalog: nil, + defaultPrecompiledObjects: defaultPrecompiledObjects, } go ls.startGC(ctx) @@ -124,6 +127,23 @@ func (lc *Cache) GetCatalog(ctx context.Context) ([]*pb.Categories, error) { return lc.catalog, nil } +func (lc *Cache) SetDefaultPrecompiledObject(ctx context.Context, sdk pb.Sdk, precompiledObject *pb.PrecompiledObject) error { + lc.Lock() + defer lc.Unlock() + lc.defaultPrecompiledObjects[sdk] = precompiledObject + return nil +} + +func (lc *Cache) GetDefaultPrecompiledObject(ctx context.Context, sdk pb.Sdk) (*pb.PrecompiledObject, error) { + lc.RLock() + defer lc.RUnlock() + defaultPrecompiledObject := lc.defaultPrecompiledObjects[sdk] + if defaultPrecompiledObject == nil { + return nil, fmt.Errorf("default precompiled obejct is not found for %s sdk", sdk.String()) + } + return defaultPrecompiledObject, nil +} + func (lc *Cache) startGC(ctx context.Context) { ticker := time.NewTicker(lc.cleanupInterval) for { diff --git a/playground/backend/internal/cache/local/local_cache_test.go b/playground/backend/internal/cache/local/local_cache_test.go index 34b2654ad065..839d2dd5407a 100644 --- a/playground/backend/internal/cache/local/local_cache_test.go +++ b/playground/backend/internal/cache/local/local_cache_test.go @@ -19,6 +19,7 @@ import ( pb "beam.apache.org/playground/backend/internal/api/v1" "beam.apache.org/playground/backend/internal/cache" "context" + "fmt" "github.com/google/uuid" "go.uber.org/goleak" "reflect" @@ -625,7 +626,7 @@ func TestNew(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - if got := New(tt.args.ctx); !reflect.DeepEqual(got, tt.want) { + if got := New(tt.args.ctx); !reflect.DeepEqual(fmt.Sprint(got), fmt.Sprint(tt.want)) { t.Errorf("New() = %v, want %v", got, tt.want) } }) diff --git a/playground/backend/internal/cache/redis/redis_cache.go b/playground/backend/internal/cache/redis/redis_cache.go index b3cf206e30ce..31bc0a479d24 100644 --- a/playground/backend/internal/cache/redis/redis_cache.go +++ b/playground/backend/internal/cache/redis/redis_cache.go @@ -126,6 +126,47 @@ func (rc *Cache) GetCatalog(ctx context.Context) ([]*pb.Categories, error) { return result, nil } +func (rc *Cache) SetDefaultPrecompiledObject(ctx context.Context, sdk pb.Sdk, precompiledObject *pb.PrecompiledObject) error { + precompiledObjectMarsh, err := json.Marshal(precompiledObject) + if err != nil { + logger.Errorf("Redis Cache: set default precompiled object: error during marshal precompiled object: %s, err: %s\n", precompiledObject, err.Error()) + return err + } + sdkMarsh, err := json.Marshal(sdk) + if err != nil { + logger.Errorf("Redis Cache: set default precompiled object: error during marshal sdk: %s, err: %s\n", sdk, err.Error()) + return err + } + err = rc.HSet(ctx, cache.DefaultPrecompiledExamples, sdkMarsh, precompiledObjectMarsh).Err() + if err != nil { + logger.Errorf("Redis Cache: set default precompiled object: error during HGet operation, err: %s\n", err.Error()) + return err + } + return nil + +} + +func (rc *Cache) GetDefaultPrecompiledObject(ctx context.Context, sdk pb.Sdk) (*pb.PrecompiledObject, error) { + sdkMarsh, err := json.Marshal(sdk) + if err != nil { + logger.Errorf("Redis Cache: get default precompiled object: error during marshal sdk: %s, err: %s\n", sdk, err.Error()) + return nil, err + } + + value, err := rc.HGet(ctx, cache.DefaultPrecompiledExamples, string(sdkMarsh)).Result() + if err != nil { + logger.Errorf("Redis Cache: get default precompiled object: error during HGet operation for key: %s, subKey: %s, err: %s\n", cache.DefaultPrecompiledExamples, sdkMarsh, err.Error()) + return nil, err + } + + result := new(pb.PrecompiledObject) + err = json.Unmarshal([]byte(value), &result) + if err != nil { + logger.Errorf("Redis Cache: get default precompiled object: error during unmarshal value, err: %s\n", err.Error()) + } + return result, nil +} + // unmarshalBySubKey unmarshal value by subKey func unmarshalBySubKey(subKey cache.SubKey, value string) (interface{}, error) { var result interface{} diff --git a/playground/backend/internal/cloud_bucket/precompiled_objects.go b/playground/backend/internal/cloud_bucket/precompiled_objects.go index 09dda526a9a2..945c5bcfb229 100644 --- a/playground/backend/internal/cloud_bucket/precompiled_objects.go +++ b/playground/backend/internal/cloud_bucket/precompiled_objects.go @@ -24,6 +24,7 @@ import ( "fmt" "google.golang.org/api/iterator" "google.golang.org/api/option" + "io" "io/ioutil" "os" "path/filepath" @@ -32,16 +33,17 @@ import ( ) const ( - OutputExtension = "output" - LogsExtension = "log" - GraphExtension = "graph" - MetaInfoName = "meta.info" - Timeout = time.Minute - javaExtension = "java" - goExtension = "go" - pyExtension = "py" - scioExtension = "scala" - separatorsNumber = 3 + OutputExtension = "output" + LogsExtension = "log" + GraphExtension = "graph" + defaultPrecompiledObjectInfo = "defaultPrecompiledObject.info" + MetaInfoName = "meta.info" + Timeout = time.Minute + javaExtension = "java" + goExtension = "go" + pyExtension = "py" + scioExtension = "scala" + separatorsNumber = 3 ) type ObjectInfo struct { @@ -54,6 +56,7 @@ type ObjectInfo struct { Link string `protobuf:"bytes,3,opt,name=link,proto3" json:"link,omitempty"` Multifile bool `protobuf:"varint,7,opt,name=multifile,proto3" json:"multifile,omitempty"` ContextLine int32 `protobuf:"varint,7,opt,name=context_line,proto3" json:"context_line,omitempty"` + DefaultExample bool `protobuf:"varint,7,opt,name=default_example,json=defaultExample,proto3" json:"default_example,omitempty"` } type PrecompiledObjects []ObjectInfo @@ -65,6 +68,7 @@ type SdkToCategories map[string]CategoryToPrecompiledObjects // the bucket where examples are stored would be public, // and it has a specific structure of files, namely: // SDK_JAVA/ +// ----defaultPrecompiledObject.info // ----PRECOMPILED_OBJECT_TYPE_EXAMPLE/ // --------MinimalWordCount/ // ----------- MinimalWordCount.java @@ -82,6 +86,7 @@ type SdkToCategories map[string]CategoryToPrecompiledObjects // --------... // ----... // SDK_GO/ +// ----defaultPrecompiledObject.info // ----PRECOMPILED_OBJECT_TYPE_EXAMPLE/ // --------MinimalWordCount/ // ----------- MinimalWordCount.go @@ -93,6 +98,12 @@ type SdkToCategories map[string]CategoryToPrecompiledObjects // ----PRECOMPILED_OBJECT_TYPE_KATA/ // --------... // ----... +// +// defaultPrecompiledObject.info is a file that contains path to the default example: +// { +// "SDK_JAVA": "SDK_JAVA/PRECOMPILED_OBJECT_TYPE_EXAMPLE/MinimalWordCount" +// } +// // meta.info is a json file that has the following fields: // { // "name": "name of the example", @@ -112,8 +123,8 @@ func New() *CloudStorage { return &CloudStorage{} } -// GetPrecompiledObject returns the source code of the example -func (cd *CloudStorage) GetPrecompiledObject(ctx context.Context, precompiledObjectPath, bucketName string) (string, error) { +// GetPrecompiledObjectCode returns the source code of the example +func (cd *CloudStorage) GetPrecompiledObjectCode(ctx context.Context, precompiledObjectPath, bucketName string) (string, error) { extension, err := getFileExtensionBySdk(precompiledObjectPath) if err != nil { return "", err @@ -210,6 +221,77 @@ func (cd *CloudStorage) GetPrecompiledObjects(ctx context.Context, targetSdk pb. return &precompiledObjects, nil } +// GetDefaultPrecompiledObjects returns the default precompiled objects +func (cd *CloudStorage) GetDefaultPrecompiledObjects(ctx context.Context, bucketName string) (map[pb.Sdk]*pb.PrecompiledObject, error) { + client, err := storage.NewClient(ctx, option.WithoutAuthentication()) + if err != nil { + return nil, fmt.Errorf("storage.NewClient: %v", err) + } + defer client.Close() + bucket := client.Bucket(bucketName) + + paths := make(map[pb.Sdk]string, 0) + for _, sdkName := range pb.Sdk_name { + sdk := pb.Sdk(pb.Sdk_value[sdkName]) + if sdk == pb.Sdk_SDK_UNSPECIFIED { + continue + } + path, err := cd.getDefaultPrecompiledObjectsPath(ctx, bucket, sdk) + if err != nil { + return nil, err + } + paths[sdk] = path + } + + defaultPrecompiledObjects := make(map[pb.Sdk]*pb.PrecompiledObject, 0) + for sdk, path := range paths { + infoPath := filepath.Join(path, MetaInfoName) + rc, err := bucket.Object(infoPath).NewReader(ctx) + if err != nil { + logger.Errorf("Object(%q).NewReader: %v", infoPath, err.Error()) + continue + } + metaFile, err := ioutil.ReadAll(rc) + if err != nil { + logger.Errorf("ioutil.ReadAll: %v", err.Error()) + continue + } + rc.Close() + + precompiledObject := &pb.PrecompiledObject{} + err = json.Unmarshal(metaFile, &precompiledObject) + if err != nil { + logger.Errorf("json.Unmarshal: %v", err.Error()) + return nil, err + } + precompiledObject.CloudPath = path + defaultPrecompiledObjects[sdk] = precompiledObject + } + return defaultPrecompiledObjects, nil +} + +// getDefaultPrecompiledObjectsPath returns path for SDK to the default precompiled object +func (cd *CloudStorage) getDefaultPrecompiledObjectsPath(ctx context.Context, bucket *storage.BucketHandle, sdk pb.Sdk) (string, error) { + pathToFile := fmt.Sprintf("%s/%s", sdk.String(), defaultPrecompiledObjectInfo) + rc, err := bucket.Object(pathToFile).NewReader(ctx) + if err != nil { + logger.Errorf("Object(%q).NewReader: %v", pathToFile, err.Error()) + return "", err + } + + data, err := io.ReadAll(rc) + if err != nil { + logger.Errorf("ioutil.ReadAll: %v", err.Error()) + return "", err + } + + path := make(map[string]string, 0) + if err := json.Unmarshal(data, &path); err != nil { + return "", err + } + return path[sdk.String()], nil +} + // getPrecompiledObjectsDirs finds directories with precompiled objects // Since there is no notion of directory at cloud storage, then // to avoid duplicates of a base path (directory) need to store it in a set/map. diff --git a/playground/backend/internal/cloud_bucket/precompiled_objects_test.go b/playground/backend/internal/cloud_bucket/precompiled_objects_test.go index 30beb10ea2b5..52a8a85a134c 100644 --- a/playground/backend/internal/cloud_bucket/precompiled_objects_test.go +++ b/playground/backend/internal/cloud_bucket/precompiled_objects_test.go @@ -251,8 +251,8 @@ func Benchmark_GetPrecompiledObjectOutput(b *testing.B) { } } -func Benchmark_GetPrecompiledObject(b *testing.B) { +func Benchmark_GetPrecompiledObjectCode(b *testing.B) { for i := 0; i < b.N; i++ { - _, _ = bucket.GetPrecompiledObject(ctx, precompiledObjectPath, defaultBucketName) + _, _ = bucket.GetPrecompiledObjectCode(ctx, precompiledObjectPath, defaultBucketName) } } diff --git a/playground/backend/internal/utils/precompiled_objects_utils.go b/playground/backend/internal/utils/precompiled_objects_utils.go index 9f38c7d0026a..2b779ab5878d 100644 --- a/playground/backend/internal/utils/precompiled_objects_utils.go +++ b/playground/backend/internal/utils/precompiled_objects_utils.go @@ -17,9 +17,11 @@ package utils import ( pb "beam.apache.org/playground/backend/internal/api/v1" + "beam.apache.org/playground/backend/internal/cache" "beam.apache.org/playground/backend/internal/cloud_bucket" "beam.apache.org/playground/backend/internal/logger" "context" + "fmt" ) // PutPrecompiledObjectsToCategory adds categories with precompiled objects to protobuf object @@ -38,6 +40,7 @@ func PutPrecompiledObjectsToCategory(categoryName string, precompiledObjects *cl Link: object.Link, Multifile: object.Multifile, ContextLine: object.ContextLine, + DefaultExample: object.DefaultExample, }) } sdkCategory.Categories = append(sdkCategory.Categories, &category) @@ -93,3 +96,44 @@ func FilterCatalog(catalog []*pb.Categories, sdk pb.Sdk, categoryName string) [] } return result } + +// GetDefaultPrecompiledObject returns the default precompiled objects from cache for sdk +func GetDefaultPrecompiledObject(ctx context.Context, sdk pb.Sdk, cacheService cache.Cache, bucketName string) (*pb.PrecompiledObject, error) { + precompiledObject, err := cacheService.GetDefaultPrecompiledObject(ctx, sdk) + if err != nil { + logger.Errorf("GetDefaultPrecompiledObject(): error during getting default precompiled object %s", err.Error()) + bucket := cloud_bucket.New() + defaultPrecompiledObjects, err := bucket.GetDefaultPrecompiledObjects(ctx, bucketName) + if err != nil { + return nil, err + } + for sdk, precompiledObject := range defaultPrecompiledObjects { + if err := cacheService.SetDefaultPrecompiledObject(ctx, sdk, precompiledObject); err != nil { + logger.Errorf("GetPrecompiledObjects(): cache error: %s", err.Error()) + } + } + precompiledObject, ok := defaultPrecompiledObjects[sdk] + if !ok { + return nil, fmt.Errorf("no default precompiled object found for this sdk: %s", sdk) + } + return precompiledObject, nil + } + return precompiledObject, nil +} + +// GetCatalogFromCacheOrStorage returns the precompiled objects catalog from cache +// - If there is no catalog in the cache, gets the catalog from the Storage and saves it to the cache +func GetCatalogFromCacheOrStorage(ctx context.Context, cacheService cache.Cache, bucketName string) ([]*pb.Categories, error) { + catalog, err := cacheService.GetCatalog(ctx) + if err != nil { + logger.Errorf("GetCatalog(): cache error: %s", err.Error()) + catalog, err = GetCatalogFromStorage(ctx, bucketName) + if err != nil { + return nil, err + } + if err = cacheService.SetCatalog(ctx, catalog); err != nil { + logger.Errorf("SetCatalog(): cache error: %s", err.Error()) + } + } + return catalog, nil +} diff --git a/playground/backend/internal/utils/precompiled_objects_utils_test.go b/playground/backend/internal/utils/precompiled_objects_utils_test.go index 71fbe5af16ea..e1b345a66319 100644 --- a/playground/backend/internal/utils/precompiled_objects_utils_test.go +++ b/playground/backend/internal/utils/precompiled_objects_utils_test.go @@ -17,14 +17,18 @@ package utils import ( pb "beam.apache.org/playground/backend/internal/api/v1" + "beam.apache.org/playground/backend/internal/cache" + "beam.apache.org/playground/backend/internal/cache/local" "beam.apache.org/playground/backend/internal/cloud_bucket" + "beam.apache.org/playground/backend/internal/logger" + "context" "reflect" "testing" ) func TestPutPrecompiledObjectsToCategory(t *testing.T) { precompiledObjectToAdd := &cloud_bucket.PrecompiledObjects{ - {"TestName", "SDK_JAVA/TestCategory/TestName.java", "TestDescription", pb.PrecompiledObjectType_PRECOMPILED_OBJECT_TYPE_EXAMPLE, []string{""}, "", "", false, 1}, + {"TestName", "SDK_JAVA/TestCategory/TestName.java", "TestDescription", pb.PrecompiledObjectType_PRECOMPILED_OBJECT_TYPE_EXAMPLE, []string{""}, "", "", false, 1, false}, } type args struct { categoryName string @@ -205,3 +209,67 @@ func TestFilterPrecompiledObjects(t *testing.T) { }) } } + +func TestGetDefaultPrecompiledObject(t *testing.T) { + ctx := context.Background() + cacheService := local.New(ctx) + defaultPrecompiledObject := &pb.PrecompiledObject{ + CloudPath: "cloudPath", + Name: "Name", + Description: "Description", + Type: pb.PrecompiledObjectType_PRECOMPILED_OBJECT_TYPE_EXAMPLE, + PipelineOptions: "--key value", + Link: "Link", + ContextLine: 1, + DefaultExample: true, + } + err := cacheService.SetDefaultPrecompiledObject(ctx, pb.Sdk_SDK_JAVA, defaultPrecompiledObject) + if err != nil { + logger.Errorf("Error during set up test") + } + + type args struct { + ctx context.Context + sdk pb.Sdk + cacheService cache.Cache + } + tests := []struct { + name string + args args + want *pb.PrecompiledObject + wantErr bool + }{ + { + name: "there is default precompiled object", + args: args{ + ctx: ctx, + sdk: pb.Sdk_SDK_JAVA, + cacheService: cacheService, + }, + want: defaultPrecompiledObject, + wantErr: false, + }, + { + name: "there is no default precompiled object", + args: args{ + ctx: ctx, + sdk: pb.Sdk_SDK_UNSPECIFIED, + cacheService: cacheService, + }, + want: nil, + wantErr: true, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got, err := GetDefaultPrecompiledObject(tt.args.ctx, tt.args.sdk, tt.args.cacheService, "") + if (err != nil) != tt.wantErr { + t.Errorf("GetDefaultPrecompiledObject() error = %v, wantErr %v", err, tt.wantErr) + return + } + if !reflect.DeepEqual(got, tt.want) { + t.Errorf("GetDefaultPrecompiledObject() got = %v, want %v", got, tt.want) + } + }) + } +} diff --git a/playground/frontend/lib/api/v1/api.pb.dart b/playground/frontend/lib/api/v1/api.pb.dart index 3e388da97d12..de916d4552f3 100644 --- a/playground/frontend/lib/api/v1/api.pb.dart +++ b/playground/frontend/lib/api/v1/api.pb.dart @@ -991,6 +991,7 @@ class PrecompiledObject extends $pb.GeneratedMessage { ..aOS(6, const $core.bool.fromEnvironment('protobuf.omit_field_names') ? '' : 'link') ..aOB(7, const $core.bool.fromEnvironment('protobuf.omit_field_names') ? '' : 'multifile') ..a<$core.int>(8, const $core.bool.fromEnvironment('protobuf.omit_field_names') ? '' : 'contextLine', $pb.PbFieldType.O3) + ..aOB(9, const $core.bool.fromEnvironment('protobuf.omit_field_names') ? '' : 'defaultExample') ..hasRequiredFields = false ; @@ -1004,6 +1005,7 @@ class PrecompiledObject extends $pb.GeneratedMessage { $core.String? link, $core.bool? multifile, $core.int? contextLine, + $core.bool? defaultExample, }) { final _result = create(); if (cloudPath != null) { @@ -1030,6 +1032,9 @@ class PrecompiledObject extends $pb.GeneratedMessage { if (contextLine != null) { _result.contextLine = contextLine; } + if (defaultExample != null) { + _result.defaultExample = defaultExample; + } return _result; } factory PrecompiledObject.fromBuffer($core.List<$core.int> i, [$pb.ExtensionRegistry r = $pb.ExtensionRegistry.EMPTY]) => create()..mergeFromBuffer(i, r); @@ -1124,6 +1129,15 @@ class PrecompiledObject extends $pb.GeneratedMessage { $core.bool hasContextLine() => $_has(7); @$pb.TagNumber(8) void clearContextLine() => clearField(8); + + @$pb.TagNumber(9) + $core.bool get defaultExample => $_getBF(8); + @$pb.TagNumber(9) + set defaultExample($core.bool v) { $_setBool(8, v); } + @$pb.TagNumber(9) + $core.bool hasDefaultExample() => $_has(8); + @$pb.TagNumber(9) + void clearDefaultExample() => clearField(9); } class Categories_Category extends $pb.GeneratedMessage { diff --git a/playground/frontend/lib/api/v1/api.pbjson.dart b/playground/frontend/lib/api/v1/api.pbjson.dart index 73986428da05..a952e8404987 100644 --- a/playground/frontend/lib/api/v1/api.pbjson.dart +++ b/playground/frontend/lib/api/v1/api.pbjson.dart @@ -286,11 +286,12 @@ const PrecompiledObject$json = const { const {'1': 'link', '3': 6, '4': 1, '5': 9, '10': 'link'}, const {'1': 'multifile', '3': 7, '4': 1, '5': 8, '10': 'multifile'}, const {'1': 'context_line', '3': 8, '4': 1, '5': 5, '10': 'contextLine'}, + const {'1': 'default_example', '3': 9, '4': 1, '5': 8, '10': 'defaultExample'}, ], }; /// Descriptor for `PrecompiledObject`. Decode as a `google.protobuf.DescriptorProto`. -final $typed_data.Uint8List precompiledObjectDescriptor = $convert.base64Decode('ChFQcmVjb21waWxlZE9iamVjdBIdCgpjbG91ZF9wYXRoGAEgASgJUgljbG91ZFBhdGgSEgoEbmFtZRgCIAEoCVIEbmFtZRIgCgtkZXNjcmlwdGlvbhgDIAEoCVILZGVzY3JpcHRpb24SMQoEdHlwZRgEIAEoDjIdLmFwaS52MS5QcmVjb21waWxlZE9iamVjdFR5cGVSBHR5cGUSKQoQcGlwZWxpbmVfb3B0aW9ucxgFIAEoCVIPcGlwZWxpbmVPcHRpb25zEhIKBGxpbmsYBiABKAlSBGxpbmsSHAoJbXVsdGlmaWxlGAcgASgIUgltdWx0aWZpbGUSIQoMY29udGV4dF9saW5lGAggASgFUgtjb250ZXh0TGluZQ=='); +final $typed_data.Uint8List precompiledObjectDescriptor = $convert.base64Decode('ChFQcmVjb21waWxlZE9iamVjdBIdCgpjbG91ZF9wYXRoGAEgASgJUgljbG91ZFBhdGgSEgoEbmFtZRgCIAEoCVIEbmFtZRIgCgtkZXNjcmlwdGlvbhgDIAEoCVILZGVzY3JpcHRpb24SMQoEdHlwZRgEIAEoDjIdLmFwaS52MS5QcmVjb21waWxlZE9iamVjdFR5cGVSBHR5cGUSKQoQcGlwZWxpbmVfb3B0aW9ucxgFIAEoCVIPcGlwZWxpbmVPcHRpb25zEhIKBGxpbmsYBiABKAlSBGxpbmsSHAoJbXVsdGlmaWxlGAcgASgIUgltdWx0aWZpbGUSIQoMY29udGV4dF9saW5lGAggASgFUgtjb250ZXh0TGluZRInCg9kZWZhdWx0X2V4YW1wbGUYCSABKAhSDmRlZmF1bHRFeGFtcGxl'); @$core.Deprecated('Use categoriesDescriptor instead') const Categories$json = const { '1': 'Categories', diff --git a/playground/infrastructure/api/v1/api_pb2.py b/playground/infrastructure/api/v1/api_pb2.py index 77a0f2b15587..3108cd746977 100644 --- a/playground/infrastructure/api/v1/api_pb2.py +++ b/playground/infrastructure/api/v1/api_pb2.py @@ -35,7 +35,7 @@ syntax='proto3', serialized_options=b'Z6beam.apache.org/playground/backend/internal;playground', create_key=_descriptor._internal_create_key, - serialized_pb=b'\n\tapi.proto\x12\x06\x61pi.v1\"R\n\x0eRunCodeRequest\x12\x0c\n\x04\x63ode\x18\x01 \x01(\t\x12\x18\n\x03sdk\x18\x02 \x01(\x0e\x32\x0b.api.v1.Sdk\x12\x18\n\x10pipeline_options\x18\x03 \x01(\t\"(\n\x0fRunCodeResponse\x12\x15\n\rpipeline_uuid\x18\x01 \x01(\t\"+\n\x12\x43heckStatusRequest\x12\x15\n\rpipeline_uuid\x18\x01 \x01(\t\"5\n\x13\x43heckStatusResponse\x12\x1e\n\x06status\x18\x01 \x01(\x0e\x32\x0e.api.v1.Status\"3\n\x1aGetValidationOutputRequest\x12\x15\n\rpipeline_uuid\x18\x01 \x01(\t\"-\n\x1bGetValidationOutputResponse\x12\x0e\n\x06output\x18\x01 \x01(\t\"4\n\x1bGetPreparationOutputRequest\x12\x15\n\rpipeline_uuid\x18\x01 \x01(\t\".\n\x1cGetPreparationOutputResponse\x12\x0e\n\x06output\x18\x01 \x01(\t\"0\n\x17GetCompileOutputRequest\x12\x15\n\rpipeline_uuid\x18\x01 \x01(\t\"*\n\x18GetCompileOutputResponse\x12\x0e\n\x06output\x18\x01 \x01(\t\",\n\x13GetRunOutputRequest\x12\x15\n\rpipeline_uuid\x18\x01 \x01(\t\"&\n\x14GetRunOutputResponse\x12\x0e\n\x06output\x18\x01 \x01(\t\"+\n\x12GetRunErrorRequest\x12\x15\n\rpipeline_uuid\x18\x01 \x01(\t\"%\n\x13GetRunErrorResponse\x12\x0e\n\x06output\x18\x01 \x01(\t\"\'\n\x0eGetLogsRequest\x12\x15\n\rpipeline_uuid\x18\x01 \x01(\t\"!\n\x0fGetLogsResponse\x12\x0e\n\x06output\x18\x01 \x01(\t\"(\n\x0fGetGraphRequest\x12\x15\n\rpipeline_uuid\x18\x01 \x01(\t\"!\n\x10GetGraphResponse\x12\r\n\x05graph\x18\x01 \x01(\t\"&\n\rCancelRequest\x12\x15\n\rpipeline_uuid\x18\x01 \x01(\t\"\x10\n\x0e\x43\x61ncelResponse\"\xc8\x01\n\x11PrecompiledObject\x12\x12\n\ncloud_path\x18\x01 \x01(\t\x12\x0c\n\x04name\x18\x02 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x03 \x01(\t\x12+\n\x04type\x18\x04 \x01(\x0e\x32\x1d.api.v1.PrecompiledObjectType\x12\x18\n\x10pipeline_options\x18\x05 \x01(\t\x12\x0c\n\x04link\x18\x06 \x01(\t\x12\x11\n\tmultifile\x18\x07 \x01(\x08\x12\x14\n\x0c\x63ontext_line\x18\x08 \x01(\x05\"\xb2\x01\n\nCategories\x12\x18\n\x03sdk\x18\x01 \x01(\x0e\x32\x0b.api.v1.Sdk\x12/\n\ncategories\x18\x02 \x03(\x0b\x32\x1b.api.v1.Categories.Category\x1aY\n\x08\x43\x61tegory\x12\x15\n\rcategory_name\x18\x01 \x01(\t\x12\x36\n\x13precompiled_objects\x18\x02 \x03(\x0b\x32\x19.api.v1.PrecompiledObject\"J\n\x1cGetPrecompiledObjectsRequest\x12\x18\n\x03sdk\x18\x01 \x01(\x0e\x32\x0b.api.v1.Sdk\x12\x10\n\x08\x63\x61tegory\x18\x02 \x01(\t\"5\n\x1fGetPrecompiledObjectCodeRequest\x12\x12\n\ncloud_path\x18\x01 \x01(\t\"7\n!GetPrecompiledObjectOutputRequest\x12\x12\n\ncloud_path\x18\x01 \x01(\t\"5\n\x1fGetPrecompiledObjectLogsRequest\x12\x12\n\ncloud_path\x18\x01 \x01(\t\"6\n GetPrecompiledObjectGraphRequest\x12\x12\n\ncloud_path\x18\x01 \x01(\t\">\n\"GetDefaultPrecompiledObjectRequest\x12\x18\n\x03sdk\x18\x01 \x01(\x0e\x32\x0b.api.v1.Sdk\"K\n\x1dGetPrecompiledObjectsResponse\x12*\n\x0esdk_categories\x18\x01 \x03(\x0b\x32\x12.api.v1.Categories\"0\n GetPrecompiledObjectCodeResponse\x12\x0c\n\x04\x63ode\x18\x01 \x01(\t\"4\n\"GetPrecompiledObjectOutputResponse\x12\x0e\n\x06output\x18\x01 \x01(\t\"2\n GetPrecompiledObjectLogsResponse\x12\x0e\n\x06output\x18\x01 \x01(\t\"2\n!GetPrecompiledObjectGraphResponse\x12\r\n\x05graph\x18\x01 \x01(\t\"\\\n#GetDefaultPrecompiledObjectResponse\x12\x35\n\x12precompiled_object\x18\x01 \x01(\x0b\x32\x19.api.v1.PrecompiledObject*R\n\x03Sdk\x12\x13\n\x0fSDK_UNSPECIFIED\x10\x00\x12\x0c\n\x08SDK_JAVA\x10\x01\x12\n\n\x06SDK_GO\x10\x02\x12\x0e\n\nSDK_PYTHON\x10\x03\x12\x0c\n\x08SDK_SCIO\x10\x04*\xb8\x02\n\x06Status\x12\x16\n\x12STATUS_UNSPECIFIED\x10\x00\x12\x15\n\x11STATUS_VALIDATING\x10\x01\x12\x1b\n\x17STATUS_VALIDATION_ERROR\x10\x02\x12\x14\n\x10STATUS_PREPARING\x10\x03\x12\x1c\n\x18STATUS_PREPARATION_ERROR\x10\x04\x12\x14\n\x10STATUS_COMPILING\x10\x05\x12\x18\n\x14STATUS_COMPILE_ERROR\x10\x06\x12\x14\n\x10STATUS_EXECUTING\x10\x07\x12\x13\n\x0fSTATUS_FINISHED\x10\x08\x12\x14\n\x10STATUS_RUN_ERROR\x10\t\x12\x10\n\x0cSTATUS_ERROR\x10\n\x12\x16\n\x12STATUS_RUN_TIMEOUT\x10\x0b\x12\x13\n\x0fSTATUS_CANCELED\x10\x0c*\xae\x01\n\x15PrecompiledObjectType\x12\'\n#PRECOMPILED_OBJECT_TYPE_UNSPECIFIED\x10\x00\x12#\n\x1fPRECOMPILED_OBJECT_TYPE_EXAMPLE\x10\x01\x12 \n\x1cPRECOMPILED_OBJECT_TYPE_KATA\x10\x02\x12%\n!PRECOMPILED_OBJECT_TYPE_UNIT_TEST\x10\x03\x32\x9b\x0b\n\x11PlaygroundService\x12:\n\x07RunCode\x12\x16.api.v1.RunCodeRequest\x1a\x17.api.v1.RunCodeResponse\x12\x46\n\x0b\x43heckStatus\x12\x1a.api.v1.CheckStatusRequest\x1a\x1b.api.v1.CheckStatusResponse\x12I\n\x0cGetRunOutput\x12\x1b.api.v1.GetRunOutputRequest\x1a\x1c.api.v1.GetRunOutputResponse\x12:\n\x07GetLogs\x12\x16.api.v1.GetLogsRequest\x1a\x17.api.v1.GetLogsResponse\x12=\n\x08GetGraph\x12\x17.api.v1.GetGraphRequest\x1a\x18.api.v1.GetGraphResponse\x12\x46\n\x0bGetRunError\x12\x1a.api.v1.GetRunErrorRequest\x1a\x1b.api.v1.GetRunErrorResponse\x12^\n\x13GetValidationOutput\x12\".api.v1.GetValidationOutputRequest\x1a#.api.v1.GetValidationOutputResponse\x12\x61\n\x14GetPreparationOutput\x12#.api.v1.GetPreparationOutputRequest\x1a$.api.v1.GetPreparationOutputResponse\x12U\n\x10GetCompileOutput\x12\x1f.api.v1.GetCompileOutputRequest\x1a .api.v1.GetCompileOutputResponse\x12\x37\n\x06\x43\x61ncel\x12\x15.api.v1.CancelRequest\x1a\x16.api.v1.CancelResponse\x12\x64\n\x15GetPrecompiledObjects\x12$.api.v1.GetPrecompiledObjectsRequest\x1a%.api.v1.GetPrecompiledObjectsResponse\x12m\n\x18GetPrecompiledObjectCode\x12\'.api.v1.GetPrecompiledObjectCodeRequest\x1a(.api.v1.GetPrecompiledObjectCodeResponse\x12s\n\x1aGetPrecompiledObjectOutput\x12).api.v1.GetPrecompiledObjectOutputRequest\x1a*.api.v1.GetPrecompiledObjectOutputResponse\x12m\n\x18GetPrecompiledObjectLogs\x12\'.api.v1.GetPrecompiledObjectLogsRequest\x1a(.api.v1.GetPrecompiledObjectLogsResponse\x12p\n\x19GetPrecompiledObjectGraph\x12(.api.v1.GetPrecompiledObjectGraphRequest\x1a).api.v1.GetPrecompiledObjectGraphResponse\x12v\n\x1bGetDefaultPrecompiledObject\x12*.api.v1.GetDefaultPrecompiledObjectRequest\x1a+.api.v1.GetDefaultPrecompiledObjectResponseB8Z6beam.apache.org/playground/backend/internal;playgroundb\x06proto3' + serialized_pb=b'\n\tapi.proto\x12\x06\x61pi.v1\"R\n\x0eRunCodeRequest\x12\x0c\n\x04\x63ode\x18\x01 \x01(\t\x12\x18\n\x03sdk\x18\x02 \x01(\x0e\x32\x0b.api.v1.Sdk\x12\x18\n\x10pipeline_options\x18\x03 \x01(\t\"(\n\x0fRunCodeResponse\x12\x15\n\rpipeline_uuid\x18\x01 \x01(\t\"+\n\x12\x43heckStatusRequest\x12\x15\n\rpipeline_uuid\x18\x01 \x01(\t\"5\n\x13\x43heckStatusResponse\x12\x1e\n\x06status\x18\x01 \x01(\x0e\x32\x0e.api.v1.Status\"3\n\x1aGetValidationOutputRequest\x12\x15\n\rpipeline_uuid\x18\x01 \x01(\t\"-\n\x1bGetValidationOutputResponse\x12\x0e\n\x06output\x18\x01 \x01(\t\"4\n\x1bGetPreparationOutputRequest\x12\x15\n\rpipeline_uuid\x18\x01 \x01(\t\".\n\x1cGetPreparationOutputResponse\x12\x0e\n\x06output\x18\x01 \x01(\t\"0\n\x17GetCompileOutputRequest\x12\x15\n\rpipeline_uuid\x18\x01 \x01(\t\"*\n\x18GetCompileOutputResponse\x12\x0e\n\x06output\x18\x01 \x01(\t\",\n\x13GetRunOutputRequest\x12\x15\n\rpipeline_uuid\x18\x01 \x01(\t\"&\n\x14GetRunOutputResponse\x12\x0e\n\x06output\x18\x01 \x01(\t\"+\n\x12GetRunErrorRequest\x12\x15\n\rpipeline_uuid\x18\x01 \x01(\t\"%\n\x13GetRunErrorResponse\x12\x0e\n\x06output\x18\x01 \x01(\t\"\'\n\x0eGetLogsRequest\x12\x15\n\rpipeline_uuid\x18\x01 \x01(\t\"!\n\x0fGetLogsResponse\x12\x0e\n\x06output\x18\x01 \x01(\t\"(\n\x0fGetGraphRequest\x12\x15\n\rpipeline_uuid\x18\x01 \x01(\t\"!\n\x10GetGraphResponse\x12\r\n\x05graph\x18\x01 \x01(\t\"&\n\rCancelRequest\x12\x15\n\rpipeline_uuid\x18\x01 \x01(\t\"\x10\n\x0e\x43\x61ncelResponse\"\xe1\x01\n\x11PrecompiledObject\x12\x12\n\ncloud_path\x18\x01 \x01(\t\x12\x0c\n\x04name\x18\x02 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x03 \x01(\t\x12+\n\x04type\x18\x04 \x01(\x0e\x32\x1d.api.v1.PrecompiledObjectType\x12\x18\n\x10pipeline_options\x18\x05 \x01(\t\x12\x0c\n\x04link\x18\x06 \x01(\t\x12\x11\n\tmultifile\x18\x07 \x01(\x08\x12\x14\n\x0c\x63ontext_line\x18\x08 \x01(\x05\x12\x17\n\x0f\x64\x65\x66\x61ult_example\x18\t \x01(\x08\"\xb2\x01\n\nCategories\x12\x18\n\x03sdk\x18\x01 \x01(\x0e\x32\x0b.api.v1.Sdk\x12/\n\ncategories\x18\x02 \x03(\x0b\x32\x1b.api.v1.Categories.Category\x1aY\n\x08\x43\x61tegory\x12\x15\n\rcategory_name\x18\x01 \x01(\t\x12\x36\n\x13precompiled_objects\x18\x02 \x03(\x0b\x32\x19.api.v1.PrecompiledObject\"J\n\x1cGetPrecompiledObjectsRequest\x12\x18\n\x03sdk\x18\x01 \x01(\x0e\x32\x0b.api.v1.Sdk\x12\x10\n\x08\x63\x61tegory\x18\x02 \x01(\t\"5\n\x1fGetPrecompiledObjectCodeRequest\x12\x12\n\ncloud_path\x18\x01 \x01(\t\"7\n!GetPrecompiledObjectOutputRequest\x12\x12\n\ncloud_path\x18\x01 \x01(\t\"5\n\x1fGetPrecompiledObjectLogsRequest\x12\x12\n\ncloud_path\x18\x01 \x01(\t\"6\n GetPrecompiledObjectGraphRequest\x12\x12\n\ncloud_path\x18\x01 \x01(\t\">\n\"GetDefaultPrecompiledObjectRequest\x12\x18\n\x03sdk\x18\x01 \x01(\x0e\x32\x0b.api.v1.Sdk\"K\n\x1dGetPrecompiledObjectsResponse\x12*\n\x0esdk_categories\x18\x01 \x03(\x0b\x32\x12.api.v1.Categories\"0\n GetPrecompiledObjectCodeResponse\x12\x0c\n\x04\x63ode\x18\x01 \x01(\t\"4\n\"GetPrecompiledObjectOutputResponse\x12\x0e\n\x06output\x18\x01 \x01(\t\"2\n GetPrecompiledObjectLogsResponse\x12\x0e\n\x06output\x18\x01 \x01(\t\"2\n!GetPrecompiledObjectGraphResponse\x12\r\n\x05graph\x18\x01 \x01(\t\"\\\n#GetDefaultPrecompiledObjectResponse\x12\x35\n\x12precompiled_object\x18\x01 \x01(\x0b\x32\x19.api.v1.PrecompiledObject*R\n\x03Sdk\x12\x13\n\x0fSDK_UNSPECIFIED\x10\x00\x12\x0c\n\x08SDK_JAVA\x10\x01\x12\n\n\x06SDK_GO\x10\x02\x12\x0e\n\nSDK_PYTHON\x10\x03\x12\x0c\n\x08SDK_SCIO\x10\x04*\xb8\x02\n\x06Status\x12\x16\n\x12STATUS_UNSPECIFIED\x10\x00\x12\x15\n\x11STATUS_VALIDATING\x10\x01\x12\x1b\n\x17STATUS_VALIDATION_ERROR\x10\x02\x12\x14\n\x10STATUS_PREPARING\x10\x03\x12\x1c\n\x18STATUS_PREPARATION_ERROR\x10\x04\x12\x14\n\x10STATUS_COMPILING\x10\x05\x12\x18\n\x14STATUS_COMPILE_ERROR\x10\x06\x12\x14\n\x10STATUS_EXECUTING\x10\x07\x12\x13\n\x0fSTATUS_FINISHED\x10\x08\x12\x14\n\x10STATUS_RUN_ERROR\x10\t\x12\x10\n\x0cSTATUS_ERROR\x10\n\x12\x16\n\x12STATUS_RUN_TIMEOUT\x10\x0b\x12\x13\n\x0fSTATUS_CANCELED\x10\x0c*\xae\x01\n\x15PrecompiledObjectType\x12\'\n#PRECOMPILED_OBJECT_TYPE_UNSPECIFIED\x10\x00\x12#\n\x1fPRECOMPILED_OBJECT_TYPE_EXAMPLE\x10\x01\x12 \n\x1cPRECOMPILED_OBJECT_TYPE_KATA\x10\x02\x12%\n!PRECOMPILED_OBJECT_TYPE_UNIT_TEST\x10\x03\x32\x9b\x0b\n\x11PlaygroundService\x12:\n\x07RunCode\x12\x16.api.v1.RunCodeRequest\x1a\x17.api.v1.RunCodeResponse\x12\x46\n\x0b\x43heckStatus\x12\x1a.api.v1.CheckStatusRequest\x1a\x1b.api.v1.CheckStatusResponse\x12I\n\x0cGetRunOutput\x12\x1b.api.v1.GetRunOutputRequest\x1a\x1c.api.v1.GetRunOutputResponse\x12:\n\x07GetLogs\x12\x16.api.v1.GetLogsRequest\x1a\x17.api.v1.GetLogsResponse\x12=\n\x08GetGraph\x12\x17.api.v1.GetGraphRequest\x1a\x18.api.v1.GetGraphResponse\x12\x46\n\x0bGetRunError\x12\x1a.api.v1.GetRunErrorRequest\x1a\x1b.api.v1.GetRunErrorResponse\x12^\n\x13GetValidationOutput\x12\".api.v1.GetValidationOutputRequest\x1a#.api.v1.GetValidationOutputResponse\x12\x61\n\x14GetPreparationOutput\x12#.api.v1.GetPreparationOutputRequest\x1a$.api.v1.GetPreparationOutputResponse\x12U\n\x10GetCompileOutput\x12\x1f.api.v1.GetCompileOutputRequest\x1a .api.v1.GetCompileOutputResponse\x12\x37\n\x06\x43\x61ncel\x12\x15.api.v1.CancelRequest\x1a\x16.api.v1.CancelResponse\x12\x64\n\x15GetPrecompiledObjects\x12$.api.v1.GetPrecompiledObjectsRequest\x1a%.api.v1.GetPrecompiledObjectsResponse\x12m\n\x18GetPrecompiledObjectCode\x12\'.api.v1.GetPrecompiledObjectCodeRequest\x1a(.api.v1.GetPrecompiledObjectCodeResponse\x12s\n\x1aGetPrecompiledObjectOutput\x12).api.v1.GetPrecompiledObjectOutputRequest\x1a*.api.v1.GetPrecompiledObjectOutputResponse\x12m\n\x18GetPrecompiledObjectLogs\x12\'.api.v1.GetPrecompiledObjectLogsRequest\x1a(.api.v1.GetPrecompiledObjectLogsResponse\x12p\n\x19GetPrecompiledObjectGraph\x12(.api.v1.GetPrecompiledObjectGraphRequest\x1a).api.v1.GetPrecompiledObjectGraphResponse\x12v\n\x1bGetDefaultPrecompiledObject\x12*.api.v1.GetDefaultPrecompiledObjectRequest\x1a+.api.v1.GetDefaultPrecompiledObjectResponseB8Z6beam.apache.org/playground/backend/internal;playgroundb\x06proto3' ) _SDK = _descriptor.EnumDescriptor( @@ -73,8 +73,8 @@ ], containing_type=None, serialized_options=None, - serialized_start=2050, - serialized_end=2132, + serialized_start=2075, + serialized_end=2157, ) _sym_db.RegisterEnumDescriptor(_SDK) @@ -154,8 +154,8 @@ ], containing_type=None, serialized_options=None, - serialized_start=2135, - serialized_end=2447, + serialized_start=2160, + serialized_end=2472, ) _sym_db.RegisterEnumDescriptor(_STATUS) @@ -190,8 +190,8 @@ ], containing_type=None, serialized_options=None, - serialized_start=2450, - serialized_end=2624, + serialized_start=2475, + serialized_end=2649, ) _sym_db.RegisterEnumDescriptor(_PRECOMPILEDOBJECTTYPE) @@ -932,6 +932,13 @@ message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key), + _descriptor.FieldDescriptor( + name='default_example', full_name='api.v1.PrecompiledObject.default_example', index=8, + number=9, type=8, cpp_type=7, label=1, + has_default_value=False, default_value=False, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key), ], extensions=[ ], @@ -945,7 +952,7 @@ oneofs=[ ], serialized_start=925, - serialized_end=1125, + serialized_end=1150, ) @@ -983,8 +990,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1217, - serialized_end=1306, + serialized_start=1242, + serialized_end=1331, ) _CATEGORIES = _descriptor.Descriptor( @@ -1021,8 +1028,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1128, - serialized_end=1306, + serialized_start=1153, + serialized_end=1331, ) @@ -1060,8 +1067,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1308, - serialized_end=1382, + serialized_start=1333, + serialized_end=1407, ) @@ -1092,8 +1099,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1384, - serialized_end=1437, + serialized_start=1409, + serialized_end=1462, ) @@ -1124,8 +1131,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1439, - serialized_end=1494, + serialized_start=1464, + serialized_end=1519, ) @@ -1156,8 +1163,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1496, - serialized_end=1549, + serialized_start=1521, + serialized_end=1574, ) @@ -1188,8 +1195,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1551, - serialized_end=1605, + serialized_start=1576, + serialized_end=1630, ) @@ -1220,8 +1227,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1607, - serialized_end=1669, + serialized_start=1632, + serialized_end=1694, ) @@ -1252,8 +1259,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1671, - serialized_end=1746, + serialized_start=1696, + serialized_end=1771, ) @@ -1284,8 +1291,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1748, - serialized_end=1796, + serialized_start=1773, + serialized_end=1821, ) @@ -1316,8 +1323,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1798, - serialized_end=1850, + serialized_start=1823, + serialized_end=1875, ) @@ -1348,8 +1355,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1852, - serialized_end=1902, + serialized_start=1877, + serialized_end=1927, ) @@ -1380,8 +1387,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1904, - serialized_end=1954, + serialized_start=1929, + serialized_end=1979, ) @@ -1412,8 +1419,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1956, - serialized_end=2048, + serialized_start=1981, + serialized_end=2073, ) _RUNCODEREQUEST.fields_by_name['sdk'].enum_type = _SDK @@ -1722,8 +1729,8 @@ index=0, serialized_options=None, create_key=_descriptor._internal_create_key, - serialized_start=2627, - serialized_end=4062, + serialized_start=2652, + serialized_end=4087, methods=[ _descriptor.MethodDescriptor( name='RunCode', diff --git a/playground/infrastructure/cd_helper.py b/playground/infrastructure/cd_helper.py index 5797b1f8af1a..4b19d5514854 100644 --- a/playground/infrastructure/cd_helper.py +++ b/playground/infrastructure/cd_helper.py @@ -103,12 +103,47 @@ def _save_to_cloud_storage(self, examples: List[Example]): """ self._storage_client = storage.Client() self._bucket = self._storage_client.bucket(Config.BUCKET_NAME) + for example in tqdm(examples): file_names = self._write_to_local_fs(example) + + if example.tag.default_example: + default_example_path = str(Path([*file_names].pop()).parent) + cloud_path = self._write_default_example_path_to_local_fs( + default_example_path) + + self._upload_blob( + source_file=os.path.join(Config.TEMP_FOLDER, cloud_path), + destination_blob_name=cloud_path) + for cloud_file_name, local_file_name in file_names.items(): self._upload_blob( source_file=local_file_name, destination_blob_name=cloud_file_name) + def _write_default_example_path_to_local_fs(self, path: str) -> str: + """ + Write default example path to the file (in temp folder) + + Args: + path: path of the default example + + Returns: name of the file + + """ + sdk = Path(path).parts[0] + cloud_path = os.path.join(sdk, Config.DEFAULT_PRECOMPILED_OBJECT) + + path_to_file = os.path.join(Config.TEMP_FOLDER, sdk) + Path(path_to_file).mkdir(parents=True, exist_ok=True) + + local_path = os.path.join(path_to_file, Config.DEFAULT_PRECOMPILED_OBJECT) + + content = json.dumps({sdk: path}) + with open(local_path, "w", encoding="utf-8") as file: + file.write(content) + + return cloud_path + def _write_to_local_fs(self, example: Example): """ Write code of an example, output and meta info diff --git a/playground/infrastructure/config.py b/playground/infrastructure/config.py index 7cc816ccb348..e072297486de 100644 --- a/playground/infrastructure/config.py +++ b/playground/infrastructure/config.py @@ -43,6 +43,7 @@ class Config: Sdk.Name(SDK_SCIO)) BUCKET_NAME = "playground-precompiled-objects" TEMP_FOLDER = "temp" + DEFAULT_PRECOMPILED_OBJECT = "defaultPrecompiledObject.info" SDK_TO_EXTENSION = { SDK_JAVA: "java", SDK_GO: "go", SDK_PYTHON: "py", SDK_SCIO: "scala" } diff --git a/playground/infrastructure/proxy/allow_list.py b/playground/infrastructure/proxy/allow_list.py index 002dc41b1468..d5261a2e60cf 100644 --- a/playground/infrastructure/proxy/allow_list.py +++ b/playground/infrastructure/proxy/allow_list.py @@ -27,4 +27,5 @@ "dataflow-samples", "beam-samples", "apache-beam-samples", + "playground-precompiled-objects", ] diff --git a/playground/infrastructure/test_cd_helper.py b/playground/infrastructure/test_cd_helper.py index 681851b544e8..3628540d0117 100644 --- a/playground/infrastructure/test_cd_helper.py +++ b/playground/infrastructure/test_cd_helper.py @@ -14,11 +14,12 @@ # limitations under the License. import os +import pathlib import shutil import pytest -from api.v1.api_pb2 import SDK_JAVA, STATUS_UNSPECIFIED, \ +from api.v1.api_pb2 import Sdk, SDK_JAVA, SDK_GO, STATUS_UNSPECIFIED, \ PRECOMPILED_OBJECT_TYPE_UNIT_TEST from cd_helper import CDHelper from config import Config @@ -127,3 +128,20 @@ def test__save_to_cloud_storage(mocker): CDHelper()._save_to_cloud_storage([example]) write_to_os_mock.assert_called_with(example) upload_blob_mock.assert_called_with(source_file="", destination_blob_name="") + + +def test__write_default_example_path_to_local_fs(delete_temp_folder): + """ + Test writing default example link of sdk to + the filesystem (in temp folder) + Args: + delete_temp_folder: python fixture to clean up temp folder + after method execution + """ + sdk = Sdk.Name(SDK_GO) + default_example_path = "SDK_GO/PRECOMPILED_OBJECT_TYPE_EXAMPLE/WordCount" + expected_result = str(pathlib.Path(sdk, Config.DEFAULT_PRECOMPILED_OBJECT)) + cloud_path = CDHelper()._write_default_example_path_to_local_fs( + default_example_path) + assert cloud_path == expected_result + assert os.path.exists(os.path.join("temp", cloud_path)) diff --git a/playground/infrastructure/test_helper.py b/playground/infrastructure/test_helper.py index 2ae71ef7b138..6c733cc061bf 100644 --- a/playground/infrastructure/test_helper.py +++ b/playground/infrastructure/test_helper.py @@ -177,7 +177,8 @@ def test__get_example(mock_get_name): "description": "Description", "multifile": "False", "categories": [""], - "pipeline_options": "--option option" + "pipeline_options": "--option option", + "context_line": 1 }, "") @@ -189,7 +190,8 @@ def test__get_example(mock_get_name): filepath="/root/filepath.java", code="data", status=STATUS_UNSPECIFIED, - tag=Tag("Name", "Description", "False", [""], "--option option"), + tag=Tag( + "Name", "Description", "False", [""], "--option option", False, 1), link="https://github.com/apache/beam/blob/master/root/filepath.java") mock_get_name.assert_called_once_with("filepath.java") @@ -245,7 +247,8 @@ def test__validate_with_all_fields(): "description": "Description", "multifile": "true", "categories": ["category"], - "pipeline_options": "--option option" + "pipeline_options": "--option option", + "context_line": 1 } assert _validate(tag, ["category"]) is True From f8ca92aa3536ec1098e722b6d467743e13d25a1c Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Tue, 1 Mar 2022 11:27:44 -0800 Subject: [PATCH 08/68] Add 2022 events blog post (#16975) --- .../blog/upcoming-events-for-beam-in-2022.md | 82 +++++++++++++++++++ website/www/site/data/authors.yml | 5 +- 2 files changed, 86 insertions(+), 1 deletion(-) create mode 100644 website/www/site/content/en/blog/upcoming-events-for-beam-in-2022.md diff --git a/website/www/site/content/en/blog/upcoming-events-for-beam-in-2022.md b/website/www/site/content/en/blog/upcoming-events-for-beam-in-2022.md new file mode 100644 index 000000000000..3d3f35b4e535 --- /dev/null +++ b/website/www/site/content/en/blog/upcoming-events-for-beam-in-2022.md @@ -0,0 +1,82 @@ +--- +title: "Upcoming Events for Beam in 2022" +date: 2022-02-28 00:00:01 -0800 +categories: + - blog +authors: + - hermannb +--- + + +We are so excited to announce the upcoming Beam events for this year! We believe that events are an important mechanism to foster the community around Apache Beam as an Open Source Project. Our events are focused on a developer experience by giving spaces for the community to connect, facilitate collaboration, and enable knowledge sharing. + + + +Here is an overview of some upcoming events and ways for everyone to help foster additional community growth: + +## Beam Summit + +The **[Beam Summit 2022](https://2022.beamsummit.org/)** is approaching! The event will be in a hybrid in-person and virtual format from Austin, TX on July 18-20, 2022. The conference will include three full days of lightning talks, roadmap updates, use cases, demos, and workshops for Beam users of all levels. This is a great opportunity to collaborate, share ideas, and work together in the improvement of the project. + +Check out talks from prior editions of Beam Summit **[here](https://www.youtube.com/watch?v=jses0W4Zalc&list=PL4dEBWmGSIU8vLWF56shrSuTsLXvO6Ex3)**! + +### The Experience + +We are so excited to see some of you in person again and the rest of the community online! The **[Beam Summit Steering Committee](https://2022.beamsummit.org/team/)** in partnership with an event production company is working hard to ensure that we provide the community with the best possible experience, no matter which format you choose to attend in. + +If you have any ideas on how we can make this year’s event better, please **[reach out to us](mailto:contact@beamsummit.org)**! + +### Ways to Help & Participate + +1. **[Submit a proposal](https://sessionize.com/beam-summit-2022)** to talk! The deadline for submissions is _March 15th_. +2. **[Register](https://2022.beamsummit.org/tickets/)** to join as an attendee in person or online. +3. Consider sponsoring the event. If your company is interested in engaging with members of the community, please check out the **[sponsoring prospectus](https://2022.beamsummit.org/sponsors/).** +4. Help us get the word out. Please make sure to let your colleagues and friends know about the Beam Summit. + +Don’t forget to follow our Beam Summit **[Twitter](https://twitter.com/BeamSummit?ref_src=twsrc%5Egoogle%7Ctwcamp%5Eserp%7Ctwgr%5Eauthor)** and **[LinkedIn](https://www.linkedin.com/company/beam-summit/?viewAsMember=true)** pages to receive event updates! + +## Beam College + +**[Beam College 2022](https://beamcollege.dev/)** is around the corner for the second season of training! The event will be hosted virtually from May 10-13, 2022. The training is focused on providing more hands-on experience around end-to-end code samples in an interactive environment, and helping attendees see the applications of concepts covered in other venues, such as the Beam Summit. + +Check out talks from prior editions of Beam College **[here](https://www.youtube.com/playlist?list=PLjYq1UNvv2UcrfapfgKrnLXtYpkvHmpIh)**! + +This year, the training will consist of learning modules such as: + +* The Data movement ecosystem and distributed processing the Beam way +* Scaling, productionalizing, and developing your Beam pipelines +* Use Cases +* Beam ML Use Cases + +Be sure to check out our **[website](https://beamcollege.dev/)** as we continue updating the schedule and follow our **[Twitter](https://twitter.com/beam_college?ref_src=twsrc%5Egoogle%7Ctwcamp%5Eserp%7Ctwgr%5Eauthor)** and **[LinkedIn](https://www.linkedin.com/showcase/beam-college/)** pages to receive event updates! + +### Ways to Help & Participate + +1. Interested in instructing? Submit a **[proposal](https://docs.google.com/forms/d/e/1FAIpQLSct6RCrKtgsvxlgngKUGwKoB_iOKihXi1OadKyBQIsi00p3cQ/viewform?usp=sf_link)**! The deadline is: _February 28th._ +2. Enroll in Beam College. Registration is now open on the **[registration page](https://beamcollege.dev/step/2022/)**. +3. Consider partnering with the event. If your company is interested in helping to promote the event and being a part of the branding, please fill out this **form**. +4. Help us get the word out by letting your network know about this exciting opportunity to help users uplevel data processing skills, solve complex data applications, and optimize data pipelines! + +## Beam Meetups + +In partnership with an event production company, Beam will be hosting an average of one virtual Meetup per month. These Meetups will be relaxed presentations on topics or demos followed by a Q&A session. The objective of our virtual meetups is to give the community an update on the most recent Beam features launched within the past six months. These meetups are free and open to the public. + +Check out recordings from previous Meetups **[here](https://www.youtube.com/watch?v=8fNEs7SbefM&list=PL4dEBWmGSIU-cQSpYP7R1lSC6e2K_pTf1)**! + +### Ways to Help & Participate + +1. Are you interested in sharing a feature launch or sharing a step-by-step use case for Beam? Submit a **[talk idea](https://docs.google.com/forms/d/e/1FAIpQLScFg7fmOFc7fTvnJL_dmdhia4HDesW4HYxJsDeulnsHzIzqCg/viewform)**! +2. Register for the events. Registration is now open on the **[registration page](https://clowder.space/projects/apache-beam/)**. +3. Help us get the word out by spreading the word throughout the community to enable more knowledge sharing and collaboration! \ No newline at end of file diff --git a/website/www/site/data/authors.yml b/website/www/site/data/authors.yml index 39f53abad901..85d37b29b9a3 100644 --- a/website/www/site/data/authors.yml +++ b/website/www/site/data/authors.yml @@ -21,6 +21,7 @@ aljoscha: altay: name: Ahmet Altay email: altay@apache.org + twitter: iridium77 angoenka: name: Ankur Goenka email: goenka@apache.org @@ -208,7 +209,6 @@ yifanzou: name: Yifan Zou email: yifanzou@apache.org twitter: - arturkhanin: name: Artur Khanin email: artur.khanin@akvelon.com @@ -221,3 +221,6 @@ alexkosolapov: name: Alex Kosolapov email: alex.kosolapov@akvelon.com twitter: +hermannb: + name: Brittany Hermann + email: hermannb@google.com From 1f2f0534ed3b4315c9d45144a4b439e3e20a8da8 Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Tue, 1 Mar 2022 15:56:17 -0500 Subject: [PATCH 09/68] Clean up Go formatter suggestions (#16973) --- .../beam/core/runtime/exec/sideinput_test.go | 40 +++++++++---------- sdks/go/pkg/beam/util/harnessopts/cache.go | 2 +- .../pkg/beam/util/syscallx/syscall_default.go | 1 + .../pkg/beam/util/syscallx/syscall_linux.go | 1 + 4 files changed, 23 insertions(+), 21 deletions(-) diff --git a/sdks/go/pkg/beam/core/runtime/exec/sideinput_test.go b/sdks/go/pkg/beam/core/runtime/exec/sideinput_test.go index 30990642aed3..3293fe44a51e 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/sideinput_test.go +++ b/sdks/go/pkg/beam/core/runtime/exec/sideinput_test.go @@ -67,26 +67,26 @@ func TestNewSideInputAdapter(t *testing.T) { } for _, test := range tests { t.Run(test.name, func(t *testing.T) { - adapter := NewSideInputAdapter(test.sid, test.sideInputID, test.c, nil) - adapterStruct, ok := adapter.(*sideInputAdapter) - if !ok { - t.Errorf("failed to convert interface to sideInputAdapter struct in test %v", test) - } - if got, want := adapterStruct.sid, test.sid; got != want { - t.Errorf("got SID %v, want %v", got, want) - } - if got, want := adapterStruct.sideInputID, test.sideInputID; got != want { - t.Errorf("got sideInputID %v, want %v", got, want) - } - if got, want := adapterStruct.c, test.c; got != want { - t.Errorf("got coder %v, want %v", got, want) - } - if got, want := reflect.TypeOf(adapterStruct.kc), reflect.TypeOf(test.kc); got != want { - t.Errorf("got ElementEncoder type %v, want %v", got, want) - } - if got, want := reflect.TypeOf(adapterStruct.ec), reflect.TypeOf(test.ec); got != want { - t.Errorf("got ElementDecoder type %v, want %v", got, want) - } + adapter := NewSideInputAdapter(test.sid, test.sideInputID, test.c, nil) + adapterStruct, ok := adapter.(*sideInputAdapter) + if !ok { + t.Errorf("failed to convert interface to sideInputAdapter struct in test %v", test) + } + if got, want := adapterStruct.sid, test.sid; got != want { + t.Errorf("got SID %v, want %v", got, want) + } + if got, want := adapterStruct.sideInputID, test.sideInputID; got != want { + t.Errorf("got sideInputID %v, want %v", got, want) + } + if got, want := adapterStruct.c, test.c; got != want { + t.Errorf("got coder %v, want %v", got, want) + } + if got, want := reflect.TypeOf(adapterStruct.kc), reflect.TypeOf(test.kc); got != want { + t.Errorf("got ElementEncoder type %v, want %v", got, want) + } + if got, want := reflect.TypeOf(adapterStruct.ec), reflect.TypeOf(test.ec); got != want { + t.Errorf("got ElementDecoder type %v, want %v", got, want) + } }) } } diff --git a/sdks/go/pkg/beam/util/harnessopts/cache.go b/sdks/go/pkg/beam/util/harnessopts/cache.go index 117f45f00825..2ad538b73af2 100644 --- a/sdks/go/pkg/beam/util/harnessopts/cache.go +++ b/sdks/go/pkg/beam/util/harnessopts/cache.go @@ -13,7 +13,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -// Package harnessopts defines user-facing entrypoints into Beam hooks +// Package harnessopts defines user-facing entrypoints into Beam hooks // affecting the SDK harness. Call these functions at any time before // submitting your pipeline to a runner, for that pipeline's workers to be affected. package harnessopts diff --git a/sdks/go/pkg/beam/util/syscallx/syscall_default.go b/sdks/go/pkg/beam/util/syscallx/syscall_default.go index a85cd3f0e7e2..55756d0dbd44 100644 --- a/sdks/go/pkg/beam/util/syscallx/syscall_default.go +++ b/sdks/go/pkg/beam/util/syscallx/syscall_default.go @@ -13,6 +13,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +//go:build !linux // +build !linux package syscallx diff --git a/sdks/go/pkg/beam/util/syscallx/syscall_linux.go b/sdks/go/pkg/beam/util/syscallx/syscall_linux.go index c639f876bebc..379437ad0a17 100644 --- a/sdks/go/pkg/beam/util/syscallx/syscall_linux.go +++ b/sdks/go/pkg/beam/util/syscallx/syscall_linux.go @@ -13,6 +13,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +//go:build linux // +build linux package syscallx From dd1d343fa621ebfc41c1ac5eba1e334a37ce1f73 Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Tue, 1 Mar 2022 15:57:57 -0500 Subject: [PATCH 10/68] [BEAM-14012] Add go fmt to Github Actions (#16978) --- .github/workflows/go_tests.yml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/.github/workflows/go_tests.yml b/.github/workflows/go_tests.yml index f58bfcc8a8a3..b70ec8141049 100644 --- a/.github/workflows/go_tests.yml +++ b/.github/workflows/go_tests.yml @@ -49,5 +49,7 @@ jobs: flags: go files: ./sdks/go/pkg/coverage.txt name: go-unittests + - name: Run fmt + run: cd sdks/go/pkg/beam && go fmt ./...; git diff-index --quiet HEAD || (echo "Run go fmt before checking in changes" && exit 1) - name: Run vet - run: cd sdks/go/pkg/beam && go vet --copylocks=false --unsafeptr=false ./... + run: cd sdks/go/pkg/beam && go vet --copylocks=false --unsafeptr=false ./... || (echo "Run go vet and fix warnings before checking in changes" && exit 1) From 118a9210ccfa1cfd872b64c0709c9130e87f6c18 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Tue, 1 Mar 2022 13:37:30 -0800 Subject: [PATCH 11/68] [BEAM-13911] Add basic tests to Go direct runner. (#16979) --- .../go/pkg/beam/runners/direct/direct_test.go | 455 ++++++++++++++++++ 1 file changed, 455 insertions(+) create mode 100644 sdks/go/pkg/beam/runners/direct/direct_test.go diff --git a/sdks/go/pkg/beam/runners/direct/direct_test.go b/sdks/go/pkg/beam/runners/direct/direct_test.go new file mode 100644 index 000000000000..b65d33f90919 --- /dev/null +++ b/sdks/go/pkg/beam/runners/direct/direct_test.go @@ -0,0 +1,455 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package direct + +import ( + "context" + "flag" + "fmt" + "os" + "reflect" + "sort" + "testing" + + "github.com/apache/beam/sdks/v2/go/pkg/beam" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/metrics" + "github.com/google/go-cmp/cmp" +) + +func executeWithT(ctx context.Context, t *testing.T, p *beam.Pipeline) (beam.PipelineResult, error) { + fmt.Println("startingTest - ", t.Name()) + return Execute(ctx, p) +} + +func init() { + beam.RegisterFunction(dofn1) + beam.RegisterFunction(dofn1x2) + beam.RegisterFunction(dofn1x5) + beam.RegisterFunction(dofn2x1) + beam.RegisterFunction(dofn3x1) + beam.RegisterFunction(dofn2x2KV) + beam.RegisterFunction(dofn2) + beam.RegisterFunction(dofnKV) + beam.RegisterFunction(dofnKV2) + beam.RegisterFunction(dofnGBK) + beam.RegisterFunction(dofnGBK2) + beam.RegisterType(reflect.TypeOf((*int64Check)(nil))) + beam.RegisterType(reflect.TypeOf((*stringCheck)(nil))) + + beam.RegisterType(reflect.TypeOf((*testRow)(nil))) + beam.RegisterFunction(dofnKV3) + beam.RegisterFunction(dofnGBK3) + + beam.RegisterFunction(dofn1Counter) + beam.RegisterFunction(dofnSink) +} + +func dofn1(imp []byte, emit func(int64)) { + emit(1) + emit(2) + emit(3) +} + +func dofn1x2(imp []byte, emitA func(int64), emitB func(int64)) { + emitA(1) + emitA(2) + emitA(3) + emitB(4) + emitB(5) + emitB(6) +} + +func dofn1x5(imp []byte, emitA, emitB, emitC, emitD, emitE func(int64)) { + emitA(1) + emitB(2) + emitC(3) + emitD(4) + emitE(5) + emitA(6) + emitB(7) + emitC(8) + emitD(9) + emitE(10) +} + +func dofn2x1(imp []byte, iter func(*int64) bool, emit func(int64)) { + var v, sum int64 + for iter(&v) { + sum += v + } + emit(sum) +} + +func dofn3x1(sum int64, iter1, iter2 func(*int64) bool, emit func(int64)) { + var v int64 + for iter1(&v) { + sum += v + } + for iter2(&v) { + sum += v + } + emit(sum) +} + +func dofn2x2KV(imp []byte, iter func(*string, *int64) bool, emitK func(string), emitV func(int64)) { + var k string + var v, sum int64 + for iter(&k, &v) { + sum += v + emitK(k) + } + emitV(sum) +} + +// int64Check validates that within a single bundle, +// we received the expected int64 values. +type int64Check struct { + Name string + Want []int + got []int +} + +func (fn *int64Check) ProcessElement(v int64, _ func(int64)) { + fn.got = append(fn.got, int(v)) +} + +func (fn *int64Check) FinishBundle(_ func(int64)) error { + sort.Ints(fn.got) + sort.Ints(fn.Want) + if d := cmp.Diff(fn.Want, fn.got); d != "" { + return fmt.Errorf("int64Check[%v] (-want, +got): %v", fn.Name, d) + } + return nil +} + +// stringCheck validates that within a single bundle, +// we received the expected string values. +type stringCheck struct { + Name string + Want []string + got []string +} + +func (fn *stringCheck) ProcessElement(v string, _ func(string)) { + fn.got = append(fn.got, v) +} + +func (fn *stringCheck) FinishBundle(_ func(string)) error { + sort.Strings(fn.got) + sort.Strings(fn.Want) + if d := cmp.Diff(fn.Want, fn.got); d != "" { + return fmt.Errorf("stringCheck[%v] (-want, +got): %v", fn.Name, d) + } + return nil +} + +func dofn2(v int64, emit func(int64)) { + emit(v + 1) +} + +func dofnKV(imp []byte, emit func(string, int64)) { + emit("a", 1) + emit("b", 2) + emit("a", 3) + emit("b", 4) + emit("a", 5) + emit("b", 6) +} + +func dofnKV2(imp []byte, emit func(int64, string)) { + emit(1, "a") + emit(2, "b") + emit(1, "a") + emit(2, "b") + emit(1, "a") + emit(2, "b") +} + +func dofnGBK(k string, vs func(*int64) bool, emit func(int64)) { + var v, sum int64 + for vs(&v) { + sum += v + } + emit(sum) +} + +func dofnGBK2(k int64, vs func(*string) bool, emit func(string)) { + var v, sum string + for vs(&v) { + sum += v + } + emit(sum) +} + +type testRow struct { + A string + B int64 +} + +func dofnKV3(imp []byte, emit func(testRow, testRow)) { + emit(testRow{"a", 1}, testRow{"a", 1}) +} + +func dofnGBK3(k testRow, vs func(*testRow) bool, emit func(string)) { + var v testRow + vs(&v) + emit(fmt.Sprintf("%v: %v", k, v)) +} + +const ( + ns = "directtest" +) + +func dofnSink(ctx context.Context, _ []byte) { + beam.NewCounter(ns, "sunk").Inc(ctx, 73) +} + +func dofn1Counter(ctx context.Context, _ []byte, emit func(int64)) { + beam.NewCounter(ns, "count").Inc(ctx, 1) +} + +func TestRunner_Pipelines(t *testing.T) { + t.Run("simple", func(t *testing.T) { + p, s := beam.NewPipelineWithRoot() + imp := beam.Impulse(s) + col := beam.ParDo(s, dofn1, imp) + beam.ParDo(s, &int64Check{ + Name: "simple", + Want: []int{1, 2, 3}, + }, col) + + if _, err := executeWithT(context.Background(), t, p); err != nil { + t.Fatal(err) + } + }) + t.Run("sequence", func(t *testing.T) { + p, s := beam.NewPipelineWithRoot() + imp := beam.Impulse(s) + beam.Seq(s, imp, dofn1, dofn2, dofn2, dofn2, &int64Check{Name: "sequence", Want: []int{4, 5, 6}}) + if _, err := executeWithT(context.Background(), t, p); err != nil { + t.Fatal(err) + } + }) + t.Run("gbk", func(t *testing.T) { + p, s := beam.NewPipelineWithRoot() + imp := beam.Impulse(s) + col := beam.ParDo(s, dofnKV, imp) + gbk := beam.GroupByKey(s, col) + beam.Seq(s, gbk, dofnGBK, &int64Check{Name: "gbk", Want: []int{9, 12}}) + if _, err := executeWithT(context.Background(), t, p); err != nil { + t.Fatal(err) + } + }) + t.Run("gbk2", func(t *testing.T) { + p, s := beam.NewPipelineWithRoot() + imp := beam.Impulse(s) + col := beam.ParDo(s, dofnKV2, imp) + gbk := beam.GroupByKey(s, col) + beam.Seq(s, gbk, dofnGBK2, &stringCheck{Name: "gbk2", Want: []string{"aaa", "bbb"}}) + if _, err := executeWithT(context.Background(), t, p); err != nil { + t.Fatal(err) + } + }) + t.Run("gbk3", func(t *testing.T) { + p, s := beam.NewPipelineWithRoot() + imp := beam.Impulse(s) + col := beam.ParDo(s, dofnKV3, imp) + gbk := beam.GroupByKey(s, col) + beam.Seq(s, gbk, dofnGBK3, &stringCheck{Name: "gbk3", Want: []string{"{a 1}: {a 1}"}}) + if _, err := executeWithT(context.Background(), t, p); err != nil { + t.Fatal(err) + } + }) + t.Run("sink_nooutputs", func(t *testing.T) { + p, s := beam.NewPipelineWithRoot() + imp := beam.Impulse(s) + beam.ParDo0(s, dofnSink, imp) + pr, err := executeWithT(context.Background(), t, p) + if err != nil { + t.Fatal(err) + } + qr := pr.Metrics().Query(func(sr metrics.SingleResult) bool { + return sr.Name() == "sunk" + }) + if got, want := qr.Counters()[0].Committed, int64(73); got != want { + t.Errorf("pr.Metrics.Query(Name = \"sunk\")).Committed = %v, want %v", got, want) + } + }) + t.Run("fork_impulse", func(t *testing.T) { + p, s := beam.NewPipelineWithRoot() + imp := beam.Impulse(s) + col1 := beam.ParDo(s, dofn1, imp) + col2 := beam.ParDo(s, dofn1, imp) + beam.ParDo(s, &int64Check{ + Name: "fork check1", + Want: []int{1, 2, 3}, + }, col1) + beam.ParDo(s, &int64Check{ + Name: "fork check2", + Want: []int{1, 2, 3}, + }, col2) + if _, err := executeWithT(context.Background(), t, p); err != nil { + t.Fatal(err) + } + }) + t.Run("fork_postDoFn", func(t *testing.T) { + p, s := beam.NewPipelineWithRoot() + imp := beam.Impulse(s) + col := beam.ParDo(s, dofn1, imp) + beam.ParDo(s, &int64Check{ + Name: "fork check1", + Want: []int{1, 2, 3}, + }, col) + beam.ParDo(s, &int64Check{ + Name: "fork check2", + Want: []int{1, 2, 3}, + }, col) + if _, err := executeWithT(context.Background(), t, p); err != nil { + t.Fatal(err) + } + }) + t.Run("fork_multipleOutputs1", func(t *testing.T) { + p, s := beam.NewPipelineWithRoot() + imp := beam.Impulse(s) + col1, col2 := beam.ParDo2(s, dofn1x2, imp) + beam.ParDo(s, &int64Check{ + Name: "col1", + Want: []int{1, 2, 3}, + }, col1) + beam.ParDo(s, &int64Check{ + Name: "col2", + Want: []int{4, 5, 6}, + }, col2) + if _, err := executeWithT(context.Background(), t, p); err != nil { + t.Fatal(err) + } + }) + t.Run("fork_multipleOutputs2", func(t *testing.T) { + p, s := beam.NewPipelineWithRoot() + imp := beam.Impulse(s) + col1, col2, col3, col4, col5 := beam.ParDo5(s, dofn1x5, imp) + beam.ParDo(s, &int64Check{ + Name: "col1", + Want: []int{1, 6}, + }, col1) + beam.ParDo(s, &int64Check{ + Name: "col2", + Want: []int{2, 7}, + }, col2) + beam.ParDo(s, &int64Check{ + Name: "col3", + Want: []int{3, 8}, + }, col3) + beam.ParDo(s, &int64Check{ + Name: "col4", + Want: []int{4, 9}, + }, col4) + beam.ParDo(s, &int64Check{ + Name: "col5", + Want: []int{5, 10}, + }, col5) + if _, err := executeWithT(context.Background(), t, p); err != nil { + t.Fatal(err) + } + }) + t.Run("flatten", func(t *testing.T) { + p, s := beam.NewPipelineWithRoot() + imp := beam.Impulse(s) + col1 := beam.ParDo(s, dofn1, imp) + col2 := beam.ParDo(s, dofn1, imp) + flat := beam.Flatten(s, col1, col2) + beam.ParDo(s, &int64Check{ + Name: "flatten check", + Want: []int{1, 1, 2, 2, 3, 3}, + }, flat) + if _, err := executeWithT(context.Background(), t, p); err != nil { + t.Fatal(err) + } + }) + t.Run("sideinput_iterable", func(t *testing.T) { + p, s := beam.NewPipelineWithRoot() + imp := beam.Impulse(s) + col1 := beam.ParDo(s, dofn1, imp) + sum := beam.ParDo(s, dofn2x1, imp, beam.SideInput{Input: col1}) + beam.ParDo(s, &int64Check{ + Name: "iter sideinput check", + Want: []int{6}, + }, sum) + if _, err := executeWithT(context.Background(), t, p); err != nil { + t.Fatal(err) + } + }) + t.Run("sideinput_iterableKV", func(t *testing.T) { + p, s := beam.NewPipelineWithRoot() + imp := beam.Impulse(s) + col1 := beam.ParDo(s, dofnKV, imp) + keys, sum := beam.ParDo2(s, dofn2x2KV, imp, beam.SideInput{Input: col1}) + beam.ParDo(s, &stringCheck{ + Name: "iterKV sideinput check K", + Want: []string{"a", "a", "a", "b", "b", "b"}, + }, keys) + beam.ParDo(s, &int64Check{ + Name: "iterKV sideinput check V", + Want: []int{21}, + }, sum) + if _, err := executeWithT(context.Background(), t, p); err != nil { + t.Fatal(err) + } + }) + // Validates the waiting on side input readiness in buffer. + t.Run("sideinput_2iterable", func(t *testing.T) { + p, s := beam.NewPipelineWithRoot() + imp := beam.Impulse(s) + col0 := beam.ParDo(s, dofn1, imp) + col1 := beam.ParDo(s, dofn1, imp) + col2 := beam.ParDo(s, dofn2, col1) + sum := beam.ParDo(s, dofn3x1, col0, beam.SideInput{Input: col1}, beam.SideInput{Input: col2}) + beam.ParDo(s, &int64Check{ + Name: "iter sideinput check", + Want: []int{16, 17, 18}, + }, sum) + if _, err := executeWithT(context.Background(), t, p); err != nil { + t.Fatal(err) + } + }) +} + +func TestRunner_Metrics(t *testing.T) { + t.Run("counter", func(t *testing.T) { + p, s := beam.NewPipelineWithRoot() + imp := beam.Impulse(s) + beam.ParDo(s, dofn1Counter, imp) + pr, err := executeWithT(context.Background(), t, p) + if err != nil { + t.Fatal(err) + } + qr := pr.Metrics().Query(func(sr metrics.SingleResult) bool { + return sr.Name() == "count" + }) + if got, want := qr.Counters()[0].Committed, int64(1); got != want { + t.Errorf("pr.Metrics.Query(Name = \"count\")).Committed = %v, want %v", got, want) + } + }) +} + +func TestMain(m *testing.M) { + // Can't use ptest since it causes a loop. + if !flag.Parsed() { + flag.Parse() + } + beam.Init() + os.Exit(m.Run()) +} From ac9cdc617a4a706f214d63addf9df03f384f57c6 Mon Sep 17 00:00:00 2001 From: laraschmidt Date: Tue, 1 Mar 2022 14:14:50 -0800 Subject: [PATCH 12/68] [BEAM-13960] Add support for more types when converting from between row and proto (#16875) * Adding schema support. * Addressing feedback. --- model/pipeline/src/main/proto/schema.proto | 3 +- .../beam/sdk/schemas/SchemaTranslation.java | 145 +++++++++++++++++- .../apache/beam/sdk/schemas/SchemaUtils.java | 21 +++ .../java/org/apache/beam/sdk/values/Row.java | 9 +- .../sdk/schemas/SchemaTranslationTest.java | 73 +++++++++ 5 files changed, 242 insertions(+), 9 deletions(-) diff --git a/model/pipeline/src/main/proto/schema.proto b/model/pipeline/src/main/proto/schema.proto index 23c32a0db968..b26fc8fef8d6 100644 --- a/model/pipeline/src/main/proto/schema.proto +++ b/model/pipeline/src/main/proto/schema.proto @@ -113,7 +113,7 @@ message LogicalType { message Option { // REQUIRED. Identifier for the option. string name = 1; - // REQUIRED. Type specifer for the structure of value. + // REQUIRED. Type specifier for the structure of value. // Conventionally, options that don't require additional configuration should // use a boolean type, with the value set to true. FieldType type = 2; @@ -125,6 +125,7 @@ message Row { } message FieldValue { + // If none of these are set, value is considered null. oneof field_value { AtomicTypeValue atomic_value = 1; ArrayTypeValue array_value = 2; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java index 4e6921b5226b..25983caf2b15 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java @@ -17,6 +17,11 @@ */ package org.apache.beam.sdk.schemas; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.PipedInputStream; +import java.io.PipedOutputStream; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -28,6 +33,7 @@ import org.apache.beam.model.pipeline.v1.SchemaApi.AtomicTypeValue; import org.apache.beam.model.pipeline.v1.SchemaApi.FieldValue; import org.apache.beam.model.pipeline.v1.SchemaApi.IterableTypeValue; +import org.apache.beam.model.pipeline.v1.SchemaApi.LogicalTypeValue; import org.apache.beam.model.pipeline.v1.SchemaApi.MapTypeEntry; import org.apache.beam.model.pipeline.v1.SchemaApi.MapTypeValue; import org.apache.beam.sdk.annotations.Experimental; @@ -45,6 +51,7 @@ import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.ByteStreams; import org.checkerframework.checker.nullness.qual.Nullable; /** Utility methods for translating schemas. */ @@ -319,6 +326,7 @@ private static FieldType fieldTypeFromProtoWithoutNullable(SchemaApi.FieldType p fieldTypeFromProto(protoFieldType.getMapType().getValueType())); case LOGICAL_TYPE: String urn = protoFieldType.getLogicalType().getUrn(); + SchemaApi.LogicalType logicalType = protoFieldType.getLogicalType(); Class> logicalTypeClass = STANDARD_LOGICAL_TYPES.get(urn); if (logicalTypeClass != null) { try { @@ -351,22 +359,21 @@ private static FieldType fieldTypeFromProtoWithoutNullable(SchemaApi.FieldType p return FieldType.logicalType( (LogicalType) SerializableUtils.deserializeFromByteArray( - protoFieldType.getLogicalType().getPayload().toByteArray(), "logicalType")); + logicalType.getPayload().toByteArray(), "logicalType")); } else { @Nullable FieldType argumentType = null; @Nullable Object argumentValue = null; - if (protoFieldType.getLogicalType().hasArgumentType()) { - argumentType = fieldTypeFromProto(protoFieldType.getLogicalType().getArgumentType()); - argumentValue = - fieldValueFromProto(argumentType, protoFieldType.getLogicalType().getArgument()); + if (logicalType.hasArgumentType()) { + argumentType = fieldTypeFromProto(logicalType.getArgumentType()); + argumentValue = fieldValueFromProto(argumentType, logicalType.getArgument()); } return FieldType.logicalType( new UnknownLogicalType( urn, - protoFieldType.getLogicalType().getPayload().toByteArray(), + logicalType.getPayload().toByteArray(), argumentType, argumentValue, - fieldTypeFromProto(protoFieldType.getLogicalType().getRepresentation()))); + fieldTypeFromProto(logicalType.getRepresentation()))); } default: throw new IllegalArgumentException( @@ -393,6 +400,14 @@ public static Object rowFromProto(SchemaApi.Row row, FieldType fieldType) { static SchemaApi.FieldValue fieldValueToProto(FieldType fieldType, Object value) { FieldValue.Builder builder = FieldValue.newBuilder(); + if (value == null) { + if (fieldType.getNullable()) { + return builder.build(); + } else { + throw new RuntimeException("Null value found for field that doesn't support nulls."); + } + } + switch (fieldType.getTypeName()) { case ARRAY: return builder @@ -411,26 +426,74 @@ static SchemaApi.FieldValue fieldValueToProto(FieldType fieldType, Object value) .build(); case ROW: return builder.setRowValue(rowToProto((Row) value)).build(); + case DATETIME: + return builder + .setLogicalTypeValue(logicalTypeToProto(FieldType.INT64, fieldType, value)) + .build(); + case DECIMAL: + return builder + .setLogicalTypeValue(logicalTypeToProto(FieldType.BYTES, fieldType, value)) + .build(); case LOGICAL_TYPE: + return builder + .setLogicalTypeValue(logicalTypeToProto(fieldType.getLogicalType(), value)) + .build(); default: return builder.setAtomicValue(primitiveRowFieldToProto(fieldType, value)).build(); } } + /** Returns if the given field is null and throws exception if it is and can't be. */ + static boolean isNullFieldValueFromProto(FieldType fieldType, boolean hasNonNullValue) { + if (!hasNonNullValue && !fieldType.getNullable()) { + throw new RuntimeException("FieldTypeValue has no value but the field cannot be null."); + } + return !hasNonNullValue; + } + static Object fieldValueFromProto(FieldType fieldType, SchemaApi.FieldValue value) { switch (fieldType.getTypeName()) { case ARRAY: + if (isNullFieldValueFromProto(fieldType, value.hasArrayValue())) { + return null; + } return arrayValueFromProto(fieldType.getCollectionElementType(), value.getArrayValue()); case ITERABLE: + if (isNullFieldValueFromProto(fieldType, value.hasIterableValue())) { + return null; + } return iterableValueFromProto( fieldType.getCollectionElementType(), value.getIterableValue()); case MAP: + if (isNullFieldValueFromProto(fieldType, value.hasMapValue())) { + return null; + } return mapFromProto( fieldType.getMapKeyType(), fieldType.getMapValueType(), value.getMapValue()); case ROW: + if (isNullFieldValueFromProto(fieldType, value.hasRowValue())) { + return null; + } return rowFromProto(value.getRowValue(), fieldType); case LOGICAL_TYPE: + if (isNullFieldValueFromProto(fieldType, value.hasLogicalTypeValue())) { + return null; + } + return logicalTypeFromProto(fieldType.getLogicalType(), value); + case DATETIME: + if (isNullFieldValueFromProto(fieldType, value.hasLogicalTypeValue())) { + return null; + } + return logicalTypeFromProto(FieldType.INT64, fieldType, value.getLogicalTypeValue()); + case DECIMAL: + if (isNullFieldValueFromProto(fieldType, value.hasLogicalTypeValue())) { + return null; + } + return logicalTypeFromProto(FieldType.BYTES, fieldType, value.getLogicalTypeValue()); default: + if (isNullFieldValueFromProto(fieldType, value.hasAtomicValue())) { + return null; + } return primitiveFromProto(fieldType, value.getAtomicValue()); } } @@ -485,6 +548,74 @@ private static Object mapFromProto( entry -> fieldValueFromProto(mapValueType, entry.getValue()))); } + /** Converts logical type value from proto using a default type coder. */ + private static Object logicalTypeFromProto( + FieldType baseType, FieldType inputType, LogicalTypeValue value) { + try { + PipedInputStream in = new PipedInputStream(); + DataOutputStream stream = new DataOutputStream(new PipedOutputStream(in)); + switch (baseType.getTypeName()) { + case INT64: + stream.writeLong(value.getValue().getAtomicValue().getInt64()); + break; + case BYTES: + stream.write(value.getValue().getAtomicValue().getBytes().toByteArray()); + break; + default: + throw new UnsupportedOperationException( + "Unsupported underlying type for parsing logical type via coder."); + } + stream.close(); + return SchemaCoderHelpers.coderForFieldType(inputType).decode(in); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + /** Converts logical type value to a proto using a default type coder. */ + private static LogicalTypeValue logicalTypeToProto( + FieldType baseType, FieldType inputType, Object value) { + try { + PipedInputStream in = new PipedInputStream(); + PipedOutputStream out = new PipedOutputStream(in); + SchemaCoderHelpers.coderForFieldType(inputType).encode(value, out); + out.close(); // Close required for toByteArray. + Object baseObject; + switch (baseType.getTypeName()) { + case INT64: + baseObject = new DataInputStream(in).readLong(); + break; + case BYTES: + baseObject = ByteStreams.toByteArray(in); + break; + default: + throw new UnsupportedOperationException( + "Unsupported underlying type for producing LogicalType via coder."); + } + return LogicalTypeValue.newBuilder() + .setValue(fieldValueToProto(baseType, baseObject)) + .build(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private static LogicalTypeValue logicalTypeToProto(LogicalType logicalType, Object value) { + return LogicalTypeValue.newBuilder() + .setValue( + fieldValueToProto( + logicalType.getBaseType(), SchemaUtils.toLogicalBaseType(logicalType, value))) + .build(); + } + + private static Object logicalTypeFromProto( + LogicalType logicalType, SchemaApi.FieldValue logicalValue) { + return SchemaUtils.toLogicalInputType( + logicalType, + fieldValueFromProto( + logicalType.getBaseType(), logicalValue.getLogicalTypeValue().getValue())); + } + private static AtomicTypeValue primitiveRowFieldToProto(FieldType fieldType, Object value) { switch (fieldType.getTypeName()) { case BYTE: diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaUtils.java index ebddfb38afa7..da08269fd4b1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaUtils.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.schemas; import org.apache.beam.sdk.schemas.Schema.FieldType; +import org.apache.beam.sdk.schemas.Schema.LogicalType; /** A set of utility functions for schemas. */ @SuppressWarnings({ @@ -101,4 +102,24 @@ static FieldType widenNullableTypes(FieldType fieldType1, FieldType fieldType2) } return result.withNullable(fieldType1.getNullable() || fieldType2.getNullable()); } + + /** + * Returns the base type given a logical type and the input type. + * + *

This function can be used to handle logical types without knowing InputT or BaseT. + */ + public static BaseT toLogicalBaseType( + LogicalType logicalType, InputT inputType) { + return logicalType.toBaseType(inputType); + } + + /** + * Returns the input type given a logical type and the base type. + * + *

This function can be used to handle logical types without knowing InputT or BaseT. + */ + public static InputT toLogicalInputType( + LogicalType logicalType, BaseT baseType) { + return logicalType.toInputType(baseType); + } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java index 9dd02d32c2f3..f7f434d02c47 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java @@ -43,6 +43,7 @@ import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.FieldType; import org.apache.beam.sdk.schemas.Schema.TypeName; +import org.apache.beam.sdk.schemas.SchemaUtils; import org.apache.beam.sdk.values.RowUtils.CapturingRowCases; import org.apache.beam.sdk.values.RowUtils.FieldOverride; import org.apache.beam.sdk.values.RowUtils.FieldOverrides; @@ -460,7 +461,10 @@ public static boolean deepEquals(Object a, Object b, Schema.FieldType fieldType) if (a == null || b == null) { return a == b; } else if (fieldType.getTypeName() == TypeName.LOGICAL_TYPE) { - return deepEquals(a, b, fieldType.getLogicalType().getBaseType()); + return deepEquals( + SchemaUtils.toLogicalBaseType(fieldType.getLogicalType(), a), + SchemaUtils.toLogicalBaseType(fieldType.getLogicalType(), b), + fieldType.getLogicalType().getBaseType()); } else if (fieldType.getTypeName() == Schema.TypeName.BYTES) { return Arrays.equals((byte[]) a, (byte[]) b); } else if (fieldType.getTypeName() == TypeName.ARRAY) { @@ -598,6 +602,9 @@ public String toString(boolean includeFieldNames) { } private String toString(Schema.FieldType fieldType, Object value, boolean includeFieldNames) { + if (value == null) { + return ""; + } StringBuilder builder = new StringBuilder(); switch (fieldType.getTypeName()) { case ARRAY: diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java index 9f1f7d47efa7..2c0cadb45eaa 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java @@ -22,6 +22,8 @@ import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThrows; +import java.math.BigDecimal; +import java.time.LocalDateTime; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -36,12 +38,14 @@ import org.apache.beam.model.pipeline.v1.SchemaApi.LogicalType; import org.apache.beam.sdk.schemas.Schema.Field; import org.apache.beam.sdk.schemas.Schema.FieldType; +import org.apache.beam.sdk.schemas.logicaltypes.DateTime; import org.apache.beam.sdk.schemas.logicaltypes.FixedBytes; import org.apache.beam.sdk.schemas.logicaltypes.MicrosInstant; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.grpc.v1p43p2.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.joda.time.Instant; import org.junit.Test; import org.junit.experimental.runners.Enclosed; import org.junit.runner.RunWith; @@ -173,6 +177,9 @@ public static Iterable data() { .add( Schema.of( Field.of("null_argument", FieldType.logicalType(new NullArgumentLogicalType())))) + .add(Schema.of(Field.of("logical_argument", FieldType.logicalType(new DateTime())))) + .add( + Schema.of(Field.of("single_arg_argument", FieldType.logicalType(FixedBytes.of(100))))) .build(); } @@ -290,6 +297,72 @@ public void fromProtoAndToProto() throws Exception { } } + /** Tests round-trip proto encodings for {@link Row}. */ + @RunWith(Parameterized.class) + public static class RowToFromProtoTest { + + public static Row simpleRow(FieldType type, Object value) { + return Row.withSchema(Schema.of(Field.of("s", type))).addValue(value).build(); + } + + public static Row simpleNullRow(FieldType type) { + return Row.withSchema(Schema.of(Field.nullable("s", type))).addValue(null).build(); + } + + @Parameters(name = "{index}: {0}") + public static Iterable data() { + Map map = new HashMap<>(); + map.put("string", 42); + List list = new ArrayList<>(); + list.add("string"); + Schema schema = + Schema.builder() + .addField("field_one", FieldType.STRING) + .addField("field_two", FieldType.INT32) + .build(); + Row row = Row.withSchema(schema).addValue("value").addValue(42).build(); + + return ImmutableList.builder() + .add(simpleRow(FieldType.STRING, "string")) + .add(simpleRow(FieldType.BOOLEAN, true)) + .add(simpleRow(FieldType.BYTE, (byte) 12)) + .add(simpleRow(FieldType.INT16, (short) 12)) + .add(simpleRow(FieldType.INT32, 12)) + .add(simpleRow(FieldType.INT64, 12L)) + .add(simpleRow(FieldType.BYTES, new byte[] {0x42, 0x69, 0x00})) + .add(simpleRow(FieldType.FLOAT, (float) 12)) + .add(simpleRow(FieldType.DOUBLE, 12.0)) + .add(simpleRow(FieldType.map(FieldType.STRING, FieldType.INT32), map)) + .add(simpleRow(FieldType.array(FieldType.STRING), list)) + .add(simpleRow(FieldType.row(row.getSchema()), row)) + .add(simpleRow(FieldType.DATETIME, new Instant(23L))) + .add(simpleRow(FieldType.DECIMAL, BigDecimal.valueOf(100000))) + .add(simpleRow(FieldType.logicalType(new NullArgumentLogicalType()), "str")) + .add(simpleRow(FieldType.logicalType(new DateTime()), LocalDateTime.of(2000, 1, 3, 3, 1))) + .add(simpleNullRow(FieldType.STRING)) + .add(simpleNullRow(FieldType.INT32)) + .add(simpleNullRow(FieldType.map(FieldType.STRING, FieldType.INT32))) + .add(simpleNullRow(FieldType.array(FieldType.STRING))) + .add(simpleNullRow(FieldType.row(row.getSchema()))) + .add(simpleNullRow(FieldType.logicalType(new NullArgumentLogicalType()))) + .add(simpleNullRow(FieldType.logicalType(new DateTime()))) + .add(simpleNullRow(FieldType.DECIMAL)) + .add(simpleNullRow(FieldType.DATETIME)) + .build(); + } + + @Parameter(0) + public Row row; + + @Test + public void toAndFromProto() throws Exception { + SchemaApi.Row rowProto = SchemaTranslation.rowToProto(row); + Row decodedRow = + (Row) SchemaTranslation.rowFromProto(rowProto, FieldType.row(row.getSchema())); + assertThat(decodedRow, equalTo(row)); + } + } + /** Tests that we raise helpful errors when decoding bad {@link Schema} protos. */ @RunWith(JUnit4.class) public static class DecodeErrorTest { From f4fa0ea922d4237b8bb91e9a7cedecff76647960 Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Wed, 2 Mar 2022 18:24:49 +0100 Subject: [PATCH 13/68] Bump org.mongodb:mongo-java-driver to 3.12.10 --- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index cb5d97953615..08b3dca52a3d 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -644,7 +644,7 @@ class BeamModulePlugin implements Plugin { kafka : "org.apache.kafka:kafka_2.11:$kafka_version", kafka_clients : "org.apache.kafka:kafka-clients:$kafka_version", mockito_core : "org.mockito:mockito-core:3.7.7", - mongo_java_driver : "org.mongodb:mongo-java-driver:3.12.7", + mongo_java_driver : "org.mongodb:mongo-java-driver:3.12.10", nemo_compiler_frontend_beam : "org.apache.nemo:nemo-compiler-frontend-beam:$nemo_version", netty_all : "io.netty:netty-all:$netty_version", netty_handler : "io.netty:netty-handler:$netty_version", From 9be4e23a2dc535b94537a78f4bd7ecda6c1d95cd Mon Sep 17 00:00:00 2001 From: Victor Date: Wed, 2 Mar 2022 14:06:56 -0500 Subject: [PATCH 14/68] [BEAM-13973] Link Dataproc Flink master URLs to the InteractiveRunner when FlinkRunner is used (#16904) --- .../dataproc/dataproc_cluster_manager.py | 153 +++++++++++- .../dataproc/dataproc_cluster_manager_test.py | 232 ++++++++++++++++-- .../runners/interactive/interactive_beam.py | 13 +- .../interactive/interactive_beam_test.py | 17 +- .../runners/interactive/interactive_runner.py | 23 +- .../interactive/interactive_runner_test.py | 37 ++- .../runners/interactive/utils_test.py | 3 - 7 files changed, 412 insertions(+), 66 deletions(-) diff --git a/sdks/python/apache_beam/runners/interactive/dataproc/dataproc_cluster_manager.py b/sdks/python/apache_beam/runners/interactive/dataproc/dataproc_cluster_manager.py index d1c2734a8c7a..4a9c688f2f97 100644 --- a/sdks/python/apache_beam/runners/interactive/dataproc/dataproc_cluster_manager.py +++ b/sdks/python/apache_beam/runners/interactive/dataproc/dataproc_cluster_manager.py @@ -18,10 +18,26 @@ # pytype: skip-file import logging +import re +import time from dataclasses import dataclass from typing import Optional +from typing import Tuple +from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.runners.interactive import interactive_environment as ie +from apache_beam.runners.interactive.utils import progress_indicated +from apache_beam.version import __version__ as beam_version + +try: + from google.cloud import dataproc_v1 + from apache_beam.io.gcp import gcsfilesystem #pylint: disable=ungrouped-imports +except ImportError: + + class UnimportedDataproc: + Cluster = None + + dataproc_v1 = UnimportedDataproc() _LOGGER = logging.getLogger(__name__) @@ -51,6 +67,9 @@ class DataprocClusterManager: required for creating and deleting Dataproc clusters for use under Interactive Beam. """ + IMAGE_VERSION = '2.0.31-debian10' + STAGING_LOG_NAME = 'dataproc-startup-script_output' + def __init__(self, cluster_metadata: MasterURLIdentifier) -> None: """Initializes the DataprocClusterManager with properties required to interface with the Dataproc ClusterControllerClient. @@ -69,7 +88,6 @@ def __init__(self, cluster_metadata: MasterURLIdentifier) -> None: self.cluster_metadata.cluster_name = ie.current_env( ).clusters.default_cluster_name - from google.cloud import dataproc_v1 self._cluster_client = dataproc_v1.ClusterControllerClient( client_options={ 'api_endpoint': \ @@ -79,9 +97,16 @@ def __init__(self, cluster_metadata: MasterURLIdentifier) -> None: if self.cluster_metadata in ie.current_env().clusters.master_urls.inverse: self.master_url = ie.current_env().clusters.master_urls.inverse[ self.cluster_metadata] + self.dashboard = ie.current_env().clusters.master_urls_to_dashboards[ + self.master_url] else: self.master_url = None + self.dashboard = None + self._fs = gcsfilesystem.GCSFileSystem(PipelineOptions()) + self._staging_directory = None + + @progress_indicated def create_cluster(self, cluster: dict) -> None: """Attempts to create a cluster using attributes that were initialized with the DataprocClusterManager instance. @@ -103,7 +128,10 @@ def create_cluster(self, cluster: dict) -> None: _LOGGER.info( 'Cluster created successfully: %s', self.cluster_metadata.cluster_name) - self.master_url = self.get_master_url(self.cluster_metadata) + self._staging_directory = self.get_staging_location(self.cluster_metadata) + self.master_url, self.dashboard = self.get_master_url_and_dashboard( + self.cluster_metadata, + self._staging_directory) except Exception as e: if e.code == 409: _LOGGER.info( @@ -127,7 +155,6 @@ def create_cluster(self, cluster: dict) -> None: 'Unable to create cluster: %s', self.cluster_metadata.cluster_name) raise e - # TODO(victorhc): Add support for user-specified pip packages def create_flink_cluster(self) -> None: """Calls _create_cluster with a configuration that enables FlinkRunner.""" cluster = { @@ -135,11 +162,13 @@ def create_flink_cluster(self) -> None: 'cluster_name': self.cluster_metadata.cluster_name, 'config': { 'software_config': { + 'image_version': self.IMAGE_VERSION, 'optional_components': ['DOCKER', 'FLINK'] }, 'gce_cluster_config': { 'metadata': { - 'flink-start-yarn-session': 'true' + 'flink-start-yarn-session': 'true', + 'PIP_PACKAGES': 'apache-beam[gcp]=={}'.format(beam_version) }, 'service_account_scopes': [ 'https://www.googleapis.com/auth/cloud-platform' @@ -156,6 +185,8 @@ def cleanup(self) -> None: """Deletes the cluster that uses the attributes initialized with the DataprocClusterManager instance.""" try: + if self._staging_directory: + self.cleanup_staging_files(self._staging_directory) self._cluster_client.delete_cluster( request={ 'project_id': self.cluster_metadata.project_id, @@ -186,15 +217,111 @@ def describe(self) -> None: """Returns a dictionary describing the cluster.""" return { 'cluster_metadata': self.cluster_metadata, - 'master_url': self.master_url + 'master_url': self.master_url, + 'dashboard': self.dashboard } - def get_master_url(self, identifier) -> None: + def get_cluster_details( + self, cluster_metadata: MasterURLIdentifier) -> dataproc_v1.Cluster: + """Gets the Dataproc_v1 Cluster object for the current cluster manager.""" + try: + return self._cluster_client.get_cluster( + request={ + 'project_id': cluster_metadata.project_id, + 'region': cluster_metadata.region, + 'cluster_name': cluster_metadata.cluster_name + }) + except Exception as e: + if e.code == 403: + _LOGGER.error( + 'Due to insufficient project permissions, ' + 'unable to retrieve information for cluster: %s', + cluster_metadata.cluster_name) + raise ValueError( + 'You cannot view clusters in project: {}'.format( + cluster_metadata.project_id)) + elif e.code == 404: + _LOGGER.error( + 'Cluster does not exist: %s', cluster_metadata.cluster_name) + raise ValueError( + 'Cluster was not found: {}'.format(cluster_metadata.cluster_name)) + else: + _LOGGER.error( + 'Failed to get information for cluster: %s', + cluster_metadata.cluster_name) + raise e + + def wait_for_cluster_to_provision( + self, cluster_metadata: MasterURLIdentifier) -> None: + while self.get_cluster_details( + cluster_metadata).status.state.name == 'CREATING': + time.sleep(15) + + def get_staging_location(self, cluster_metadata: MasterURLIdentifier) -> str: + """Gets the staging bucket of an existing Dataproc cluster.""" + try: + self.wait_for_cluster_to_provision(cluster_metadata) + cluster_details = self.get_cluster_details(cluster_metadata) + bucket_name = cluster_details.config.config_bucket + gcs_path = 'gs://' + bucket_name + '/google-cloud-dataproc-metainfo/' + for file in self._fs._list(gcs_path): + if cluster_metadata.cluster_name in file.path: + # this file path split will look something like: + # ['gs://.../google-cloud-dataproc-metainfo/{staging_dir}/', + # '-{node-type}/dataproc-startup-script_output'] + return file.path.split(cluster_metadata.cluster_name)[0] + except Exception as e: + _LOGGER.error( + 'Failed to get %s cluster staging bucket.', + cluster_metadata.cluster_name) + raise e + + def parse_master_url_and_dashboard( + self, cluster_metadata: MasterURLIdentifier, + line: str) -> Tuple[str, str]: + """Parses the master_url and YARN application_id of the Flink process from + an input line. The line containing both the master_url and application id + is always formatted as such: + {text} Found Web Interface {master_url} of application + '{application_id}'.\\n + + Truncated example where '...' represents additional text between segments: + ... google-dataproc-startup[000]: ... activate-component-flink[0000]: + ...org.apache.flink.yarn.YarnClusterDescriptor... [] - + Found Web Interface example-master-url:50000 of application + 'application_123456789000_0001'. + + Returns the flink_master_url and dashboard link as a tuple.""" + cluster_details = self.get_cluster_details(cluster_metadata) + yarn_endpoint = cluster_details.config.endpoint_config.http_ports[ + 'YARN ResourceManager'] + segment = line.split('Found Web Interface ')[1].split(' of application ') + master_url = segment[0] + application_id = re.sub('\'|.\n', '', segment[1]) + dashboard = re.sub( + '/yarn/', + '/gateway/default/yarn/proxy/' + application_id + '/', + yarn_endpoint) + return master_url, dashboard + + def get_master_url_and_dashboard( + self, cluster_metadata: MasterURLIdentifier, + staging_bucket) -> Tuple[Optional[str], Optional[str]]: """Returns the master_url of the current cluster.""" - # TODO(victorhc): Implement the following method to fetch the cluster - # master_url from Dataproc. - return '.'.join([ - self.cluster_metadata.project_id, - self.cluster_metadata.region, - self.cluster_metadata.cluster_name - ]) + startup_logs = [] + for file in self._fs._list(staging_bucket): + if self.STAGING_LOG_NAME in file.path: + startup_logs.append(file.path) + + for log in startup_logs: + content = self._fs.open(log) + for line in content.readlines(): + decoded_line = line.decode() + if 'Found Web Interface' in decoded_line: + return self.parse_master_url_and_dashboard( + cluster_metadata, decoded_line) + return None, None + + def cleanup_staging_files(self, staging_directory: str) -> None: + staging_files = [file.path for file in self._fs._list(staging_directory)] + self._fs.delete(staging_files) diff --git a/sdks/python/apache_beam/runners/interactive/dataproc/dataproc_cluster_manager_test.py b/sdks/python/apache_beam/runners/interactive/dataproc/dataproc_cluster_manager_test.py index b34641d1e537..ba59cf6c4e6e 100644 --- a/sdks/python/apache_beam/runners/interactive/dataproc/dataproc_cluster_manager_test.py +++ b/sdks/python/apache_beam/runners/interactive/dataproc/dataproc_cluster_manager_test.py @@ -15,25 +15,57 @@ # limitations under the License. # +"""Tests for apache_beam.runners.interactive.dataproc. +dataproc_cluster_manager.""" # pytype: skip-file import unittest from unittest.mock import patch +from apache_beam.runners.interactive.dataproc.dataproc_cluster_manager import DataprocClusterManager +from apache_beam.runners.interactive.dataproc.dataproc_cluster_manager import MasterURLIdentifier + try: from google.cloud import dataproc_v1 # pylint: disable=unused-import - from apache_beam.runners.interactive.dataproc import dataproc_cluster_manager except ImportError: _dataproc_imported = False else: _dataproc_imported = True +class MockProperty: + def __init__(self, property, value): + object.__setattr__(self, property, value) + + class MockException(Exception): def __init__(self, code=-1): self.code = code +class MockCluster: + def __init__(self, config_bucket=None): + self.config = MockProperty('config_bucket', config_bucket) + self.status = MockProperty('state', MockProperty('name', None)) + + +class MockFileSystem: + def _list(self, dir=None): + return [MockProperty('path', 'test-path/dataproc-startup-script_output')] + + def open(self, dir=None): + return MockFileIO('test-line Found Web Interface test-master-url' \ + ' of application \'test-app-id\'.\n') + + +class MockFileIO: + def __init__(self, contents): + self.contents = contents + + def readlines(self): + return [self.contents.encode('utf-8')] + + @unittest.skipIf(not _dataproc_imported, 'dataproc package was not imported.') class DataprocClusterManagerTest(unittest.TestCase): """Unit test for DataprocClusterManager""" @@ -45,10 +77,9 @@ def test_create_cluster_default_already_exists(self, mock_cluster_client): Tests that no exception is thrown when a cluster already exists, but is using ie.current_env().clusters.default_cluster_name. """ - cluster_metadata = dataproc_cluster_manager.MasterURLIdentifier( + cluster_metadata = MasterURLIdentifier( project_id='test-project', region='test-region') - cluster_manager = dataproc_cluster_manager.DataprocClusterManager( - cluster_metadata) + cluster_manager = DataprocClusterManager(cluster_metadata) from apache_beam.runners.interactive.dataproc.dataproc_cluster_manager import _LOGGER with self.assertLogs(_LOGGER, level='INFO') as context_manager: cluster_manager.create_cluster({}) @@ -62,10 +93,9 @@ def test_create_cluster_permission_denied(self, mock_cluster_client): Tests that an exception is thrown when a user is trying to write to a project while having insufficient permissions. """ - cluster_metadata = dataproc_cluster_manager.MasterURLIdentifier( + cluster_metadata = MasterURLIdentifier( project_id='test-project', region='test-region') - cluster_manager = dataproc_cluster_manager.DataprocClusterManager( - cluster_metadata) + cluster_manager = DataprocClusterManager(cluster_metadata) from apache_beam.runners.interactive.dataproc.dataproc_cluster_manager import _LOGGER with self.assertLogs(_LOGGER, level='ERROR') as context_manager: self.assertRaises(ValueError, cluster_manager.create_cluster, {}) @@ -81,10 +111,9 @@ def test_create_cluster_region_does_not_exist(self, mock_cluster_client): Tests that an exception is thrown when a user specifies a region that does not exist. """ - cluster_metadata = dataproc_cluster_manager.MasterURLIdentifier( + cluster_metadata = MasterURLIdentifier( project_id='test-project', region='test-region') - cluster_manager = dataproc_cluster_manager.DataprocClusterManager( - cluster_metadata) + cluster_manager = DataprocClusterManager(cluster_metadata) from apache_beam.runners.interactive.dataproc.dataproc_cluster_manager import _LOGGER with self.assertLogs(_LOGGER, level='ERROR') as context_manager: self.assertRaises(ValueError, cluster_manager.create_cluster, {}) @@ -98,27 +127,29 @@ def test_create_cluster_other_exception(self, mock_cluster_client): Tests that an exception is thrown when the exception is not handled by any other case under _create_cluster. """ - cluster_metadata = dataproc_cluster_manager.MasterURLIdentifier( + cluster_metadata = MasterURLIdentifier( project_id='test-project', region='test-region') - cluster_manager = dataproc_cluster_manager.DataprocClusterManager( - cluster_metadata) + cluster_manager = DataprocClusterManager(cluster_metadata) from apache_beam.runners.interactive.dataproc.dataproc_cluster_manager import _LOGGER with self.assertLogs(_LOGGER, level='ERROR') as context_manager: self.assertRaises(MockException, cluster_manager.create_cluster, {}) self.assertTrue('Unable to create cluster' in context_manager.output[0]) + @patch( + 'apache_beam.runners.interactive.dataproc.dataproc_cluster_manager.' + 'DataprocClusterManager.cleanup_staging_files', + return_value=None) @patch( 'google.cloud.dataproc_v1.ClusterControllerClient.delete_cluster', side_effect=MockException(403)) - def test_cleanup_permission_denied(self, mock_cluster_client): + def test_cleanup_permission_denied(self, mock_cluster_client, mock_cleanup): """ Tests that an exception is thrown when a user is trying to delete a project that they have insufficient permissions for. """ - cluster_metadata = dataproc_cluster_manager.MasterURLIdentifier( + cluster_metadata = MasterURLIdentifier( project_id='test-project', region='test-region') - cluster_manager = dataproc_cluster_manager.DataprocClusterManager( - cluster_metadata) + cluster_manager = DataprocClusterManager(cluster_metadata) from apache_beam.runners.interactive.dataproc.dataproc_cluster_manager import _LOGGER with self.assertLogs(_LOGGER, level='ERROR') as context_manager: self.assertRaises(ValueError, cluster_manager.cleanup) @@ -126,40 +157,191 @@ def test_cleanup_permission_denied(self, mock_cluster_client): 'Due to insufficient project permissions' in context_manager.output[0]) + @patch( + 'apache_beam.runners.interactive.dataproc.dataproc_cluster_manager.' + 'DataprocClusterManager.cleanup_staging_files', + return_value=None) @patch( 'google.cloud.dataproc_v1.ClusterControllerClient.delete_cluster', side_effect=MockException(404)) - def test_cleanup_does_not_exist(self, mock_cluster_client): + def test_cleanup_does_not_exist(self, mock_cluster_client, mock_cleanup): """ Tests that an exception is thrown when cleanup attempts to delete a cluster that does not exist. """ - cluster_metadata = dataproc_cluster_manager.MasterURLIdentifier( + cluster_metadata = MasterURLIdentifier( project_id='test-project', region='test-region') - cluster_manager = dataproc_cluster_manager.DataprocClusterManager( - cluster_metadata) + cluster_manager = DataprocClusterManager(cluster_metadata) from apache_beam.runners.interactive.dataproc.dataproc_cluster_manager import _LOGGER with self.assertLogs(_LOGGER, level='ERROR') as context_manager: self.assertRaises(ValueError, cluster_manager.cleanup) self.assertTrue('Cluster does not exist' in context_manager.output[0]) + @patch( + 'apache_beam.runners.interactive.dataproc.dataproc_cluster_manager.' + 'DataprocClusterManager.cleanup_staging_files', + return_value=None) @patch( 'google.cloud.dataproc_v1.ClusterControllerClient.delete_cluster', side_effect=MockException()) - def test_cleanup_other_exception(self, mock_cluster_client): + def test_cleanup_other_exception(self, mock_cluster_client, mock_cleanup): """ Tests that an exception is thrown when the exception is not handled by any other case under cleanup. """ - cluster_metadata = dataproc_cluster_manager.MasterURLIdentifier( + cluster_metadata = MasterURLIdentifier( project_id='test-project', region='test-region') - cluster_manager = dataproc_cluster_manager.DataprocClusterManager( - cluster_metadata) + cluster_manager = DataprocClusterManager(cluster_metadata) from apache_beam.runners.interactive.dataproc.dataproc_cluster_manager import _LOGGER with self.assertLogs(_LOGGER, level='ERROR') as context_manager: self.assertRaises(MockException, cluster_manager.cleanup) self.assertTrue('Failed to delete cluster' in context_manager.output[0]) + @patch( + 'apache_beam.io.gcp.gcsfilesystem.GCSFileSystem._list', + return_value=[ + MockProperty( + 'path', + 'gs://test-bucket/google-cloud-dataproc-metainfo' + '/test-cluster/item') + ]) + @patch( + 'google.cloud.dataproc_v1.ClusterControllerClient.get_cluster', + return_value=MockCluster('test-bucket')) + def test_get_staging_location(self, mock_cluster_client, mock_list): + """ + Test to receive a mock staging location successfully under + get_staging_location. + """ + cluster_metadata = MasterURLIdentifier( + project_id='test-project', + region='test-region', + cluster_name='test-cluster') + cluster_manager = DataprocClusterManager(cluster_metadata) + self.assertEqual( + cluster_manager.get_staging_location(cluster_metadata), + 'gs://test-bucket/google-cloud-dataproc-metainfo/') + + @patch( + 'google.cloud.dataproc_v1.ClusterControllerClient.get_cluster', + side_effect=MockException()) + def test_get_staging_location_exception(self, mock_cluster_client): + """ + Test to catch when an error is raised inside get_staging_location. + """ + cluster_metadata = MasterURLIdentifier( + project_id='test-project', + region='test-region', + cluster_name='test-cluster') + cluster_manager = DataprocClusterManager(cluster_metadata) + with self.assertRaises(MockException): + cluster_manager.get_staging_location(cluster_metadata) + + @patch( + 'apache_beam.runners.interactive.dataproc.dataproc_cluster_manager.' + 'DataprocClusterManager.get_cluster_details', + return_value=MockProperty( + 'config', + MockProperty( + 'endpoint_config', + MockProperty( + 'http_ports', + {'YARN ResourceManager': 'test-resource-manager/yarn/'})))) + def test_parse_master_url_and_dashboard(self, mock_cluster_details): + """ + Tests that parse_master_url_and_dashboard properly parses the input + string and produces a mock master_url and mock dashboard link. + """ + cluster_metadata = MasterURLIdentifier( + project_id='test-project', region='test-region') + cluster_manager = DataprocClusterManager(cluster_metadata) + line = 'test-line Found Web Interface test-master-url' \ + ' of application \'test-app-id\'.\n' + master_url, dashboard = cluster_manager.parse_master_url_and_dashboard( + cluster_metadata, line) + self.assertEqual('test-master-url', master_url) + self.assertEqual( + 'test-resource-manager/gateway/default/yarn/proxy/test-app-id/', + dashboard) + + @patch( + 'google.cloud.dataproc_v1.ClusterControllerClient.get_cluster', + side_effect=MockException(403)) + def test_get_cluster_details_permission_denied(self, mock_cluster_client): + """ + Tests that an exception is thrown when a user is trying to get information + for a project without sufficient permissions to do so. + """ + cluster_metadata = MasterURLIdentifier( + project_id='test-project', region='test-region') + cluster_manager = DataprocClusterManager(cluster_metadata) + from apache_beam.runners.interactive.dataproc.dataproc_cluster_manager import _LOGGER + with self.assertLogs( + _LOGGER, + level='ERROR') as context_manager, self.assertRaises(ValueError): + cluster_manager.get_cluster_details(cluster_metadata) + self.assertTrue( + 'Due to insufficient project permissions' in + context_manager.output[0]) + + @patch( + 'google.cloud.dataproc_v1.ClusterControllerClient.get_cluster', + side_effect=MockException(404)) + def test_get_cluster_details_does_not_exist(self, mock_cluster_client): + """ + Tests that an exception is thrown when cleanup attempts to get information + for a cluster that does not exist. + """ + cluster_metadata = MasterURLIdentifier( + project_id='test-project', region='test-region') + cluster_manager = DataprocClusterManager(cluster_metadata) + from apache_beam.runners.interactive.dataproc.dataproc_cluster_manager import _LOGGER + with self.assertLogs( + _LOGGER, + level='ERROR') as context_manager, self.assertRaises(ValueError): + cluster_manager.get_cluster_details(cluster_metadata) + self.assertTrue('Cluster does not exist' in context_manager.output[0]) + + @patch( + 'google.cloud.dataproc_v1.ClusterControllerClient.get_cluster', + side_effect=MockException()) + def test_get_cluster_details_other_exception(self, mock_cluster_client): + """ + Tests that an exception is thrown when the exception is not handled by + any other case under get_cluster_details. + """ + cluster_metadata = MasterURLIdentifier( + project_id='test-project', region='test-region') + cluster_manager = DataprocClusterManager(cluster_metadata) + from apache_beam.runners.interactive.dataproc.dataproc_cluster_manager import _LOGGER + with self.assertLogs( + _LOGGER, + level='ERROR') as context_manager, self.assertRaises(MockException): + cluster_manager.get_cluster_details(cluster_metadata) + self.assertTrue( + 'Failed to get information for cluster' in context_manager.output[0]) + + @patch( + 'apache_beam.runners.interactive.dataproc.dataproc_cluster_manager.' + 'DataprocClusterManager.parse_master_url_and_dashboard', + return_value=('test-master-url', 'test-dashboard-link')) + def test_get_master_url_and_dashboard(self, mock_parse_method): + """ + Tests that get_master_url_and_dashboard detect the line containing the + unique substring which identifies the location of the master_url and + application id of the Flink master. + """ + cluster_metadata = MasterURLIdentifier( + project_id='test-project', region='test-region') + cluster_manager = DataprocClusterManager(cluster_metadata) + cluster_manager._fs = MockFileSystem() + master_url, dashboard = cluster_manager.get_master_url_and_dashboard( + cluster_metadata, + 'test-staging-bucket' + ) + self.assertEqual(master_url, 'test-master-url') + self.assertEqual(dashboard, 'test-dashboard-link') + if __name__ == '__main__': unittest.main() diff --git a/sdks/python/apache_beam/runners/interactive/interactive_beam.py b/sdks/python/apache_beam/runners/interactive/interactive_beam.py index 5eab4893985a..6098b07ab09c 100644 --- a/sdks/python/apache_beam/runners/interactive/interactive_beam.py +++ b/sdks/python/apache_beam/runners/interactive/interactive_beam.py @@ -364,20 +364,23 @@ def __init__(self) -> None: # pipelines that use the corresponding master_url. self.master_urls_to_pipelines: DefaultDict[ str, List[beam.Pipeline]] = defaultdict(list) + # self.master_urls_to_dashboards map string master_urls to the + # corresponding Apache Flink dashboards. + self.master_urls_to_dashboards: Dict[str, str] = {} def describe(self, pipeline: Optional[beam.Pipeline] = None) -> dict: """Returns a description of the cluster associated to the given pipeline. If no pipeline is given then this returns a dictionary of descriptions for - all pipelines. + all pipelines, mapped to by id. """ description = { - ie.current_env().pipeline_id_to_pipeline(pid): dcm.describe() + pid: dcm.describe() for pid, dcm in self.dataproc_cluster_managers.items() } if pipeline: - return description.get(pipeline, None) + return description.get(str(id(pipeline)), None) return description def cleanup( @@ -419,6 +422,7 @@ def cleanup( cluster_manager.cleanup() self.master_urls.pop(master_url, None) self.master_urls_to_pipelines.pop(master_url, None) + self.master_urls_to_dashboards.pop(master_url, None) self.dataproc_cluster_managers.pop(str(id(pipeline)), None) else: cluster_manager_identifiers = set() @@ -429,6 +433,7 @@ def cleanup( self.dataproc_cluster_managers.clear() self.master_urls.clear() self.master_urls_to_pipelines.clear() + self.master_urls_to_dashboards.clear() # Users can set options to guide how Interactive Beam works. @@ -452,7 +457,7 @@ def cleanup( # Examples: # ib.clusters.describe(p) # Check the docstrings for detailed usages. -# TODO(victorhc): Implement all functionality for Clusters() +# TODO(victorhc): Resolve connection issue and add a working example # clusters = Clusters() diff --git a/sdks/python/apache_beam/runners/interactive/interactive_beam_test.py b/sdks/python/apache_beam/runners/interactive/interactive_beam_test.py index b14848c0afc6..4541463c1d9f 100644 --- a/sdks/python/apache_beam/runners/interactive/interactive_beam_test.py +++ b/sdks/python/apache_beam/runners/interactive/interactive_beam_test.py @@ -306,15 +306,16 @@ def test_clusters_describe(self): region=region, )) cluster_metadata = MasterURLIdentifier(project_id=project, region=region) - clusters.dataproc_cluster_managers[p] = DataprocClusterManager( - cluster_metadata) - self.assertEqual('test-project', clusters.describe()[None] \ - ['cluster_metadata'].project_id) + clusters.dataproc_cluster_managers[str( + id(p))] = DataprocClusterManager(cluster_metadata) + self.assertEqual( + 'test-project', + clusters.describe()[str(id(p))]['cluster_metadata'].project_id) @patch( 'apache_beam.runners.interactive.dataproc.dataproc_cluster_manager.' - 'DataprocClusterManager.get_master_url', - return_value='test-master-url') + 'DataprocClusterManager.get_master_url_and_dashboard', + return_value=('test-master-url', None)) @patch( 'apache_beam.runners.interactive.dataproc.dataproc_cluster_manager.' 'DataprocClusterManager.cleanup', @@ -350,8 +351,8 @@ def test_clusters_cleanup_forcefully(self, mock_cleanup, mock_master_url): @patch( 'apache_beam.runners.interactive.dataproc.dataproc_cluster_manager.' - 'DataprocClusterManager.get_master_url', - return_value='test-master-url') + 'DataprocClusterManager.get_master_url_and_dashboard', + return_value=('test-master-url', None)) def test_clusters_cleanup_skip_on_duplicate(self, mock_master_url): clusters = ib.Clusters() project = 'test-project' diff --git a/sdks/python/apache_beam/runners/interactive/interactive_runner.py b/sdks/python/apache_beam/runners/interactive/interactive_runner.py index d76d68961af0..a384be9bdebe 100644 --- a/sdks/python/apache_beam/runners/interactive/interactive_runner.py +++ b/sdks/python/apache_beam/runners/interactive/interactive_runner.py @@ -27,6 +27,7 @@ import apache_beam as beam from apache_beam import runners +from apache_beam.options.pipeline_options import FlinkRunnerOptions from apache_beam.options.pipeline_options import GoogleCloudOptions from apache_beam.pipeline import PipelineVisitor from apache_beam.runners.direct import direct_runner @@ -137,6 +138,14 @@ def run_pipeline(self, pipeline, options): watch_sources(pipeline) user_pipeline = ie.current_env().user_pipeline(pipeline) + if user_pipeline: + # When the underlying_runner is a FlinkRunner instance, create a + # corresponding DataprocClusterManager for it if no flink_master_url + # is provided. + master_url = self._get_dataproc_cluster_master_url_if_applicable( + user_pipeline) + if master_url: + options.view_as(FlinkRunnerOptions).flink_master = master_url pipeline_instrument = inst.build_pipeline_instrument(pipeline, options) # The user_pipeline analyzed might be None if the pipeline given has nothing @@ -169,11 +178,6 @@ def exception_handler(e): ie.current_env().set_test_stream_service_controller( user_pipeline, test_stream_service) - # When the underlying_runner is a FlinkRunner instance, create a - # corresponding DataprocClusterManager for it if no flink_master_url - # is provided. - self._create_dataproc_cluster_if_applicable(user_pipeline) - pipeline_to_execute = beam.pipeline.Pipeline.from_runner_api( pipeline_instrument.instrumented_pipeline_proto(), self._underlying_runner, @@ -220,7 +224,8 @@ def visit_transform(self, transform_node): # TODO(victorhc): Move this method somewhere else if performance is impacted # by generating a cluster during runtime. - def _create_dataproc_cluster_if_applicable(self, user_pipeline): + def _get_dataproc_cluster_master_url_if_applicable( + self, user_pipeline: beam.Pipeline) -> str: """ Creates a Dataproc cluster if the provided user_pipeline is running FlinkRunner and no flink_master_url was provided as an option. A cluster is not created when a flink_master_url is detected. @@ -241,7 +246,6 @@ def _create_dataproc_cluster_if_applicable(self, user_pipeline): ]) """ from apache_beam.runners.portability.flink_runner import FlinkRunner - from apache_beam.options.pipeline_options import FlinkRunnerOptions flink_master = user_pipeline.options.view_as( FlinkRunnerOptions).flink_master clusters = ie.current_env().clusters @@ -264,7 +268,7 @@ def _create_dataproc_cluster_if_applicable(self, user_pipeline): cluster_metadata = MasterURLIdentifier( project_id=project_id, region=region, cluster_name=cluster_name) else: - cluster_metadata = clusters.master_urls.inverse.get(flink_master, None) + cluster_metadata = clusters.master_urls.get(flink_master, None) # else noop, no need to log anything because we allow a master_url # (not managed by us) provided by the user. if cluster_metadata: @@ -278,6 +282,9 @@ def _create_dataproc_cluster_if_applicable(self, user_pipeline): id(user_pipeline))] = cluster_manager clusters.master_urls_to_pipelines[cluster_manager.master_url].append( str(id(user_pipeline))) + clusters.master_urls_to_dashboards[ + cluster_manager.master_url] = cluster_manager.dashboard + return cluster_manager.master_url class PipelineResult(beam.runners.runner.PipelineResult): diff --git a/sdks/python/apache_beam/runners/interactive/interactive_runner_test.py b/sdks/python/apache_beam/runners/interactive/interactive_runner_test.py index de5f1a5464d9..47dedd6578c6 100644 --- a/sdks/python/apache_beam/runners/interactive/interactive_runner_test.py +++ b/sdks/python/apache_beam/runners/interactive/interactive_runner_test.py @@ -37,6 +37,7 @@ from apache_beam.runners.interactive import interactive_beam as ib from apache_beam.runners.interactive import interactive_environment as ie from apache_beam.runners.interactive import interactive_runner +from apache_beam.runners.interactive.dataproc.dataproc_cluster_manager import MasterURLIdentifier from apache_beam.runners.interactive.testing.mock_ipython import mock_get_ipython from apache_beam.testing.test_stream import TestStream from apache_beam.transforms.window import GlobalWindow @@ -491,7 +492,7 @@ def enter_composite_transform( 'apache_beam.runners.interactive.dataproc.dataproc_cluster_manager.' 'DataprocClusterManager.create_flink_cluster', return_value=None) - def test_create_dataproc_cluster_no_flink_master_or_master_url( + def test_get_master_url_no_flink_master_or_provided_master_url( self, mock_create_cluster): from apache_beam.runners.portability.flink_runner import FlinkRunner runner = interactive_runner.InteractiveRunner( @@ -501,8 +502,7 @@ def test_create_dataproc_cluster_no_flink_master_or_master_url( project='test-project', region='test-region', )) - runner._create_dataproc_cluster_if_applicable(p) - ie.current_env()._tracked_user_pipelines.add_user_pipeline(p) + runner._get_dataproc_cluster_master_url_if_applicable(p) self.assertEqual( ie.current_env().clusters.describe(p)['cluster_metadata'].project_id, 'test-project') @@ -511,14 +511,41 @@ def test_create_dataproc_cluster_no_flink_master_or_master_url( @unittest.skipIf( not ie.current_env().is_interactive_ready, '[interactive] dependency is not installed.') - def test_create_dataproc_cluster_flink_master_provided(self): + def test_get_master_url_no_flink_master_and_master_url_exists(self): + from apache_beam.runners.portability.flink_runner import FlinkRunner + runner = interactive_runner.InteractiveRunner( + underlying_runner=FlinkRunner()) + p = beam.Pipeline( + options=PipelineOptions( + project='test-project', + region='test-region', + )) + cluster_name = ie.current_env().clusters.default_cluster_name + cluster_metadata = MasterURLIdentifier( + project_id='test-project', + region='test-region', + cluster_name=cluster_name) + ie.current_env().clusters.master_urls['test-url'] = cluster_metadata + ie.current_env( + ).clusters.master_urls_to_dashboards['test-url'] = 'test-dashboard' + flink_master = runner._get_dataproc_cluster_master_url_if_applicable(p) + self.assertEqual( + ie.current_env().clusters.describe(p)['cluster_metadata'].project_id, + 'test-project') + self.assertEqual( + flink_master, ie.current_env().clusters.describe(p)['master_url']) + + @unittest.skipIf( + not ie.current_env().is_interactive_ready, + '[interactive] dependency is not installed.') + def test_get_master_url_flink_master_provided(self): runner = interactive_runner.InteractiveRunner() from apache_beam.runners.portability.flink_runner import FlinkRunner p = beam.Pipeline( interactive_runner.InteractiveRunner(underlying_runner=FlinkRunner()), options=PipelineOptions( flink_master='--flink_master=example.internal:1')) - runner._create_dataproc_cluster_if_applicable(p) + runner._get_dataproc_cluster_master_url_if_applicable(p) self.assertEqual(ie.current_env().clusters.describe(), {}) ie.current_env().clusters = ib.Clusters() diff --git a/sdks/python/apache_beam/runners/interactive/utils_test.py b/sdks/python/apache_beam/runners/interactive/utils_test.py index 26734d409819..99847914ad93 100644 --- a/sdks/python/apache_beam/runners/interactive/utils_test.py +++ b/sdks/python/apache_beam/runners/interactive/utils_test.py @@ -55,9 +55,6 @@ class MockBuckets(): - def __init__(self): - pass - def Get(self, path): if path == 'test-bucket-not-found': raise HttpNotFoundError({'status': 404}, {}, '') From 30379eb7ffcaf757b55009c7706a4d171c88902e Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Wed, 2 Mar 2022 14:11:06 -0500 Subject: [PATCH 15/68] [BEAM-13925] Turn pr bot on for go prs (#16984) --- .github/workflows/pr-bot-new-prs.yml | 39 ++++++++++++++++++++++++ .github/workflows/pr-bot-pr-updates.yml | 40 +++++++++++++++++++++++++ 2 files changed, 79 insertions(+) create mode 100644 .github/workflows/pr-bot-new-prs.yml create mode 100644 .github/workflows/pr-bot-pr-updates.yml diff --git a/.github/workflows/pr-bot-new-prs.yml b/.github/workflows/pr-bot-new-prs.yml new file mode 100644 index 000000000000..297042bb0ad8 --- /dev/null +++ b/.github/workflows/pr-bot-new-prs.yml @@ -0,0 +1,39 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: pr-bot-new-prs + +# Run every 30 minutes +on: + schedule: + - cron: '30 * * * *' + workflow_dispatch: + +jobs: + process-prs: + # Don't run on forks + if: github.repository == 'apache/beam' + runs-on: ubuntu-latest + + steps: + - uses: actions/checkout@v2 + - run: npm install + working-directory: 'scripts/ci/pr-bot' + + # Runs a set of commands using the runners shell + - run: npm run processNewPrs + env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + working-directory: 'scripts/ci/pr-bot' diff --git a/.github/workflows/pr-bot-pr-updates.yml b/.github/workflows/pr-bot-pr-updates.yml new file mode 100644 index 000000000000..e6f8610ef0c6 --- /dev/null +++ b/.github/workflows/pr-bot-pr-updates.yml @@ -0,0 +1,40 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: pr-bot-pr-updates + +on: + pull_request_review: + pull_request_review_comment: + pull_request_target: + types: ["synchronize"] # Synchronize is the action that runs after pushes to the user branch + issue_comment: + +jobs: + process-pr-update: + # Don't run on forks + if: github.repository == 'apache/beam' + runs-on: ubuntu-latest + + steps: + - uses: actions/checkout@v2 + - run: npm install + working-directory: 'scripts/ci/pr-bot' + + # Runs a set of commands using the runners shell + - run: npm run processPrUpdate + env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + working-directory: 'scripts/ci/pr-bot' From 1246d085cd19b8116208deb484614432c9802e7b Mon Sep 17 00:00:00 2001 From: Masato Nakamura Date: Thu, 3 Mar 2022 06:37:05 +0900 Subject: [PATCH 16/68] [BEAM-13964] Bump kotlin to 1.6.x (#16882) * [BEAM-13964] Bump kotlin to 1.6.x * [BEAM-13964] Bump kotlin to 1.6.x * [BEAM-13964] fix warnings in Kotlin compilation --- examples/kotlin/build.gradle | 4 ++-- .../beam/examples/kotlin/common/WriteOneFilePerWindow.kt | 5 ++--- .../beam/examples/kotlin/cookbook/DistinctExample.kt | 2 -- .../org/apache/beam/examples/kotlin/snippets/Snippets.kt | 8 ++++---- 4 files changed, 8 insertions(+), 11 deletions(-) diff --git a/examples/kotlin/build.gradle b/examples/kotlin/build.gradle index 3abea64b03cb..0aa3dc257b09 100644 --- a/examples/kotlin/build.gradle +++ b/examples/kotlin/build.gradle @@ -19,7 +19,7 @@ import groovy.json.JsonOutput plugins { id 'org.apache.beam.module' - id 'org.jetbrains.kotlin.jvm' version '1.3.72' + id 'org.jetbrains.kotlin.jvm' version '1.6.10' } applyJavaNature(exportJavadoc: false, automaticModuleName: 'org.apache.beam.examples.kotlin') @@ -46,7 +46,7 @@ configurations.sparkRunnerPreCommit { exclude group: "org.slf4j", module: "slf4j-jdk14" } -def kotlin_version = "1.4.32" +def kotlin_version = "1.6.10" dependencies { implementation enforcedPlatform(library.java.google_cloud_platform_libraries_bom) diff --git a/examples/kotlin/src/main/java/org/apache/beam/examples/kotlin/common/WriteOneFilePerWindow.kt b/examples/kotlin/src/main/java/org/apache/beam/examples/kotlin/common/WriteOneFilePerWindow.kt index f06b5c82d521..601fe2191d0b 100644 --- a/examples/kotlin/src/main/java/org/apache/beam/examples/kotlin/common/WriteOneFilePerWindow.kt +++ b/examples/kotlin/src/main/java/org/apache/beam/examples/kotlin/common/WriteOneFilePerWindow.kt @@ -45,13 +45,12 @@ class WriteOneFilePerWindow(private val filenamePrefix: String, private val numS override fun expand(input: PCollection): PDone { val resource = FileBasedSink.convertToFileResourceIfPossible(filenamePrefix) - var write = TextIO.write() + val write = TextIO.write() .to(PerWindowFiles(resource)) .withTempDirectory(resource.currentDirectory) .withWindowedWrites() - write = numShards?.let { write.withNumShards(it) } ?: write - return input.apply(write) + return input.apply(numShards?.let { write.withNumShards(it) } ?: write) } /** diff --git a/examples/kotlin/src/main/java/org/apache/beam/examples/kotlin/cookbook/DistinctExample.kt b/examples/kotlin/src/main/java/org/apache/beam/examples/kotlin/cookbook/DistinctExample.kt index 65dbbd867bb4..f743dd1a7f70 100644 --- a/examples/kotlin/src/main/java/org/apache/beam/examples/kotlin/cookbook/DistinctExample.kt +++ b/examples/kotlin/src/main/java/org/apache/beam/examples/kotlin/cookbook/DistinctExample.kt @@ -71,8 +71,6 @@ object DistinctExample { override fun create(options: PipelineOptions): String { options.tempLocation.let { return GcsPath.fromUri(it).resolve("deduped.txt").toString() - } ?: run { - throw IllegalArgumentException("Must specify --output or --tempLocation") } } } diff --git a/examples/kotlin/src/main/java/org/apache/beam/examples/kotlin/snippets/Snippets.kt b/examples/kotlin/src/main/java/org/apache/beam/examples/kotlin/snippets/Snippets.kt index 62ef705f569b..3fdc71d0b58a 100644 --- a/examples/kotlin/src/main/java/org/apache/beam/examples/kotlin/snippets/Snippets.kt +++ b/examples/kotlin/src/main/java/org/apache/beam/examples/kotlin/snippets/Snippets.kt @@ -111,7 +111,7 @@ object Snippets { .apply>( MapElements.into(TypeDescriptors.doubles()) .via(SerializableFunction { - it["max_temperature"] as Double + it["max_temperature"] as Double? }) ) // [END BigQueryReadTable] @@ -121,7 +121,7 @@ object Snippets { val tableSpec = "clouddataflow-readonly:samples.weather_stations" // [START BigQueryReadFunction] val maxTemperatures = pipeline.apply( - BigQueryIO.read { it.record["max_temperature"] as Double } + BigQueryIO.read { it.record["max_temperature"] as Double? } .from(tableSpec) .withCoder(DoubleCoder.of())) // [END BigQueryReadFunction] @@ -130,7 +130,7 @@ object Snippets { run { // [START BigQueryReadQuery] val maxTemperatures = pipeline.apply( - BigQueryIO.read { it.record["max_temperature"] as Double } + BigQueryIO.read { it.record["max_temperature"] as Double? } .fromQuery( "SELECT max_temperature FROM [clouddataflow-readonly:samples.weather_stations]") .withCoder(DoubleCoder.of())) @@ -140,7 +140,7 @@ object Snippets { run { // [START BigQueryReadQueryStdSQL] val maxTemperatures = pipeline.apply( - BigQueryIO.read { it.record["max_temperature"] as Double } + BigQueryIO.read { it.record["max_temperature"] as Double? } .fromQuery( "SELECT max_temperature FROM `clouddataflow-readonly.samples.weather_stations`") .usingStandardSql() From c709e29e3b9fb72f3de3dd82d5195923f71c1ad4 Mon Sep 17 00:00:00 2001 From: Pablo Estrada Date: Wed, 2 Mar 2022 14:20:21 -0500 Subject: [PATCH 17/68] Skipping flaky sad-path tests for Spanner changestreams --- .../spanner/changestreams/SpannerChangeStreamErrorTest.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/SpannerChangeStreamErrorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/SpannerChangeStreamErrorTest.java index 34d7cd4a91a7..050993391b78 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/SpannerChangeStreamErrorTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/SpannerChangeStreamErrorTest.java @@ -62,6 +62,7 @@ import org.hamcrest.Matchers; import org.junit.After; import org.junit.Before; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -128,6 +129,7 @@ public void testResourceExhaustedDoesNotRetry() { } @Test + @Ignore public void testUnavailableExceptionRetries() { mockSpannerService.setExecuteStreamingSqlExecutionTime( SimulatedExecutionTime.ofExceptions( @@ -206,6 +208,7 @@ public void testUnknownExceptionDoesNotRetry() { } @Test + @Ignore public void testInvalidRecordReceived() { final Timestamp now = Timestamp.now(); final Timestamp after3Seconds = From 13a06f49851dbf95cd4f88ef5762c4e0f779c04a Mon Sep 17 00:00:00 2001 From: reuvenlax Date: Wed, 2 Mar 2022 14:17:36 -0800 Subject: [PATCH 18/68] Merge pull request #16906: [BEAM-13974] Handle idle Storage Api streams --- .../StorageApiFlushAndFinalizeDoFn.java | 10 ++ .../StorageApiWritesShardedRecords.java | 94 +++++++++++++++---- 2 files changed, 87 insertions(+), 17 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFlushAndFinalizeDoFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFlushAndFinalizeDoFn.java index 1c3686a08096..719e1ddb97b7 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFlushAndFinalizeDoFn.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFlushAndFinalizeDoFn.java @@ -172,6 +172,9 @@ public void process(PipelineOptions pipelineOptions, @Element KV { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java index 15982a3ea46f..9e966310b5ce 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java @@ -22,6 +22,7 @@ import com.google.api.core.ApiFuture; import com.google.api.services.bigquery.model.TableSchema; import com.google.cloud.bigquery.storage.v1.AppendRowsResponse; +import com.google.cloud.bigquery.storage.v1.Exceptions.StreamFinalizedException; import com.google.cloud.bigquery.storage.v1.ProtoRows; import com.google.cloud.bigquery.storage.v1.WriteStream.Type; import com.google.protobuf.Descriptors.Descriptor; @@ -56,6 +57,10 @@ import org.apache.beam.sdk.schemas.SchemaRegistry; import org.apache.beam.sdk.state.StateSpec; import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.TimeDomain; +import org.apache.beam.sdk.state.Timer; +import org.apache.beam.sdk.state.TimerSpec; +import org.apache.beam.sdk.state.TimerSpecs; import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.DoFn; @@ -63,6 +68,7 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.Repeatedly; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.ShardedKey; @@ -92,12 +98,14 @@ public class StorageApiWritesShardedRecords extends PTransform< PCollection, Iterable>>, PCollection> { private static final Logger LOG = LoggerFactory.getLogger(StorageApiWritesShardedRecords.class); + private static final Duration DEFAULT_STREAM_IDLE_TIME = Duration.standardHours(1); private final StorageApiDynamicDestinations dynamicDestinations; private final CreateDisposition createDisposition; private final String kmsKey; private final BigQueryServices bqServices; private final Coder destinationCoder; + private final Duration streamIdleTime = DEFAULT_STREAM_IDLE_TIME; private static final ExecutorService closeWriterExecutor = Executors.newCachedThreadPool(); private static final Cache APPEND_CLIENTS = @@ -148,7 +156,7 @@ public PCollection expand( PCollection> written = input.apply( "Write Records", - ParDo.of(new WriteRecordsDoFn(operationName)) + ParDo.of(new WriteRecordsDoFn(operationName, streamIdleTime)) .withSideInputs(dynamicDestinations.getSideInputs())); SchemaCoder operationCoder; @@ -185,6 +193,8 @@ class WriteRecordsDoFn Metrics.counter(WriteRecordsDoFn.class, "recordsAppended"); private final Counter streamsCreated = Metrics.counter(WriteRecordsDoFn.class, "streamsCreated"); + private final Counter streamsIdle = + Metrics.counter(WriteRecordsDoFn.class, "idleStreamsFinalized"); private final Counter appendFailures = Metrics.counter(WriteRecordsDoFn.class, "appendFailures"); private final Counter appendOffsetFailures = @@ -212,8 +222,14 @@ class WriteRecordsDoFn @StateId("streamOffset") private final StateSpec> streamOffsetSpec = StateSpecs.value(); - public WriteRecordsDoFn(String operationName) { + @TimerId("idleTimer") + private final TimerSpec idleTimer = TimerSpecs.timer(TimeDomain.PROCESSING_TIME); + + private final Duration streamIdleTime; + + public WriteRecordsDoFn(String operationName, Duration streamIdleTime) { this.messageConverters = new TwoLevelMessageConverterCache<>(operationName); + this.streamIdleTime = streamIdleTime; } @StartBundle @@ -229,6 +245,7 @@ String getOrCreateStream( String tableId, ValueState streamName, ValueState streamOffset, + Timer streamIdleTimer, DatasetService datasetService) throws IOException, InterruptedException { String stream = streamName.read(); @@ -239,6 +256,12 @@ String getOrCreateStream( streamOffset.write(0L); streamsCreated.inc(); } + // Reset the idle timer. + streamIdleTimer + .offset(streamIdleTime) + .withOutputTimestamp(GlobalWindow.INSTANCE.maxTimestamp()) + .setRelative(); + return stream; } @@ -270,6 +293,7 @@ public void process( @Element KV, Iterable> element, final @AlwaysFetched @StateId("streamName") ValueState streamName, final @AlwaysFetched @StateId("streamOffset") ValueState streamOffset, + @TimerId("idleTimer") Timer idleTimer, final OutputReceiver> o) throws Exception { dynamicDestinations.setSideInputAccessorFromProcessContext(c); @@ -343,7 +367,8 @@ public String toString() { // Clear the stream name, forcing a new one to be created. streamName.write(""); } - String stream = getOrCreateStream(tableId, streamName, streamOffset, datasetService); + String stream = + getOrCreateStream(tableId, streamName, streamOffset, idleTimer, datasetService); StreamAppendClient appendClient = APPEND_CLIENTS.get( stream, () -> datasetService.getStreamAppendClient(stream, descriptor)); @@ -398,13 +423,32 @@ public String toString() { // The first context is always the one that fails. AppendRowsContext failedContext = Preconditions.checkNotNull(Iterables.getFirst(failedContexts, null)); - Status.Code statusCode = Status.fromThrowable(failedContext.getError()).getCode(); // Invalidate the StreamWriter and force a new one to be created. LOG.error( "Got error " + failedContext.getError() + " closing " + failedContext.streamName); clearClients.accept(contexts); appendFailures.inc(); - if (statusCode.equals(Code.OUT_OF_RANGE) || statusCode.equals(Code.ALREADY_EXISTS)) { + + boolean explicitStreamFinalized = + failedContext.getError() instanceof StreamFinalizedException; + Status.Code statusCode = Status.fromThrowable(failedContext.getError()).getCode(); + // This means that the offset we have stored does not match the current end of + // the stream in the Storage API. Usually this happens because a crash or a bundle + // failure + // happened after an append but before the worker could checkpoint it's + // state. The records that were appended in a failed bundle will be retried, + // meaning that the unflushed tail of the stream must be discarded to prevent + // duplicates. + boolean offsetMismatch = + statusCode.equals(Code.OUT_OF_RANGE) || statusCode.equals(Code.ALREADY_EXISTS); + // This implies that the stream doesn't exist or has already been finalized. In this + // case we have no choice but to create a new stream. + boolean streamDoesNotExist = + explicitStreamFinalized + || statusCode.equals(Code.INVALID_ARGUMENT) + || statusCode.equals(Code.NOT_FOUND) + || statusCode.equals(Code.FAILED_PRECONDITION); + if (offsetMismatch || streamDoesNotExist) { appendOffsetFailures.inc(); LOG.warn( "Append to " @@ -412,14 +456,6 @@ public String toString() { + " failed with " + failedContext.getError() + " Will retry with a new stream"); - // This means that the offset we have stored does not match the current end of - // the stream in the Storage API. Usually this happens because a crash or a bundle - // failure - // happened after an append but before the worker could checkpoint it's - // state. The records that were appended in a failed bundle will be retried, - // meaning that the unflushed tail of the stream must be discarded to prevent - // duplicates. - // Finalize the stream and clear streamName so a new stream will be created. o.output( KV.of(failedContext.streamName, new Operation(failedContext.offset - 1, true))); @@ -466,24 +502,48 @@ public String toString() { java.time.Duration timeElapsed = java.time.Duration.between(now, Instant.now()); appendLatencyDistribution.update(timeElapsed.toMillis()); + idleTimer + .offset(streamIdleTime) + .withOutputTimestamp(GlobalWindow.INSTANCE.maxTimestamp()) + .setRelative(); } - @OnWindowExpiration - public void onWindowExpiration( + // called by the idleTimer and window-expiration handlers. + private void finalizeStream( @AlwaysFetched @StateId("streamName") ValueState streamName, @AlwaysFetched @StateId("streamOffset") ValueState streamOffset, OutputReceiver> o) { - // Window is done - usually because the pipeline has been drained. Make sure to clean up - // streams so that they are not leaked. String stream = MoreObjects.firstNonNull(streamName.read(), null); if (!Strings.isNullOrEmpty(stream)) { // Finalize the stream long nextOffset = MoreObjects.firstNonNull(streamOffset.read(), 0L); o.output(KV.of(stream, new Operation(nextOffset - 1, true))); + streamName.clear(); + streamOffset.clear(); // Make sure that the stream object is closed. APPEND_CLIENTS.invalidate(stream); } } + + @OnTimer("idleTimer") + public void onTimer( + @AlwaysFetched @StateId("streamName") ValueState streamName, + @AlwaysFetched @StateId("streamOffset") ValueState streamOffset, + OutputReceiver> o) { + // Stream is idle - clear it. + finalizeStream(streamName, streamOffset, o); + streamsIdle.inc(); + } + + @OnWindowExpiration + public void onWindowExpiration( + @AlwaysFetched @StateId("streamName") ValueState streamName, + @AlwaysFetched @StateId("streamOffset") ValueState streamOffset, + OutputReceiver> o) { + // Window is done - usually because the pipeline has been drained. Make sure to clean up + // streams so that they are not leaked. + finalizeStream(streamName, streamOffset, o); + } } } From 402356da4abf961a74ce71df45735243892a5d87 Mon Sep 17 00:00:00 2001 From: Miguel Hernandez <61989986+roger-mike@users.noreply.github.com> Date: Wed, 2 Mar 2022 16:53:47 -0600 Subject: [PATCH 19/68] Merge pull request #16562 from [BEAM-13051][D] Enable pylint warnings (no-name-in-module/no-value-for-parameter) * [BEAM-13051] Pylint no-name-in-module and no-value-for-parameter warnings enabled * [BEAM-13051] Fixed no-value-for-parameter warning for missing default values * [BEAM-13051] Fixed parameters warnings --- sdks/python/apache_beam/io/avroio_test.py | 8 +++++++- sdks/python/apache_beam/io/filesystem.py | 3 +-- .../testing/load_tests/load_test_metrics_utils.py | 9 ++++++--- 3 files changed, 14 insertions(+), 6 deletions(-) diff --git a/sdks/python/apache_beam/io/avroio_test.py b/sdks/python/apache_beam/io/avroio_test.py index 3d0db805cbf2..bcffac528a19 100644 --- a/sdks/python/apache_beam/io/avroio_test.py +++ b/sdks/python/apache_beam/io/avroio_test.py @@ -95,7 +95,13 @@ def tearDown(self): os.remove(path) self._temp_files = [] - def _write_data(self, directory, prefix, codec, count, sync_interval): + def _write_data( + self, + directory=None, + prefix=None, + codec=None, + count=None, + sync_interval=None): raise NotImplementedError def _write_pattern(self, num_files, return_filenames=False): diff --git a/sdks/python/apache_beam/io/filesystem.py b/sdks/python/apache_beam/io/filesystem.py index 92d55929309f..c854e5879722 100644 --- a/sdks/python/apache_beam/io/filesystem.py +++ b/sdks/python/apache_beam/io/filesystem.py @@ -257,8 +257,7 @@ def _read_from_internal_buffer(self, read_fn): self._read_buffer.seek(0, os.SEEK_END) # Allow future writes. return result - def read(self, num_bytes): - # type: (int) -> bytes + def read(self, num_bytes: Optional[int] = None) -> bytes: if not self._decompressor: raise ValueError('decompressor not initialized') diff --git a/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py b/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py index 7b975bbe8feb..0099d462af98 100644 --- a/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py +++ b/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py @@ -216,7 +216,7 @@ def __init__( 'InfluxDB') self.filters = filters - def publish_metrics(self, result, extra_metrics: dict): + def publish_metrics(self, result, extra_metrics: Optional[dict] = None): metric_id = uuid.uuid4().hex metrics = result.metrics().query(self.filters) @@ -227,13 +227,16 @@ def publish_metrics(self, result, extra_metrics: dict): # a list of dictionaries matching the schema. insert_dicts = self._prepare_all_metrics(metrics, metric_id) - insert_dicts += self._prepare_extra_metrics(extra_metrics, metric_id) + insert_dicts += self._prepare_extra_metrics(metric_id, extra_metrics) if len(insert_dicts) > 0: for publisher in self.publishers: publisher.publish(insert_dicts) - def _prepare_extra_metrics(self, extra_metrics: dict, metric_id: str): + def _prepare_extra_metrics( + self, metric_id: str, extra_metrics: Optional[dict] = None): ts = time.time() + if not extra_metrics: + extra_metrics = {} return [ Metric(ts, metric_id, v, label=k).as_dict() for k, v in extra_metrics.items() From a53c600dbf8a728fd33f2fdacb8161ee0a4a6eb2 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Wed, 2 Mar 2022 18:48:37 -0500 Subject: [PATCH 20/68] [BEAM-13925] A couple small pr-bot bug fixes (#16996) --- scripts/ci/pr-bot/processNewPrs.ts | 8 ++++++++ scripts/ci/pr-bot/shared/persistentState.ts | 8 +++++++- 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/scripts/ci/pr-bot/processNewPrs.ts b/scripts/ci/pr-bot/processNewPrs.ts index 411e3b5428ec..1f178900e4f5 100644 --- a/scripts/ci/pr-bot/processNewPrs.ts +++ b/scripts/ci/pr-bot/processNewPrs.ts @@ -35,6 +35,7 @@ import { CheckStatus } from "./shared/checks"; * 5) Have already been processed * 6) Have notifications stopped * 7) The pr doesn't contain the go label (temporary). TODO(damccorm) - remove this when we're ready to roll this out to everyone. + * 8) The pr happens after the date we turn on the automation. TODO(damccorm) - remove this once this has been rolled out for a while. * unless we're supposed to remind the user after tests pass * (in which case that's all we need to do). */ @@ -45,6 +46,10 @@ function needsProcessed(pull: any, prState: typeof Pr): boolean { ); return false; } + let firstPrToProcess = new Date(2022, 3, 2, 20); + if (new Date(pull.created_at) < firstPrToProcess) { + return false; + } if (prState.remindAfterTestsPass && prState.remindAfterTestsPass.length > 0) { return true; } @@ -164,6 +169,9 @@ async function processPull( } if (!checkState.succeeded) { + if (!checkState.completed) { + return; + } return await notifyChecksFailed(pull, stateClient, prState); } prState.commentedAboutFailingChecks = false; diff --git a/scripts/ci/pr-bot/shared/persistentState.ts b/scripts/ci/pr-bot/shared/persistentState.ts index f7fade16f9b9..e440d2ae3d8d 100644 --- a/scripts/ci/pr-bot/shared/persistentState.ts +++ b/scripts/ci/pr-bot/shared/persistentState.ts @@ -32,7 +32,13 @@ function getReviewersForLabelFileName(label) { } async function commitStateToRepo() { - await exec.exec("git pull origin pr-bot-state"); + try { + await exec.exec("git pull origin pr-bot-state"); + } catch (err) { + console.log( + `Unable to get most recent repo contents, commit may fail: ${err}` + ); + } await exec.exec("git add state/*"); await exec.exec(`git commit -m "Updating config from bot" --allow-empty`); await exec.exec("git push origin pr-bot-state"); From 3748dd5cf3dc833fe7f6dd64989b3182f1dae648 Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Wed, 2 Mar 2022 19:14:21 -0500 Subject: [PATCH 21/68] [BEAM-14029] Add getter, setter for target maven repo (#16995) --- .../runtime/xlangx/expansionx/download.go | 32 ++++++++- .../xlangx/expansionx/download_test.go | 66 +++++++++++++++++++ 2 files changed, 97 insertions(+), 1 deletion(-) diff --git a/sdks/go/pkg/beam/core/runtime/xlangx/expansionx/download.go b/sdks/go/pkg/beam/core/runtime/xlangx/expansionx/download.go index 0f794668d87e..59a1d575248b 100644 --- a/sdks/go/pkg/beam/core/runtime/xlangx/expansionx/download.go +++ b/sdks/go/pkg/beam/core/runtime/xlangx/expansionx/download.go @@ -50,6 +50,24 @@ func newJarGetter() *jarGetter { return &jarGetter{repository: apacheRepository, groupID: beamGroupID, jarCache: cacheDir} } +// GetRepositoryURL returns the current target URL for the defaultJarGetter, +// indicating what repository will be connected to when getting a Beam JAR. +func GetDefaultRepositoryURL() string { + return defaultJarGetter.getRepositoryURL() +} + +// SetRepositoryURL updates the target URL for the defaultJarGetter, changing +// which Maven repository will be connected to when getting a Beam JAR. Also +// validates that it has been passed a URL and returns an error if not. +// +// When changing the target repository, make sure that the value is the prefix +// up to "org/apache/beam" and that the organization of the repository matches +// that of the default from that point on to ensure that the conversion of the +// Gradle target to the JAR name is correct. +func SetDefaultRepositoryURL(repoURL string) error { + return defaultJarGetter.setRepositoryURL(repoURL) +} + // GetBeamJar checks a temporary directory for the desired Beam JAR, downloads the // appropriate JAR from Maven if not present, then returns the file path to the // JAR. @@ -57,6 +75,18 @@ func GetBeamJar(gradleTarget, version string) (string, error) { return defaultJarGetter.getJar(gradleTarget, version) } +func (j *jarGetter) getRepositoryURL() string { + return string(j.repository) +} + +func (j *jarGetter) setRepositoryURL(repoURL string) error { + if !strings.HasPrefix(repoURL, "http") { + return fmt.Errorf("repo URL %v does not have an http or https prefix", repoURL) + } + j.repository = url(strings.TrimSuffix(repoURL, "/")) + return nil +} + func (j *jarGetter) getJar(gradleTarget, version string) (string, error) { strippedTarget := dropEndOfGradleTarget(gradleTarget) fullURL, jarName := j.getURLForBeamJar(strippedTarget, version) @@ -84,7 +114,7 @@ func (j *jarGetter) getJar(gradleTarget, version string) (string, error) { defer resp.Body.Close() if resp.StatusCode != 200 { - return "", fmt.Errorf("received non 200 response code, got %v", resp.StatusCode) + return "", fmt.Errorf("failed to connect to %v: received non 200 response code, got %v", fullURL, resp.StatusCode) } file, err := os.Create(jarPath) diff --git a/sdks/go/pkg/beam/core/runtime/xlangx/expansionx/download_test.go b/sdks/go/pkg/beam/core/runtime/xlangx/expansionx/download_test.go index 1d13faca6bbf..b20977b5e704 100644 --- a/sdks/go/pkg/beam/core/runtime/xlangx/expansionx/download_test.go +++ b/sdks/go/pkg/beam/core/runtime/xlangx/expansionx/download_test.go @@ -23,6 +23,72 @@ import ( "testing" ) +func TestGetAndSetRepositoryURL(t *testing.T) { + tests := []struct { + name string + newRepo string + expRepo string + }{ + { + "correct URL", + "http://new.repo.org", + "http://new.repo.org", + }, + { + "correct URL https", + "https://new.repo.org", + "https://new.repo.org", + }, + { + "correct URL with trailing backslash", + "http://new.repo.org/", + "http://new.repo.org", + }, + { + "correct URL https with trailing backslash", + "https://new.repo.org/", + "https://new.repo.org", + }, + } + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + j := newJarGetter() + err := j.setRepositoryURL(test.newRepo) + if err != nil { + t.Errorf("failed to set repository URL, got %v", err) + } + if got, want := j.getRepositoryURL(), test.expRepo; got != want { + t.Errorf("getRepositoryURL() got %v, want %v", got, want) + } + }) + } +} + +func TestGetAndSetRepositoryURL_bad(t *testing.T) { + tests := []struct { + name string + newRepo string + }{ + { + "no http", + "new.maven.repo.com", + }, + } + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + j := newJarGetter() + err := j.setRepositoryURL(test.newRepo) + if err == nil { + t.Errorf("setRepositoryURL(%v) succeeded when it should have failed", test.newRepo) + } + // Check that the failed Set call did not change the URL. + if got, want := j.getRepositoryURL(), string(apacheRepository); got != want { + t.Errorf("getRepositoryURL() got %v, want %v", got, want) + } + }) + } +} + func TestGetURLForBeamJar(t *testing.T) { tests := []struct { name string From ee7a1fbf67f530a0896008e915aa2db6656a354e Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Wed, 2 Mar 2022 20:52:33 -0500 Subject: [PATCH 22/68] [BEAM-13903] Improve coverage of metricsx package (#16994) --- .../core/runtime/metricsx/metricsx_test.go | 184 +++++++++++++++++- 1 file changed, 181 insertions(+), 3 deletions(-) diff --git a/sdks/go/pkg/beam/core/runtime/metricsx/metricsx_test.go b/sdks/go/pkg/beam/core/runtime/metricsx/metricsx_test.go index 20492ab2e699..95b92f929c08 100644 --- a/sdks/go/pkg/beam/core/runtime/metricsx/metricsx_test.go +++ b/sdks/go/pkg/beam/core/runtime/metricsx/metricsx_test.go @@ -59,7 +59,7 @@ func TestFromMonitoringInfos_Counters(t *testing.T) { got := FromMonitoringInfos(p, attempted, committed).AllMetrics().Counters() size := len(got) - if size < 1 { + if size != 1 { t.Fatalf("Invalid array's size: got: %v, want: %v", size, 1) } if d := cmp.Diff(want, got[0]); d != "" { @@ -68,6 +68,184 @@ func TestFromMonitoringInfos_Counters(t *testing.T) { } } +func TestFromMonitoringInfos_Msec(t *testing.T) { + want := metrics.MsecResult{ + Attempted: metrics.MsecValue{ + Start: 15 * time.Millisecond, + Process: 20 * time.Millisecond, + Finish: 40 * time.Millisecond, + Total: 25 * time.Millisecond, + }, + Committed: metrics.MsecValue{ + Start: 0 * time.Millisecond, + Process: 0 * time.Millisecond, + Finish: 0 * time.Millisecond, + Total: 0 * time.Millisecond, + }, + Key: metrics.StepKey{ + Step: "main.customDoFn", + Name: "customCounter", + Namespace: "customDoFn", + }, + } + + labels := map[string]string{ + "PTRANSFORM": "main.customDoFn", + "NAMESPACE": "customDoFn", + "NAME": "customCounter", + } + + startValue, err := Int64Counter(int64(15)) + if err != nil { + t.Fatalf("Failed to encode Int64Counter: %v", err) + } + processValue, err := Int64Counter(int64(20)) + if err != nil { + t.Fatalf("Failed to encode Int64Counter: %v", err) + } + finishValue, err := Int64Counter(int64(40)) + if err != nil { + t.Fatalf("Failed to encode Int64Counter: %v", err) + } + totalValue, err := Int64Counter(int64(25)) + if err != nil { + t.Fatalf("Failed to encode Int64Counter: %v", err) + } + mStartBundleInfo := &pipepb.MonitoringInfo{ + Urn: UrnToString(ExecutionMsecUrn(0)), + Type: UrnToType(ExecutionMsecUrn(0)), + Labels: labels, + Payload: startValue, + } + mProcessBundleInfo := &pipepb.MonitoringInfo{ + Urn: UrnToString(ExecutionMsecUrn(1)), + Type: UrnToType(ExecutionMsecUrn(1)), + Labels: labels, + Payload: processValue, + } + mFinishBundleInfo := &pipepb.MonitoringInfo{ + Urn: UrnToString(ExecutionMsecUrn(2)), + Type: UrnToType(ExecutionMsecUrn(2)), + Labels: labels, + Payload: finishValue, + } + mTotalTimeInfo := &pipepb.MonitoringInfo{ + Urn: UrnToString(ExecutionMsecUrn(3)), + Type: UrnToType(ExecutionMsecUrn(3)), + Labels: labels, + Payload: totalValue, + } + + attempted := []*pipepb.MonitoringInfo{mStartBundleInfo, mProcessBundleInfo, mFinishBundleInfo, mTotalTimeInfo} + committed := []*pipepb.MonitoringInfo{} + p := &pipepb.Pipeline{} + + got := FromMonitoringInfos(p, attempted, committed).AllMetrics().Msecs() + size := len(got) + if size != 1 { + t.Fatalf("Invalid array's size: got: %v, want: %v", size, 1) + } + if d := cmp.Diff(want, got[0]); d != "" { + t.Fatalf("Invalid MsecResult: got: %v, want: %v, diff(-want,+got):\n %v", + got[0], want, d) + } +} + +func TestFromMonitoringInfos_PColCounters(t *testing.T) { + var value int64 = 15 + want := metrics.PColResult{ + Attempted: metrics.PColValue{ + ElementCount: 15, + }, + Key: metrics.StepKey{ + Step: "main.customDoFn", + Name: "customCounter", + Namespace: "customDoFn", + }} + + payload, err := Int64Counter(value) + if err != nil { + t.Fatalf("Failed to encode Int64Counter: %v", err) + } + + labels := map[string]string{ + "PTRANSFORM": "main.customDoFn", + "NAMESPACE": "customDoFn", + "NAME": "customCounter", + } + + mInfo := &pipepb.MonitoringInfo{ + Urn: UrnToString(UrnElementCount), + Type: UrnToType(UrnElementCount), + Labels: labels, + Payload: payload, + } + + attempted := []*pipepb.MonitoringInfo{mInfo} + committed := []*pipepb.MonitoringInfo{} + p := &pipepb.Pipeline{} + + got := FromMonitoringInfos(p, attempted, committed).AllMetrics().PCols() + size := len(got) + if size != 1 { + t.Fatalf("Invalid array's size: got: %v, want: %v", size, 1) + } + if d := cmp.Diff(want, got[0]); d != "" { + t.Fatalf("Invalid counter: got: %v, want: %v, diff(-want,+got):\n %v", + got[0], want, d) + } +} + +func TestFromMonitoringInfos_SampledByteSize(t *testing.T) { + want := metrics.PColResult{ + Attempted: metrics.PColValue{ + SampledByteSize: metrics.DistributionValue{ + Count: 100, + Sum: 5, + Min: -12, + Max: 30, + }, + }, + Key: metrics.StepKey{ + Step: "main.customDoFn", + Name: "customCounter", + Namespace: "customDoFn", + }} + + var count, sum, min, max int64 = 100, 5, -12, 30 + payload, err := Int64Distribution(count, sum, min, max) + if err != nil { + t.Fatalf("Failed to encode Int64Distribution: %v", err) + } + + labels := map[string]string{ + "PTRANSFORM": "main.customDoFn", + "NAMESPACE": "customDoFn", + "NAME": "customCounter", + } + + mInfo := &pipepb.MonitoringInfo{ + Urn: UrnToString(UrnSampledByteSize), + Type: UrnToType(UrnSampledByteSize), + Labels: labels, + Payload: payload, + } + + attempted := []*pipepb.MonitoringInfo{mInfo} + committed := []*pipepb.MonitoringInfo{} + p := &pipepb.Pipeline{} + + got := FromMonitoringInfos(p, attempted, committed).AllMetrics().PCols() + size := len(got) + if size != 1 { + t.Fatalf("Invalid array's size: got: %v, want: %v", size, FromMonitoringInfos(p, attempted, committed).AllMetrics()) + } + if d := cmp.Diff(want, got[0]); d != "" { + t.Fatalf("Invalid counter: got: %v, want: %v, diff(-want,+got):\n %v", + got[0], want, d) + } +} + func TestFromMonitoringInfos_Distributions(t *testing.T) { var count, sum, min, max int64 = 100, 5, -12, 30 @@ -109,7 +287,7 @@ func TestFromMonitoringInfos_Distributions(t *testing.T) { got := FromMonitoringInfos(p, attempted, committed).AllMetrics().Distributions() size := len(got) - if size < 1 { + if size != 1 { t.Fatalf("Invalid array's size: got: %v, want: %v", size, 1) } if d := cmp.Diff(want, got[0]); d != "" { @@ -159,7 +337,7 @@ func TestFromMonitoringInfos_Gauges(t *testing.T) { got := FromMonitoringInfos(p, attempted, committed).AllMetrics().Gauges() size := len(got) - if size < 1 { + if size != 1 { t.Fatalf("Invalid array's size: got: %v, want: %v", size, 1) } if d := cmp.Diff(want, got[0]); d != "" { From 62e0c815bc616de0d5b81a76feb2535e89c8d6ef Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Wed, 2 Mar 2022 20:59:00 -0500 Subject: [PATCH 23/68] [BEAM-13892] Improve coverage of avroio package (#16990) --- sdks/go/data/tweet.avro | Bin 0 -> 275 bytes sdks/go/pkg/beam/io/avroio/avroio_test.go | 133 ++++++++++++++++++++++ 2 files changed, 133 insertions(+) create mode 100644 sdks/go/data/tweet.avro create mode 100644 sdks/go/pkg/beam/io/avroio/avroio_test.go diff --git a/sdks/go/data/tweet.avro b/sdks/go/data/tweet.avro new file mode 100644 index 0000000000000000000000000000000000000000..72e0cd857b17719f175cf9900a4d864317db1ca0 GIT binary patch literal 275 zcmeZI%3@>@ODrqO*DFrWNX<>W##F78mYJH9QmkYZU9FUtn479(rBsrcn_66wm|LKv zqf}B^01`^cFHOoxRjSoNC@N1)EkP(KE-A{)OGj2vTAW%0HV|$IOiio~+$N|YAa%tB ziOFEw%QH(#Qi~8q7o{fW7o{lGihv!SoS%}K%u}3~SWr;O@X~c^cE!`TOP6M!abcUA laFWSEhJk^VF~r0{$|E%=CtsliVi*_5F@}r_9vPzR006pQV(S0^ literal 0 HcmV?d00001 diff --git a/sdks/go/pkg/beam/io/avroio/avroio_test.go b/sdks/go/pkg/beam/io/avroio/avroio_test.go new file mode 100644 index 000000000000..b87907e8fae7 --- /dev/null +++ b/sdks/go/pkg/beam/io/avroio/avroio_test.go @@ -0,0 +1,133 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package avroio + +import ( + "bytes" + "encoding/json" + "errors" + "io/ioutil" + "os" + "reflect" + "testing" + + "github.com/apache/beam/sdks/v2/go/pkg/beam" + _ "github.com/apache/beam/sdks/v2/go/pkg/beam/io/filesystem/local" + "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/passert" + "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/ptest" + + "github.com/linkedin/goavro" +) + +type Tweet struct { + Stamp int64 `json:"timestamp"` + Tweet string `json:"tweet"` + User string `json:"username"` +} + +const schema = `{ + "type": "record", + "name": "tweet", + "namespace": "twitter", + "fields": [ + { "name": "timestamp", "type": "double" }, + { "name": "tweet", "type": "string" }, + { "name": "username", "type": "string" } + ] +}` + +func TestRead(t *testing.T) { + avroFile := "../../../../data/tweet.avro" + + p := beam.NewPipeline() + s := p.Root() + tweets := Read(s, avroFile, reflect.TypeOf(Tweet{})) + passert.Count(s, tweets, "NumUsers", 1) + passert.Equals(s, tweets, Tweet{ + Stamp: int64(20), + Tweet: "Hello twitter", + User: "user1", + }) + + ptest.RunAndValidate(t, p) +} + +type TwitterUser struct { + User string `json:"username"` + Info string `json:"info"` +} + +const userSchema = `{ + "type": "record", + "name": "user", + "namespace": "twitter", + "fields": [ + { "name": "username", "type": "string" }, + { "name": "info", "type": "string" } + ] +}` + +func TestWrite(t *testing.T) { + avroFile := "./user.avro" + testUsername := "user1" + testInfo := "userInfo" + p, s, sequence := ptest.CreateList([]string{testUsername}) + format := beam.ParDo(s, func(username string, emit func(string)) { + newUser := TwitterUser{ + User: username, + Info: testInfo, + } + + b, _ := json.Marshal(newUser) + emit(string(b)) + }, sequence) + Write(s, avroFile, userSchema, format) + t.Cleanup(func() { + os.Remove(avroFile) + }) + + ptest.RunAndValidate(t, p) + + if _, err := os.Stat(avroFile); errors.Is(err, os.ErrNotExist) { + t.Fatalf("Failed to write file %v", avroFile) + } + + avroBytes, err := ioutil.ReadFile(avroFile) + if err != nil { + t.Fatalf("Failed to read avro file: %v", err) + } + ocf, err := goavro.NewOCFReader(bytes.NewReader(avroBytes)) + var nativeData []interface{} + for ocf.Scan() { + datum, err := ocf.Read() + if err != nil { + break // Read error sets OCFReader error + } + nativeData = append(nativeData, datum) + } + if err := ocf.Err(); err != nil { + t.Fatalf("Error decoding avro data: %v", err) + } + if got, want := len(nativeData), 1; got != want { + t.Fatalf("Avro data, got %v records, want %v", got, want) + } + if got, want := nativeData[0].(map[string]interface{})["username"], testUsername; got != want { + t.Fatalf("User.User=%v, want %v", got, want) + } + if got, want := nativeData[0].(map[string]interface{})["info"], testInfo; got != want { + t.Fatalf("User.User=%v, want %v", got, want) + } +} From 8478a0be7363a1e2cab37408090ce2329d1fffa3 Mon Sep 17 00:00:00 2001 From: Moritz Mack Date: Thu, 3 Mar 2022 14:38:41 +0100 Subject: [PATCH 24/68] [adhoc] Prepare aws2 ClientConfiguration for json serialization and cleanup AWS Module (#16894) * [adhoc] Prepare aws2 ClientConfiguration and related classes for json serialization and cleanup AWS Module --- .../java/io/amazon-web-services2/build.gradle | 1 + .../io/aws2/common/ClientConfiguration.java | 95 +++++++------------ .../aws2/common/HttpClientConfiguration.java | 23 ++++- .../io/aws2/common/RetryConfiguration.java | 43 ++++++++- .../beam/sdk/io/aws2/options/AwsModule.java | 80 ++-------------- .../beam/sdk/io/aws2/s3/SSECustomerKey.java | 6 ++ .../aws2/common/ClientConfigurationTest.java | 31 +++++- .../common/HttpClientConfigurationTest.java | 49 ++++++++++ .../aws2/common/RetryConfigurationTest.java | 22 +++++ .../sdk/io/aws2/options/AwsModuleTest.java | 29 ------ .../sdk/io/aws2/s3/SSECustomerKeyTest.java | 17 +++- 11 files changed, 229 insertions(+), 167 deletions(-) create mode 100644 sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/common/HttpClientConfigurationTest.java diff --git a/sdks/java/io/amazon-web-services2/build.gradle b/sdks/java/io/amazon-web-services2/build.gradle index dceb4c41bed9..817b7b48e24c 100644 --- a/sdks/java/io/amazon-web-services2/build.gradle +++ b/sdks/java/io/amazon-web-services2/build.gradle @@ -30,6 +30,7 @@ ext.summary = "IO library to read and write Amazon Web Services services from Be dependencies { implementation library.java.vendored_guava_26_0_jre + implementation library.java.error_prone_annotations implementation project(path: ":sdks:java:core", configuration: "shadow") implementation library.java.aws_java_sdk2_apache_client implementation library.java.aws_java_sdk2_netty_client diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/common/ClientConfiguration.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/common/ClientConfiguration.java index 4371d7509999..9ee8eb277ddc 100644 --- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/common/ClientConfiguration.java +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/common/ClientConfiguration.java @@ -19,19 +19,21 @@ import static org.apache.beam.sdk.io.aws2.options.AwsSerializableUtils.deserializeAwsCredentialsProvider; import static org.apache.beam.sdk.io.aws2.options.AwsSerializableUtils.serializeAwsCredentialsProvider; -import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonSetter; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.databind.annotation.JsonPOJOBuilder; import com.google.auto.value.AutoValue; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; +import com.google.auto.value.extension.memoized.Memoized; import java.io.Serializable; import java.net.URI; import java.util.function.Consumer; import javax.annotation.Nullable; import org.apache.beam.sdk.io.aws2.options.AwsOptions; import org.checkerframework.dataflow.qual.Pure; -import software.amazon.awssdk.auth.credentials.AwsCredentials; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.regions.Region; @@ -47,18 +49,28 @@ * uses a backoff strategy with equal jitter for computing the delay before the next retry. */ @AutoValue +@JsonInclude(value = JsonInclude.Include.NON_EMPTY) +@JsonDeserialize(builder = ClientConfiguration.Builder.class) public abstract class ClientConfiguration implements Serializable { /** * Optional {@link AwsCredentialsProvider}. If set, this overwrites the default in {@link * AwsOptions#getAwsCredentialsProvider()}. */ - public abstract @Nullable @Pure AwsCredentialsProvider credentialsProvider(); + @JsonProperty + @Memoized + public @Nullable @Pure AwsCredentialsProvider credentialsProvider() { + return credentialsProviderAsJson() != null + ? deserializeAwsCredentialsProvider(credentialsProviderAsJson()) + : null; + } /** * Optional {@link Region}. If set, this overwrites the default in {@link * AwsOptions#getAwsRegion()}. */ + @JsonProperty + @Memoized public @Nullable @Pure Region region() { return regionId() != null ? Region.of(regionId()) : null; } @@ -67,20 +79,24 @@ public abstract class ClientConfiguration implements Serializable { * Optional service endpoint to use AWS compatible services instead, e.g. for testing. If set, * this overwrites the default in {@link AwsOptions#getEndpoint()}. */ + @JsonProperty public abstract @Nullable @Pure URI endpoint(); /** * Optional {@link RetryConfiguration} for AWS clients. If unset, retry behavior will be unchanged * and use SDK defaults. */ + @JsonProperty public abstract @Nullable @Pure RetryConfiguration retry(); abstract @Nullable @Pure String regionId(); + abstract @Nullable @Pure String credentialsProviderAsJson(); + public abstract Builder toBuilder(); public static Builder builder() { - return new AutoValue_ClientConfiguration.Builder(); + return Builder.builder(); } public static ClientConfiguration create( @@ -93,12 +109,20 @@ public static ClientConfiguration create( } @AutoValue.Builder + @JsonPOJOBuilder(withPrefix = "") public abstract static class Builder { + @JsonCreator + static Builder builder() { + return new AutoValue_ClientConfiguration.Builder(); + } + /** * Optional {@link AwsCredentialsProvider}. If set, this overwrites the default in {@link * AwsOptions#getAwsCredentialsProvider()}. */ - public abstract Builder credentialsProvider(AwsCredentialsProvider credentialsProvider); + public Builder credentialsProvider(AwsCredentialsProvider credentialsProvider) { + return credentialsProviderAsJson(serializeAwsCredentialsProvider(credentialsProvider)); + } /** * Optional {@link Region}. If set, this overwrites the default in {@link @@ -118,6 +142,7 @@ public Builder region(Region region) { * Optional {@link RetryConfiguration} for AWS clients. If unset, retry behavior will be * unchanged and use SDK defaults. */ + @JsonSetter public abstract Builder retry(RetryConfiguration retry); /** @@ -132,58 +157,8 @@ public Builder retry(Consumer retry) { abstract Builder regionId(String region); - abstract AwsCredentialsProvider credentialsProvider(); - - abstract ClientConfiguration autoBuild(); + abstract Builder credentialsProviderAsJson(String credentialsProvider); - public ClientConfiguration build() { - if (credentialsProvider() != null) { - credentialsProvider(new SerializableAwsCredentialsProvider(credentialsProvider())); - } - return autoBuild(); - } - } - - /** Internal serializable {@link AwsCredentialsProvider}. */ - private static class SerializableAwsCredentialsProvider - implements AwsCredentialsProvider, Serializable { - private transient AwsCredentialsProvider provider; - private String serializedProvider; - - SerializableAwsCredentialsProvider(AwsCredentialsProvider provider) { - this.provider = checkNotNull(provider, "AwsCredentialsProvider cannot be null"); - this.serializedProvider = serializeAwsCredentialsProvider(provider); - } - - private void writeObject(ObjectOutputStream out) throws IOException { - out.writeUTF(serializedProvider); - } - - private void readObject(ObjectInputStream in) throws IOException { - serializedProvider = in.readUTF(); - provider = deserializeAwsCredentialsProvider(serializedProvider); - } - - @Override - public AwsCredentials resolveCredentials() { - return provider.resolveCredentials(); - } - - @Override - public boolean equals(@Nullable Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - SerializableAwsCredentialsProvider that = (SerializableAwsCredentialsProvider) o; - return serializedProvider.equals(that.serializedProvider); - } - - @Override - public int hashCode() { - return serializedProvider.hashCode(); - } + public abstract ClientConfiguration build(); } } diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/common/HttpClientConfiguration.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/common/HttpClientConfiguration.java index 3b9cb1d9f936..65b882a95391 100644 --- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/common/HttpClientConfiguration.java +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/common/HttpClientConfiguration.java @@ -17,6 +17,11 @@ */ package org.apache.beam.sdk.io.aws2.common; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.databind.annotation.JsonPOJOBuilder; import com.google.auto.value.AutoValue; import java.io.Serializable; import javax.annotation.Nullable; @@ -32,11 +37,14 @@ * HTTP Configuration */ @AutoValue +@JsonInclude(value = JsonInclude.Include.NON_EMPTY) +@JsonDeserialize(builder = HttpClientConfiguration.Builder.class) public abstract class HttpClientConfiguration implements Serializable { /** * Milliseconds to wait when acquiring a connection from the pool before giving up and timing out. */ + @JsonProperty public abstract @Nullable @Pure Integer connectionAcquisitionTimeout(); /** @@ -45,12 +53,14 @@ public abstract class HttpClientConfiguration implements Serializable { *

This will never close a connection that is currently in use, so long-lived connections may * remain open longer than this time. */ + @JsonProperty public abstract @Nullable @Pure Integer connectionMaxIdleTime(); /** * Milliseconds to wait when initially establishing a connection before giving up and timing out. * A duration of 0 means infinity, and is not recommended. */ + @JsonProperty public abstract @Nullable @Pure Integer connectionTimeout(); /** @@ -60,12 +70,14 @@ public abstract class HttpClientConfiguration implements Serializable { *

This will never close a connection that is currently in use, so long-lived connections may * remain open longer than this time. */ + @JsonProperty public abstract @Nullable @Pure Integer connectionTimeToLive(); /** * Milliseconds to wait for data to be transferred over an established, open connection before the * connection is timed out. A duration of 0 means infinity, and is not recommended. */ + @JsonProperty public abstract @Nullable @Pure Integer socketTimeout(); /** @@ -75,6 +87,7 @@ public abstract class HttpClientConfiguration implements Serializable { *

Note: Read timeout is only supported for async clients and ignored otherwise. Use {@link * #socketTimeout()} instead. */ + @JsonProperty public abstract @Nullable @Pure Integer readTimeout(); /** @@ -84,6 +97,7 @@ public abstract class HttpClientConfiguration implements Serializable { *

Note: Write timeout is only supported for async clients and ignored otherwise. Use {@link * #socketTimeout()} instead. */ + @JsonProperty public abstract @Nullable @Pure Integer writeTimeout(); /** @@ -94,14 +108,21 @@ public abstract class HttpClientConfiguration implements Serializable { * concurrent requests. When using HTTP/2 the number of connections that will be used depends on * the max streams allowed per connection. */ + @JsonProperty public abstract @Nullable @Pure Integer maxConnections(); public static Builder builder() { - return new AutoValue_HttpClientConfiguration.Builder(); + return Builder.builder(); } @AutoValue.Builder + @JsonPOJOBuilder(withPrefix = "") public abstract static class Builder { + @JsonCreator + static Builder builder() { + return new AutoValue_HttpClientConfiguration.Builder(); + } + /** * Milliseconds to wait when acquiring a connection from the pool before giving up and timing * out. diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/common/RetryConfiguration.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/common/RetryConfiguration.java index 1c226010e9e9..6ca342927a89 100644 --- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/common/RetryConfiguration.java +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/common/RetryConfiguration.java @@ -20,6 +20,13 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; import static org.joda.time.Duration.ZERO; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.databind.annotation.JsonPOJOBuilder; +import com.fasterxml.jackson.databind.annotation.JsonSerialize; +import com.fasterxml.jackson.databind.util.StdConverter; import com.google.auto.value.AutoValue; import java.io.Serializable; import javax.annotation.Nullable; @@ -36,31 +43,51 @@ * SdkDefaultRetrySetting} for further details. */ @AutoValue +@JsonInclude(value = JsonInclude.Include.NON_EMPTY) +@JsonDeserialize(builder = RetryConfiguration.Builder.class) public abstract class RetryConfiguration implements Serializable { private static final java.time.Duration BASE_BACKOFF = java.time.Duration.ofMillis(100); private static final java.time.Duration THROTTLED_BASE_BACKOFF = java.time.Duration.ofSeconds(1); private static final java.time.Duration MAX_BACKOFF = java.time.Duration.ofSeconds(20); + @JsonProperty public abstract @Pure int numRetries(); + @JsonProperty + @JsonSerialize(converter = DurationToMillis.class) public abstract @Nullable @Pure Duration baseBackoff(); + @JsonProperty + @JsonSerialize(converter = DurationToMillis.class) public abstract @Nullable @Pure Duration throttledBaseBackoff(); + @JsonProperty + @JsonSerialize(converter = DurationToMillis.class) public abstract @Nullable @Pure Duration maxBackoff(); + public abstract RetryConfiguration.Builder toBuilder(); + public static Builder builder() { - return new AutoValue_RetryConfiguration.Builder(); + return Builder.builder(); } @AutoValue.Builder + @JsonPOJOBuilder(withPrefix = "") public abstract static class Builder { + @JsonCreator + static Builder builder() { + return new AutoValue_RetryConfiguration.Builder(); + } + public abstract Builder numRetries(int numRetries); + @JsonDeserialize(converter = MillisToDuration.class) public abstract Builder baseBackoff(Duration baseBackoff); + @JsonDeserialize(converter = MillisToDuration.class) public abstract Builder throttledBaseBackoff(Duration baseBackoff); + @JsonDeserialize(converter = MillisToDuration.class) public abstract Builder maxBackoff(Duration maxBackoff); abstract RetryConfiguration autoBuild(); @@ -115,4 +142,18 @@ RetryPolicy toClientRetryPolicy() { private @Nullable static java.time.Duration toJava(@Nullable Duration duration) { return duration == null ? null : java.time.Duration.ofMillis(duration.getMillis()); } + + static class DurationToMillis extends StdConverter { + @Override + public Long convert(Duration duration) { + return duration.getMillis(); + } + } + + static class MillisToDuration extends StdConverter { + @Override + public Duration convert(Long millis) { + return Duration.millis(millis); + } + } } diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/options/AwsModule.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/options/AwsModule.java index ff54a18fcc46..0f8b138d0b95 100644 --- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/options/AwsModule.java +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/options/AwsModule.java @@ -20,6 +20,7 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.core.JsonGenerator; @@ -42,14 +43,10 @@ import com.fasterxml.jackson.databind.util.NameTransformer; import com.google.auto.service.AutoService; import java.io.IOException; -import java.net.URI; -import java.util.Map; import java.util.function.Supplier; import org.apache.beam.repackaged.core.org.apache.commons.lang3.reflect.FieldUtils; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; -import org.apache.beam.sdk.io.aws2.common.HttpClientConfiguration; -import org.apache.beam.sdk.io.aws2.s3.SSECustomerKey; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet; import org.checkerframework.checker.nullness.qual.NonNull; import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; @@ -67,7 +64,6 @@ import software.amazon.awssdk.services.sts.StsClient; import software.amazon.awssdk.services.sts.auth.StsAssumeRoleCredentialsProvider; import software.amazon.awssdk.services.sts.model.AssumeRoleRequest; -import software.amazon.awssdk.utils.AttributeMap; /** * A Jackson {@link Module} that registers a {@link JsonSerializer} and {@link JsonDeserializer} for @@ -88,15 +84,11 @@ public AwsModule() { public void setupModule(SetupContext cxt) { cxt.setMixInAnnotations(AwsCredentialsProvider.class, AwsCredentialsProviderMixin.class); cxt.setMixInAnnotations(ProxyConfiguration.class, ProxyConfigurationMixin.class); - cxt.setMixInAnnotations(HttpClientConfiguration.class, HttpClientConfigurationMixin.class); cxt.setMixInAnnotations( - HttpClientConfiguration.Builder.class, HttpClientConfigurationMixin.Builder.class); - cxt.setMixInAnnotations(SSECustomerKey.class, SSECustomerKeyMixin.class); - cxt.setMixInAnnotations(SSECustomerKey.Builder.class, SSECustomerKeyMixin.Builder.class); + ProxyConfiguration.Builder.class, ProxyConfigurationMixin.Builder.class); cxt.setMixInAnnotations(Region.class, RegionMixin.class); - addValueInstantiator(HttpClientConfiguration.Builder.class, HttpClientConfiguration::builder); - + addValueInstantiator(ProxyConfiguration.Builder.class, ProxyConfiguration::builder); super.setupModule(cxt); } @@ -266,71 +258,11 @@ private Object readField(AwsCredentialsProvider provider, String fieldName) thro } /** A mixin to add Jackson annotations to {@link ProxyConfiguration}. */ - @JsonDeserialize(using = ProxyConfigurationDeserializer.class) - @JsonSerialize(using = ProxyConfigurationSerializer.class) - private static class ProxyConfigurationMixin {} - - private static class ProxyConfigurationDeserializer extends JsonDeserializer { - @Override - public ProxyConfiguration deserialize(JsonParser jsonParser, DeserializationContext context) - throws IOException { - Map asMap = - checkNotNull( - jsonParser.readValueAs(new TypeReference>() {}), - "Serialized ProxyConfiguration is null"); - - ProxyConfiguration.Builder builder = ProxyConfiguration.builder(); - final String endpoint = asMap.get("endpoint"); - if (endpoint != null) { - builder.endpoint(URI.create(endpoint)); - } - final String username = asMap.get("username"); - if (username != null) { - builder.username(username); - } - final String password = asMap.get("password"); - if (password != null) { - builder.password(password); - } - // defaults to FALSE / disabled - Boolean useSystemPropertyValues = Boolean.valueOf(asMap.get("useSystemPropertyValues")); - return builder.useSystemPropertyValues(useSystemPropertyValues).build(); - } - } - - private static class ProxyConfigurationSerializer extends JsonSerializer { - @Override - public void serialize( - ProxyConfiguration proxyConfiguration, - JsonGenerator jsonGenerator, - SerializerProvider serializer) - throws IOException { - // proxyConfiguration.endpoint() is private so we have to build it manually. - final String endpoint = - proxyConfiguration.scheme() - + "://" - + proxyConfiguration.host() - + ":" - + proxyConfiguration.port(); - jsonGenerator.writeStartObject(); - jsonGenerator.writeStringField("endpoint", endpoint); - jsonGenerator.writeStringField("username", proxyConfiguration.username()); - jsonGenerator.writeStringField("password", proxyConfiguration.password()); - jsonGenerator.writeEndObject(); - } - } - - /** A mixin to add Jackson annotations to {@link AttributeMap}. */ - @JsonDeserialize(builder = HttpClientConfiguration.Builder.class) + @JsonDeserialize(builder = ProxyConfiguration.Builder.class) @JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.ANY) + @JsonIgnoreProperties(value = {"host", "port", "scheme"}) @JsonInclude(value = JsonInclude.Include.NON_EMPTY) - private static class HttpClientConfigurationMixin { - @JsonPOJOBuilder(withPrefix = "") - static class Builder {} - } - - @JsonDeserialize(builder = SSECustomerKey.Builder.class) - private static class SSECustomerKeyMixin { + private static class ProxyConfigurationMixin { @JsonPOJOBuilder(withPrefix = "") static class Builder {} } diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/s3/SSECustomerKey.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/s3/SSECustomerKey.java index 64ee4aa0347a..8e0ea423378f 100644 --- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/s3/SSECustomerKey.java +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/s3/SSECustomerKey.java @@ -19,12 +19,17 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.databind.annotation.JsonPOJOBuilder; import java.nio.charset.StandardCharsets; import java.util.Base64; import org.apache.commons.codec.digest.DigestUtils; import org.checkerframework.checker.nullness.qual.Nullable; /** Customer provided key for use with Amazon S3 server-side encryption. */ +@JsonInclude(value = JsonInclude.Include.NON_EMPTY) +@JsonDeserialize(builder = SSECustomerKey.Builder.class) public class SSECustomerKey { private final @Nullable String key; @@ -63,6 +68,7 @@ public static Builder builder() { return new Builder(); } + @JsonPOJOBuilder(withPrefix = "") public static class Builder { private @Nullable String key; diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/common/ClientConfigurationTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/common/ClientConfigurationTest.java index 93615f513a33..0c2c91b1f8c4 100644 --- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/common/ClientConfigurationTest.java +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/common/ClientConfigurationTest.java @@ -22,16 +22,18 @@ import static org.assertj.core.api.Assertions.assertThat; import java.net.URI; +import org.apache.beam.sdk.io.aws2.options.SerializationTestUtil; import org.junit.Test; import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; import software.amazon.awssdk.regions.Region; public class ClientConfigurationTest { @Test - public void testSerialization() { + public void testJavaSerialization() { AwsCredentialsProvider credentials = StaticCredentialsProvider.create(AwsBasicCredentials.create("key", "secret")); @@ -50,4 +52,31 @@ public void testSerialization() { assertThat(deserializedConfig).isEqualTo(config); } + + @Test + public void testJsonSerialization() { + ClientConfiguration config = ClientConfiguration.builder().build(); + assertThat(jsonSerializeDeserialize(config)).isEqualTo(config); + + config = config.toBuilder().region(Region.US_WEST_1).build(); + assertThat(jsonSerializeDeserialize(config)).isEqualTo(config); + + config = config.toBuilder().credentialsProvider(DefaultCredentialsProvider.create()).build(); + assertThat(jsonSerializeDeserialize(config)).isEqualTo(config); + + AwsBasicCredentials credentials = AwsBasicCredentials.create("key", "secret"); + StaticCredentialsProvider credentialsProvider = StaticCredentialsProvider.create(credentials); + config = config.toBuilder().credentialsProvider(credentialsProvider).build(); + assertThat(jsonSerializeDeserialize(config)).isEqualTo(config); + + config = config.toBuilder().endpoint(URI.create("https://localhost:8080")).build(); + assertThat(jsonSerializeDeserialize(config)).isEqualTo(config); + + config = config.toBuilder().retry(r -> r.numRetries(10)).build(); + assertThat(jsonSerializeDeserialize(config)).isEqualTo(config); + } + + private ClientConfiguration jsonSerializeDeserialize(ClientConfiguration obj) { + return SerializationTestUtil.serializeDeserialize(ClientConfiguration.class, obj); + } } diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/common/HttpClientConfigurationTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/common/HttpClientConfigurationTest.java new file mode 100644 index 000000000000..b3147ac4e433 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/common/HttpClientConfigurationTest.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.aws2.common; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.beam.sdk.io.aws2.options.SerializationTestUtil; +import org.junit.Test; + +public class HttpClientConfigurationTest { + @Test + public void testJsonSerialization() { + HttpClientConfiguration expected = HttpClientConfiguration.builder().build(); + assertThat(serializeAndDeserialize(expected)).isEqualTo(expected); + + expected = + HttpClientConfiguration.builder() + .connectionAcquisitionTimeout(100) + .connectionMaxIdleTime(200) + .connectionTimeout(300) + .connectionTimeToLive(400) + .socketTimeout(500) + .readTimeout(600) + .writeTimeout(700) + .maxConnections(10) + .build(); + + assertThat(serializeAndDeserialize(expected)).isEqualTo(expected); + } + + private HttpClientConfiguration serializeAndDeserialize(HttpClientConfiguration obj) { + return SerializationTestUtil.serializeDeserialize(HttpClientConfiguration.class, obj); + } +} diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/common/RetryConfigurationTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/common/RetryConfigurationTest.java index 7ec86cbc45f1..dca607dbb60e 100644 --- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/common/RetryConfigurationTest.java +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/common/RetryConfigurationTest.java @@ -17,9 +17,12 @@ */ package org.apache.beam.sdk.io.aws2.common; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.joda.time.Duration.ZERO; +import org.apache.beam.sdk.io.aws2.options.SerializationTestUtil; +import org.joda.time.Duration; import org.junit.Test; public class RetryConfigurationTest { @@ -47,4 +50,23 @@ public void verifyMaxBackoffLargerZero() { assertThatThrownBy(() -> RetryConfiguration.builder().numRetries(1).maxBackoff(ZERO).build()) .hasMessage("maxBackoff must be greater than 0"); } + + @Test + public void testJsonSerialization() { + RetryConfiguration config = RetryConfiguration.builder().numRetries(10).build(); + assertThat(jsonSerializeDeserialize(config)).isEqualTo(config); + + config = config.toBuilder().maxBackoff(Duration.millis(1000)).build(); + assertThat(jsonSerializeDeserialize(config)).isEqualTo(config); + + config = config.toBuilder().baseBackoff(Duration.millis(200)).build(); + assertThat(jsonSerializeDeserialize(config)).isEqualTo(config); + + config = config.toBuilder().throttledBaseBackoff(Duration.millis(100)).build(); + assertThat(jsonSerializeDeserialize(config)).isEqualTo(config); + } + + private RetryConfiguration jsonSerializeDeserialize(RetryConfiguration obj) { + return SerializationTestUtil.serializeDeserialize(RetryConfiguration.class, obj); + } } diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/options/AwsModuleTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/options/AwsModuleTest.java index 2876b90a6590..b294c3faf711 100644 --- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/options/AwsModuleTest.java +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/options/AwsModuleTest.java @@ -33,8 +33,6 @@ import java.util.List; import java.util.Properties; import java.util.function.Supplier; -import org.apache.beam.sdk.io.aws2.common.HttpClientConfiguration; -import org.apache.beam.sdk.io.aws2.s3.SSECustomerKey; import org.apache.beam.sdk.util.ThrowingSupplier; import org.apache.beam.sdk.util.common.ReflectHelpers; import org.hamcrest.MatcherAssert; @@ -151,33 +149,6 @@ public void testProxyConfigurationSerializationDeserialization() throws Exceptio assertEquals("password", deserializedProxyConfiguration.password()); } - @Test - public void testHttpClientConfigurationSerializationDeserialization() throws Exception { - HttpClientConfiguration expected = - HttpClientConfiguration.builder() - .connectionAcquisitionTimeout(100) - .connectionMaxIdleTime(200) - .connectionTimeout(300) - .connectionTimeToLive(400) - .socketTimeout(500) - .readTimeout(600) - .writeTimeout(700) - .maxConnections(10) - .build(); - - assertThat(serializeAndDeserialize(expected)).isEqualTo(expected); - } - - @Test - public void testSSECustomerKeySerializationDeserialization() throws Exception { - // default key created by S3Options.SSECustomerKeyFactory - SSECustomerKey emptyKey = SSECustomerKey.builder().build(); - assertThat(serializeAndDeserialize(emptyKey)).isEqualToComparingFieldByField(emptyKey); - - SSECustomerKey key = SSECustomerKey.builder().key("key").algorithm("algo").md5("md5").build(); - assertThat(serializeAndDeserialize(key)).isEqualToComparingFieldByField(key); - } - private T withSystemPropertyOverrides(Properties overrides, ThrowingSupplier fun) throws Exception { Properties systemProps = System.getProperties(); diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/s3/SSECustomerKeyTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/s3/SSECustomerKeyTest.java index ac395520ac95..943a3fded9ac 100644 --- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/s3/SSECustomerKeyTest.java +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/s3/SSECustomerKeyTest.java @@ -17,9 +17,11 @@ */ package org.apache.beam.sdk.io.aws2.s3; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThrows; +import org.apache.beam.sdk.io.aws2.options.SerializationTestUtil; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -30,7 +32,6 @@ "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) }) public class SSECustomerKeyTest { - @Test public void testBuild() { assertThrows( @@ -54,4 +55,18 @@ public void buildWithArgs(String key, String algorithm, String md5, String encod assertEquals(algorithm, sseCustomerKey.getAlgorithm()); assertEquals(encodedMD5, sseCustomerKey.getMD5()); } + + @Test + public void testJsonSerializeDeserialize() { + // default key created by S3Options.SSECustomerKeyFactory + SSECustomerKey emptyKey = SSECustomerKey.builder().build(); + assertThat(jsonSerializeDeserialize(emptyKey)).isEqualToComparingFieldByField(emptyKey); + + SSECustomerKey key = SSECustomerKey.builder().key("key").algorithm("algo").md5("md5").build(); + assertThat(jsonSerializeDeserialize(key)).isEqualToComparingFieldByField(key); + } + + private SSECustomerKey jsonSerializeDeserialize(SSECustomerKey key) { + return SerializationTestUtil.serializeDeserialize(SSECustomerKey.class, key); + } } From 9ea06075e6d5042763132419b29a640978207169 Mon Sep 17 00:00:00 2001 From: Zoe Date: Fri, 4 Mar 2022 04:56:02 +1100 Subject: [PATCH 25/68] Merge pull request #16879 from [BEAM-12164] Add javadocs to SpannerConfig * Add tests and config for retry * lint * add tests * lint * Delete tests not passing * Rebase on apache beam master * review changes * review changes * add javadocs to SpannerConfig * revert * add full stops --- .../sdk/io/gcp/spanner/SpannerConfig.java | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java index a00b7896c35a..608396c8cfec 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java @@ -148,58 +148,78 @@ abstract Builder setExecuteStreamingSqlRetrySettings( public abstract SpannerConfig build(); } + /** Specifies the Cloud Spanner project ID. */ public SpannerConfig withProjectId(ValueProvider projectId) { return toBuilder().setProjectId(projectId).build(); } + /** Specifies the Cloud Spanner project ID. */ public SpannerConfig withProjectId(String projectId) { return withProjectId(ValueProvider.StaticValueProvider.of(projectId)); } + /** Specifies the Cloud Spanner instance ID. */ public SpannerConfig withInstanceId(ValueProvider instanceId) { return toBuilder().setInstanceId(instanceId).build(); } + /** Specifies the Cloud Spanner instance ID. */ public SpannerConfig withInstanceId(String instanceId) { return withInstanceId(ValueProvider.StaticValueProvider.of(instanceId)); } + /** Specifies the Cloud Spanner database ID. */ public SpannerConfig withDatabaseId(ValueProvider databaseId) { return toBuilder().setDatabaseId(databaseId).build(); } + /** Specifies the Cloud Spanner database ID. */ public SpannerConfig withDatabaseId(String databaseId) { return withDatabaseId(ValueProvider.StaticValueProvider.of(databaseId)); } + /** Specifies the Cloud Spanner host. */ public SpannerConfig withHost(ValueProvider host) { return toBuilder().setHost(host).build(); } + /** Specifies the Cloud Spanner host, when an emulator is used. */ public SpannerConfig withEmulatorHost(ValueProvider emulatorHost) { return toBuilder().setEmulatorHost(emulatorHost).build(); } + /** + * Specifies whether a local channel provider should be used. This should be set to True when an + * emulator is used. + */ public SpannerConfig withIsLocalChannelProvider(ValueProvider isLocalChannelProvider) { return toBuilder().setIsLocalChannelProvider(isLocalChannelProvider).build(); } + /** Specifies the commit deadline. This is overridden if the CommitRetrySettings is specified. */ public SpannerConfig withCommitDeadline(Duration commitDeadline) { return withCommitDeadline(ValueProvider.StaticValueProvider.of(commitDeadline)); } + /** Specifies the commit deadline. This is overridden if the CommitRetrySettings is specified. */ public SpannerConfig withCommitDeadline(ValueProvider commitDeadline) { return toBuilder().setCommitDeadline(commitDeadline).build(); } + /** Specifies the maximum cumulative backoff. */ public SpannerConfig withMaxCumulativeBackoff(Duration maxCumulativeBackoff) { return withMaxCumulativeBackoff(ValueProvider.StaticValueProvider.of(maxCumulativeBackoff)); } + /** Specifies the maximum cumulative backoff. */ public SpannerConfig withMaxCumulativeBackoff(ValueProvider maxCumulativeBackoff) { return toBuilder().setMaxCumulativeBackoff(maxCumulativeBackoff).build(); } + /** + * Specifies the ExecuteStreamingSql retry settings. If not set, the default timeout is set to 2 + * hours. + */ public SpannerConfig withExecuteStreamingSqlRetrySettings( RetrySettings executeStreamingSqlRetrySettings) { return toBuilder() @@ -207,23 +227,28 @@ public SpannerConfig withExecuteStreamingSqlRetrySettings( .build(); } + /** Specifies the commit retry settings. Setting this overrides the commit deadline. */ public SpannerConfig withCommitRetrySettings(RetrySettings commitRetrySettings) { return toBuilder().setCommitRetrySettings(commitRetrySettings).build(); } + /** Specifies the errors that will be retried by the client library for all operations. */ public SpannerConfig withRetryableCodes(ImmutableSet retryableCodes) { return toBuilder().setRetryableCodes(retryableCodes).build(); } + /** Specifies the service factory to create instance of Spanner. */ @VisibleForTesting SpannerConfig withServiceFactory(ServiceFactory serviceFactory) { return toBuilder().setServiceFactory(serviceFactory).build(); } + /** Specifies the RPC priority. */ public SpannerConfig withRpcPriority(RpcPriority rpcPriority) { return withRpcPriority(ValueProvider.StaticValueProvider.of(rpcPriority)); } + /** Specifies the RPC priority. */ public SpannerConfig withRpcPriority(ValueProvider rpcPriority) { return toBuilder().setRpcPriority(rpcPriority).build(); } From 53c61a20972420b5016b4ca1b28751a587982088 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Thu, 3 Mar 2022 11:45:04 -0800 Subject: [PATCH 26/68] [Cleanup] Update pre-v2 go package references (#17002) --- sdks/go/pkg/beam/core/runtime/harness/datamgr.go | 2 +- .../pkg/beam/core/runtime/harness/datamgr_test.go | 2 +- sdks/go/pkg/beam/core/runtime/harness/statemgr.go | 2 +- sdks/go/pkg/beam/util/gcsx/gcs.go | 2 +- .../content/en/documentation/programming-guide.md | 8 ++++---- .../documentation/resources/learning-resources.md | 2 +- .../www/site/content/en/documentation/sdks/go.md | 2 +- .../site/content/en/get-started/quickstart-go.md | 8 ++++---- .../site/content/en/get-started/try-apache-beam.md | 10 +++++----- .../content/en/get-started/wordcount-example.md | 14 +++++++------- 10 files changed, 26 insertions(+), 26 deletions(-) diff --git a/sdks/go/pkg/beam/core/runtime/harness/datamgr.go b/sdks/go/pkg/beam/core/runtime/harness/datamgr.go index 402310d10f20..dd4d26e9a11a 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/datamgr.go +++ b/sdks/go/pkg/beam/core/runtime/harness/datamgr.go @@ -492,7 +492,7 @@ func (w *dataWriter) send(msg *fnpb.Elements) error { for err == nil { // Per GRPC stream documentation, if there's an EOF, we must call Recv // until a non-nil error is returned, to ensure resources are cleaned up. - // https://godoc.org/google.golang.org/grpc#ClientConn.NewStream + // https://pkg.go.dev/google.golang.org/grpc#ClientConn.NewStream _, err = w.ch.client.Recv() } } diff --git a/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go b/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go index 5c09440e6160..465662bcd5b7 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go +++ b/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go @@ -92,7 +92,7 @@ func (f *fakeDataClient) Send(*fnpb.Elements) error { // We skip errors on the first call to test that errors can be returned // on the sentinel value send in dataWriter.Close // Otherwise, we return an io.EOF similar to semantics documented - // in https://godoc.org/google.golang.org/grpc#ClientConn.NewStream + // in https://pkg.go.dev/google.golang.org/grpc#ClientConn.NewStream if f.skipFirstError && f.err != nil { f.skipFirstError = false return nil diff --git a/sdks/go/pkg/beam/core/runtime/harness/statemgr.go b/sdks/go/pkg/beam/core/runtime/harness/statemgr.go index 2e3c76cbcec2..8becf139c5a1 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/statemgr.go +++ b/sdks/go/pkg/beam/core/runtime/harness/statemgr.go @@ -412,7 +412,7 @@ func (c *StateChannel) write(ctx context.Context) { for err == nil { // Per GRPC stream documentation, if there's an EOF, we must call Recv // until a non-nil error is returned, to ensure resources are cleaned up. - // https://godoc.org/google.golang.org/grpc#ClientConn.NewStream + // https://pkg.go.dev/google.golang.org/grpc#ClientConn.NewStream _, err = c.client.Recv() } } diff --git a/sdks/go/pkg/beam/util/gcsx/gcs.go b/sdks/go/pkg/beam/util/gcsx/gcs.go index 703eca4b2664..2ce66c9ad00d 100644 --- a/sdks/go/pkg/beam/util/gcsx/gcs.go +++ b/sdks/go/pkg/beam/util/gcsx/gcs.go @@ -30,7 +30,7 @@ import ( ) // NewClient creates a new GCS client with default application credentials, and supplied -// OAuth scope. The OAuth scopes are defined in https://godoc.org/cloud.google.com/go/storage#pkg-constants. +// OAuth scope. The OAuth scopes are defined in https://pkg.go.dev/cloud.google.com/go/storage#pkg-constants. func NewClient(ctx context.Context, scope string) (*storage.Client, error) { return storage.NewClient(ctx, option.WithScopes(scope)) } diff --git a/website/www/site/content/en/documentation/programming-guide.md b/website/www/site/content/en/documentation/programming-guide.md index 39ec60471f03..02adfeb92b03 100644 --- a/website/www/site/content/en/documentation/programming-guide.md +++ b/website/www/site/content/en/documentation/programming-guide.md @@ -6982,7 +6982,7 @@ When an SDK-specific wrapper isn't available, you will have to access the cross- #### 13.2.3. Using cross-language transforms in a Go pipeline If a Go-specific wrapper for a cross-language is available, use that. Otherwise, you have to use the -lower-level [CrossLanguage](https://pkg.go.dev/github.com/apache/beam/sdks/go/pkg/beam#CrossLanguage) +lower-level [CrossLanguage](https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam#CrossLanguage) function to access the transform. **Expansion Services** @@ -7021,10 +7021,10 @@ When an SDK-specific wrapper isn't available, you will have to access the cross- Refer to [Creating cross-language transforms](#create-x-lang-transforms) for details. 3. Use the `beam.CrossLanguage` function in your pipeline as appropriate. Reference the URN, payload, expansion service address, and define inputs and outputs. You can use the - [beam.CrossLanguagePayload](https://pkg.go.dev/github.com/apache/beam/sdks/go/pkg/beam#CrossLanguagePayload) + [beam.CrossLanguagePayload](https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam#CrossLanguagePayload) function as a helper for encoding a payload. You can use the - [beam.UnnamedInput](https://pkg.go.dev/github.com/apache/beam/sdks/go/pkg/beam#UnnamedInput) and - [beam.UnnamedOutput](https://pkg.go.dev/github.com/apache/beam/sdks/go/pkg/beam#UnnamedOutput) + [beam.UnnamedInput](https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam#UnnamedInput) and + [beam.UnnamedOutput](https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam#UnnamedOutput) functions as shortcuts for single, unnamed inputs/outputs or define a map for named ones. ```go diff --git a/website/www/site/content/en/documentation/resources/learning-resources.md b/website/www/site/content/en/documentation/resources/learning-resources.md index 6cfce850ff56..afa87ee82d3f 100644 --- a/website/www/site/content/en/documentation/resources/learning-resources.md +++ b/website/www/site/content/en/documentation/resources/learning-resources.md @@ -145,7 +145,7 @@ complexity. Beam Katas are available for both Java and Python SDKs. * **[Java API Reference](https://beam.apache.org/documentation/sdks/javadoc/)** - Official API Reference for the Java SDK. * **[Python API Reference](https://beam.apache.org/documentation/sdks/pydoc/)** - Official API Reference for the Python SDK. -* **[Go API Reference](https://pkg.go.dev/github.com/apache/beam/sdks/go/pkg/beam)** - Official API Reference for the Go SDK. +* **[Go API Reference](https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam)** - Official API Reference for the Go SDK. ## Feedback and Suggestions {#feedback-and-suggestions} diff --git a/website/www/site/content/en/documentation/sdks/go.md b/website/www/site/content/en/documentation/sdks/go.md index 6d2c9efe7258..7e0c73935542 100644 --- a/website/www/site/content/en/documentation/sdks/go.md +++ b/website/www/site/content/en/documentation/sdks/go.md @@ -24,7 +24,7 @@ It is based on the following [design](https://s.apache.org/beam-go-sdk-design-rf Get started with the [Beam Go SDK quickstart](/get-started/quickstart-go) to set up your development environment and run an example pipeline. Then, read through the [Beam programming guide](/documentation/programming-guide) to learn the basic concepts that apply to all SDKs in Beam. -See the [godoc](https://pkg.go.dev/github.com/apache/beam/sdks/go/pkg/beam) for more detailed information. +See the [godoc](https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam) for more detailed information. ## Status diff --git a/website/www/site/content/en/get-started/quickstart-go.md b/website/www/site/content/en/get-started/quickstart-go.md index 3d5a13c198e5..a9d6e84eed45 100644 --- a/website/www/site/content/en/get-started/quickstart-go.md +++ b/website/www/site/content/en/get-started/quickstart-go.md @@ -51,12 +51,12 @@ required arguments described in the examples. For example, to run `wordcount`, run: {{< runner direct >}} -$ go install github.com/apache/beam/sdks/go/examples/wordcount +$ go install github.com/apache/beam/sdks/v2/go/examples/wordcount $ wordcount --input --output counts {{< /runner >}} {{< runner dataflow >}} -$ go install github.com/apache/beam/sdks/go/examples/wordcount +$ go install github.com/apache/beam/sdks/v2/go/examples/wordcount # As part of the initial setup, for non linux users - install package unix before run $ go get -u golang.org/x/sys/unix $ wordcount --input gs://dataflow-samples/shakespeare/kinglear.txt \ @@ -75,7 +75,7 @@ $ wordcount --input gs://dataflow-samples/shakespeare/kinglear.txt \ $ ./gradlew :runners:spark:3:job-server:runShadow -PsparkMasterUrl=spark://localhost:7077 # In a separate terminal, run: -$ go install github.com/apache/beam/sdks/go/examples/wordcount +$ go install github.com/apache/beam/sdks/v2/go/examples/wordcount $ wordcount --input \ --output counts \ --runner spark \ @@ -85,7 +85,7 @@ $ wordcount --input \ ## Next Steps * Learn more about the [Beam SDK for Go](/documentation/sdks/go/) - and look through the [godoc](https://pkg.go.dev/github.com/apache/beam/sdks/go/pkg/beam). + and look through the [godoc](https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam). * Walk through these WordCount examples in the [WordCount Example Walkthrough](/get-started/wordcount-example). * Take a self-paced tour through our [Learning Resources](/documentation/resources/learning-resources). * Dive in to some of our favorite [Videos and Podcasts](/documentation/resources/videos-and-podcasts). diff --git a/website/www/site/content/en/get-started/try-apache-beam.md b/website/www/site/content/en/get-started/try-apache-beam.md index e7d05c71b912..80682e303ae6 100644 --- a/website/www/site/content/en/get-started/try-apache-beam.md +++ b/website/www/site/content/en/get-started/try-apache-beam.md @@ -121,12 +121,12 @@ import ( "fmt" "regexp" - "github.com/apache/beam/sdks/go/pkg/beam" - "github.com/apache/beam/sdks/go/pkg/beam/io/textio" - "github.com/apache/beam/sdks/go/pkg/beam/runners/direct" - "github.com/apache/beam/sdks/go/pkg/beam/transforms/stats" + "github.com/apache/beam/sdks/v2/go/pkg/beam" + "github.com/apache/beam/sdks/v2/go/pkg/beam/io/textio" + "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/direct" + "github.com/apache/beam/sdks/v2/go/pkg/beam/transforms/stats" - _ "github.com/apache/beam/sdks/go/pkg/beam/io/filesystem/local" + _ "github.com/apache/beam/sdks/v2/go/pkg/beam/io/filesystem/local" ) var ( diff --git a/website/www/site/content/en/get-started/wordcount-example.md b/website/www/site/content/en/get-started/wordcount-example.md index 5faf05f5e697..465b2c25bd07 100644 --- a/website/www/site/content/en/get-started/wordcount-example.md +++ b/website/www/site/content/en/get-started/wordcount-example.md @@ -65,7 +65,7 @@ python -m apache_beam.examples.wordcount_minimal --input YOUR_INPUT_FILE --outpu {{< /highlight >}} {{< highlight go >}} -$ go install github.com/apache/beam/sdks/go/examples/minimal_wordcount +$ go install github.com/apache/beam/sdks/v2/go/examples/minimal_wordcount $ minimal_wordcount {{< /highlight >}} @@ -434,7 +434,7 @@ To view the full code in Python, see **To run this example in Go:** {{< runner direct >}} -$ go install github.com/apache/beam/sdks/go/examples/wordcount +$ go install github.com/apache/beam/sdks/v2/go/examples/wordcount $ wordcount --input --output counts {{< /runner >}} @@ -451,7 +451,7 @@ This runner is not yet available for the Go SDK. {{< /runner >}} {{< runner dataflow >}} -$ go install github.com/apache/beam/sdks/go/examples/wordcount +$ go install github.com/apache/beam/sdks/v2/go/examples/wordcount # As part of the initial setup, for non linux users - install package unix before run $ go get -u golang.org/x/sys/unix $ wordcount --input gs://dataflow-samples/shakespeare/kinglear.txt \ @@ -760,7 +760,7 @@ To view the full code in Python, see **To run this example in Go:** {{< runner direct >}} -$ go install github.com/apache/beam/sdks/go/examples/debugging_wordcount +$ go install github.com/apache/beam/sdks/v2/go/examples/debugging_wordcount $ debugging_wordcount --input --output counts {{< /runner >}} @@ -777,7 +777,7 @@ This runner is not yet available for the Go SDK. {{< /runner >}} {{< runner dataflow >}} -$ go install github.com/apache/beam/sdks/go/examples/debugging_wordcount +$ go install github.com/apache/beam/sdks/v2/go/examples/debugging_wordcount # As part of the initial setup, for non linux users - install package unix before run $ go get -u golang.org/x/sys/unix $ debugging_wordcount --input gs://dataflow-samples/shakespeare/kinglear.txt \ @@ -1088,7 +1088,7 @@ To view the full code in Python, see **To run this example in Go:** {{< runner direct >}} -$ go install github.com/apache/beam/sdks/go/examples/windowed_wordcount +$ go install github.com/apache/beam/sdks/v2/go/examples/windowed_wordcount $ windowed_wordcount --input --output counts {{< /runner >}} @@ -1105,7 +1105,7 @@ This runner is not yet available for the Go SDK. {{< /runner >}} {{< runner dataflow >}} -$ go install github.com/apache/beam/sdks/go/examples/windowed_wordcount +$ go install github.com/apache/beam/sdks/v2/go/examples/windowed_wordcount # As part of the initial setup, for non linux users - install package unix before run $ go get -u golang.org/x/sys/unix $ windowed_wordcount --input gs://dataflow-samples/shakespeare/kinglear.txt \ From 2cf9a2fbe6ef4b50d503e548517c44ea02892507 Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Thu, 3 Mar 2022 14:49:58 -0500 Subject: [PATCH 27/68] [BEAM-13885] Add unit tests to window package (#16971) --- sdks/go/pkg/beam/core/graph/window/fn_test.go | 92 +++++++++++ sdks/go/pkg/beam/core/graph/window/windows.go | 7 + .../beam/core/graph/window/windows_test.go | 154 ++++++++++++++++++ 3 files changed, 253 insertions(+) create mode 100644 sdks/go/pkg/beam/core/graph/window/fn_test.go create mode 100644 sdks/go/pkg/beam/core/graph/window/windows_test.go diff --git a/sdks/go/pkg/beam/core/graph/window/fn_test.go b/sdks/go/pkg/beam/core/graph/window/fn_test.go new file mode 100644 index 000000000000..6baa37c41ea0 --- /dev/null +++ b/sdks/go/pkg/beam/core/graph/window/fn_test.go @@ -0,0 +1,92 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package window + +import ( + "testing" + "time" +) + +func TestEquals(t *testing.T) { + tests := []struct { + name string + fnOne *Fn + fnTwo *Fn + expEquality bool + }{ + { + "global equal", + NewGlobalWindows(), + NewGlobalWindows(), + true, + }, + { + "fixed equal", + NewFixedWindows(100 * time.Millisecond), + NewFixedWindows(100 * time.Millisecond), + true, + }, + { + "fixed inequal duration", + NewFixedWindows(100 * time.Millisecond), + NewFixedWindows(200 * time.Millisecond), + false, + }, + { + "sliding equal", + NewSlidingWindows(10*time.Millisecond, 100*time.Millisecond), + NewSlidingWindows(10*time.Millisecond, 100*time.Millisecond), + true, + }, + { + "sliding inequal period", + NewSlidingWindows(10*time.Millisecond, 100*time.Millisecond), + NewSlidingWindows(20*time.Millisecond, 100*time.Millisecond), + false, + }, + { + "sliding inequal duration", + NewSlidingWindows(10*time.Millisecond, 100*time.Millisecond), + NewSlidingWindows(10*time.Millisecond, 110*time.Millisecond), + false, + }, + { + "session equal", + NewSessions(10 * time.Minute), + NewSessions(10 * time.Minute), + true, + }, + { + "session inequal gap", + NewSessions(5 * time.Minute), + NewSessions(10 * time.Minute), + false, + }, + { + "mismatched type", + NewFixedWindows(100 * time.Millisecond), + NewSlidingWindows(10*time.Millisecond, 100*time.Millisecond), + false, + }, + } + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + if got, want := test.fnOne.Equals(test.fnTwo), test.expEquality; got != want { + t.Errorf("(%v).Equals(%v) got %v, want %v", test.fnOne, test.fnTwo, got, want) + } + }) + } +} diff --git a/sdks/go/pkg/beam/core/graph/window/windows.go b/sdks/go/pkg/beam/core/graph/window/windows.go index 1a0643b802f7..80dfb1109676 100644 --- a/sdks/go/pkg/beam/core/graph/window/windows.go +++ b/sdks/go/pkg/beam/core/graph/window/windows.go @@ -35,6 +35,9 @@ func (GlobalWindow) MaxTimestamp() typex.EventTime { return mtime.EndOfGlobalWindowTime } +// Equals returns a boolean indicating if the window is equal to +// a given window. This is true for global windows if the provided +// window is also a global window. func (GlobalWindow) Equals(o typex.Window) bool { _, ok := o.(GlobalWindow) return ok @@ -54,6 +57,10 @@ func (w IntervalWindow) MaxTimestamp() typex.EventTime { return typex.EventTime(mtime.Time(w.End).Milliseconds() - 1) } +// Equals returns a boolean indicating if the window is equal to +// a given window. This is true for interval windows if the provided +// window is an interval window and they share the start and end +// timestamps. func (w IntervalWindow) Equals(o typex.Window) bool { ow, ok := o.(IntervalWindow) return ok && w.Start == ow.Start && w.End == ow.End diff --git a/sdks/go/pkg/beam/core/graph/window/windows_test.go b/sdks/go/pkg/beam/core/graph/window/windows_test.go new file mode 100644 index 000000000000..0b318434da4d --- /dev/null +++ b/sdks/go/pkg/beam/core/graph/window/windows_test.go @@ -0,0 +1,154 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package window + +import ( + "testing" + + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/typex" +) + +func TestWindowEquality(t *testing.T) { + tests := []struct { + name string + windowOne typex.Window + windowTwo typex.Window + expEquality bool + }{ + { + "global window == global window", + GlobalWindow{}, + GlobalWindow{}, + true, + }, + { + "interval window[0,10] == interval window[0,10]", + IntervalWindow{Start: 0, End: 10}, + IntervalWindow{Start: 0, End: 10}, + true, + }, + { + "interval window[0,10] == interval window[11 20]", + IntervalWindow{Start: 0, End: 10}, + IntervalWindow{Start: 11, End: 20}, + false, + }, + { + "interval window[0,10] == interval window[0,20]", + IntervalWindow{Start: 0, End: 10}, + IntervalWindow{Start: 0, End: 20}, + false, + }, + { + "interval window[0,10] == interval window[5,10]", + IntervalWindow{Start: 0, End: 10}, + IntervalWindow{Start: 5, End: 10}, + false, + }, + { + "global window == interval window[0,10]", + GlobalWindow{}, + IntervalWindow{Start: 0, End: 10}, + false, + }, + } + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + if got, want := test.windowOne.Equals(test.windowTwo), test.expEquality; got != want { + t.Errorf("(%v).Equals(%v) got %v, want %v", test.windowOne, test.windowTwo, got, want) + } + }) + } +} + +func TestMaxTimestamp(t *testing.T) { + tests := []struct { + name string + window typex.Window + expTimestamp typex.EventTime + }{ + { + "global window", + GlobalWindow{}, + mtime.EndOfGlobalWindowTime, + }, + { + "interval window[0,100]", + IntervalWindow{Start: 0, End: 100}, + 99, + }, + { + "interval window[-100, 0]", + IntervalWindow{Start: -100, End: 0}, + -1, + }, + } + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + if got, want := test.window.MaxTimestamp(), test.expTimestamp; got != want { + t.Errorf("(%v).MaxTimestamp got %d, want %d", test.window, got, want) + } + }) + } +} + +func TestIsEqualList(t *testing.T) { + tests := []struct { + name string + winsOne []typex.Window + winsTwo []typex.Window + expEquality bool + }{ + { + "equal lists", + []typex.Window{GlobalWindow{}, IntervalWindow{Start: 0, End: 10}, IntervalWindow{Start: 10, End: 20}}, + []typex.Window{GlobalWindow{}, IntervalWindow{Start: 0, End: 10}, IntervalWindow{Start: 10, End: 20}}, + true, + }, + { + "out-of-order lists", + []typex.Window{GlobalWindow{}, IntervalWindow{Start: 0, End: 10}, IntervalWindow{Start: 10, End: 20}}, + []typex.Window{IntervalWindow{Start: 0, End: 10}, GlobalWindow{}, IntervalWindow{Start: 10, End: 20}}, + false, + }, + { + "mismatched lengths", + []typex.Window{IntervalWindow{Start: 0, End: 10}, IntervalWindow{Start: 10, End: 20}}, + []typex.Window{IntervalWindow{Start: 0, End: 10}, GlobalWindow{}, IntervalWindow{Start: 10, End: 20}}, + false, + }, + { + "different windows", + []typex.Window{GlobalWindow{}, IntervalWindow{Start: 0, End: 10}, IntervalWindow{Start: 10, End: 20}}, + []typex.Window{GlobalWindow{}, IntervalWindow{Start: 0, End: 20}, IntervalWindow{Start: 20, End: 40}}, + false, + }, + { + "empty lists", + []typex.Window{}, + []typex.Window{}, + true, + }, + } + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + if got, want := IsEqualList(test.winsOne, test.winsTwo), test.expEquality; got != want { + t.Errorf("IsEqualList got %v, want %v", got, want) + } + }) + } +} From a372e4e70ee43b7afebc7a25f1110001ad12d316 Mon Sep 17 00:00:00 2001 From: Pavel Avilov Date: Thu, 3 Mar 2022 22:56:39 +0300 Subject: [PATCH 28/68] Merge pull request #16891 from [BEAM-13872] [Playground] Increase test coverage for the code_processing package * Increase test coverage for the code_processing package * Refactoring code * Add test cases with mock cache * Add test for processCompileSuccess method * Update test names * Refactoring code --- .../code_processing/code_processing_test.go | 705 +++++++++++++++--- .../setup_tools/builder/setup_builder.go | 2 +- 2 files changed, 622 insertions(+), 85 deletions(-) diff --git a/playground/backend/internal/code_processing/code_processing_test.go b/playground/backend/internal/code_processing/code_processing_test.go index cae46b653d06..13eaa1417517 100644 --- a/playground/backend/internal/code_processing/code_processing_test.go +++ b/playground/backend/internal/code_processing/code_processing_test.go @@ -19,13 +19,16 @@ import ( pb "beam.apache.org/playground/backend/internal/api/v1" "beam.apache.org/playground/backend/internal/cache" "beam.apache.org/playground/backend/internal/cache/local" + "beam.apache.org/playground/backend/internal/cache/redis" "beam.apache.org/playground/backend/internal/environment" "beam.apache.org/playground/backend/internal/executors" "beam.apache.org/playground/backend/internal/fs_tool" "beam.apache.org/playground/backend/internal/utils" "beam.apache.org/playground/backend/internal/validators" "context" + "encoding/json" "fmt" + "github.com/go-redis/redismock/v8" "github.com/google/uuid" "go.uber.org/goleak" "io/fs" @@ -43,9 +46,14 @@ const ( javaConfig = "{\n \"compile_cmd\": \"javac\",\n \"run_cmd\": \"java\",\n \"test_cmd\": \"java\",\n \"compile_args\": [\n \"-d\",\n \"bin\",\n \"-classpath\"\n ],\n \"run_args\": [\n \"-cp\",\n \"bin:\"\n ],\n \"test_args\": [\n \"-cp\",\n \"bin:\",\n \"JUnit\"\n ]\n}" pythonConfig = "{\n \"compile_cmd\": \"\",\n \"run_cmd\": \"python3\",\n \"compile_args\": [],\n \"run_args\": []\n}" goConfig = "{\n \"compile_cmd\": \"go\",\n \"run_cmd\": \"\",\n \"compile_args\": [\n \"build\",\n \"-o\",\n \"bin\"\n ],\n \"run_args\": [\n ]\n}" - fileName = "fakeFileName" pipelinesFolder = "executable_files" configFolder = "configs" + resourcesFolder = "resources" + helloWordPython = "if __name__ == \"__main__\":\n print(\"Hello world!\")\n" + helloWordGo = "package main\nimport \"fmt\"\nfunc main() {\n fmt.Println(\"hello world\")\n}\n" + helloWordJava = "class HelloWorld {\n public static void main(String[] args) {\n System.out.println(\"Hello world!\");\n }\n}" + graphFilePath = "resources/graph.dot" + jsonExtension = ".json" ) var opt goleak.Option @@ -61,16 +69,26 @@ func TestMain(m *testing.M) { func setup() { // create configs for java - err := os.MkdirAll("configs", fs.ModePerm) + err := os.MkdirAll(configFolder, fs.ModePerm) if err != nil { panic(err) } - filePath := filepath.Join("configs", pb.Sdk_SDK_JAVA.String()+".json") + filePath := filepath.Join(configFolder, fmt.Sprintf("%s%s", pb.Sdk_SDK_JAVA.String(), jsonExtension)) err = os.WriteFile(filePath, []byte(javaConfig), 0600) if err != nil { panic(err) } + // create dir with graph file + err = os.MkdirAll(resourcesFolder, fs.ModePerm) + if err != nil { + panic(err) + } + err = os.WriteFile(graphFilePath, []byte("graph"), 0600) + if err != nil { + panic(err) + } + path, err := os.Getwd() if err != nil { panic(err) @@ -90,6 +108,10 @@ func teardown() { if err != nil { panic(fmt.Errorf("error during test teardown: %s", err.Error())) } + err = os.RemoveAll(resourcesFolder) + if err != nil { + panic(fmt.Errorf("error during test teardown: %s", err.Error())) + } os.Clearenv() } @@ -99,11 +121,13 @@ func Test_Process(t *testing.T) { if err != nil { panic(err) } - sdkEnv, err := environment.ConfigureBeamEnvs(appEnvs.WorkingDir()) + sdkJavaEnv, err := environment.ConfigureBeamEnvs(appEnvs.WorkingDir()) if err != nil { panic(err) } - + sdkGoEnv := *sdkJavaEnv + sdkGoEnv.ApacheBeamSdk = pb.Sdk_SDK_GO + incorrectGoHelloWord := "package main\nimport \"fmt\"\nfunc main() {\n fmt.Println(\"hello world\").\n}\n" type args struct { ctx context.Context appEnv *environment.ApplicationEnvs @@ -125,7 +149,7 @@ func Test_Process(t *testing.T) { { // Test case with calling processCode method with small timeout. // As a result status into cache should be set as Status_STATUS_RUN_TIMEOUT. - name: "small pipeline execution timeout", + name: "Small pipeline execution timeout", createExecFile: false, code: "", cancelFunc: false, @@ -136,7 +160,7 @@ func Test_Process(t *testing.T) { args: args{ ctx: context.Background(), appEnv: &environment.ApplicationEnvs{}, - sdkEnv: sdkEnv, + sdkEnv: sdkJavaEnv, pipelineId: uuid.New(), pipelineOptions: "", }, @@ -144,7 +168,7 @@ func Test_Process(t *testing.T) { { // Test case with calling processCode method without preparing files with code. // As a result status into cache should be set as Status_STATUS_VALIDATION_ERROR. - name: "validation failed", + name: "Validation failed", createExecFile: false, code: "", cancelFunc: false, @@ -155,7 +179,7 @@ func Test_Process(t *testing.T) { args: args{ ctx: context.Background(), appEnv: appEnvs, - sdkEnv: sdkEnv, + sdkEnv: sdkJavaEnv, pipelineId: uuid.New(), pipelineOptions: "", }, @@ -163,7 +187,7 @@ func Test_Process(t *testing.T) { { // Test case with calling processCode method with incorrect code. // As a result status into cache should be set as Status_STATUS_COMPILE_ERROR. - name: "compilation failed", + name: "Compilation failed", createExecFile: true, code: "MOCK_CODE", cancelFunc: false, @@ -174,7 +198,7 @@ func Test_Process(t *testing.T) { args: args{ ctx: context.Background(), appEnv: appEnvs, - sdkEnv: sdkEnv, + sdkEnv: sdkJavaEnv, pipelineId: uuid.New(), pipelineOptions: "", }, @@ -182,7 +206,7 @@ func Test_Process(t *testing.T) { { // Test case with calling processCode method with incorrect logic into code. // As a result status into cache should be set as Status_STATUS_RUN_ERROR. - name: "run failed", + name: "Run failed", createExecFile: true, code: "class HelloWorld {\n public static void main(String[] args) {\n System.out.println(1/0);\n }\n}", cancelFunc: false, @@ -193,7 +217,7 @@ func Test_Process(t *testing.T) { args: args{ ctx: context.Background(), appEnv: appEnvs, - sdkEnv: sdkEnv, + sdkEnv: sdkJavaEnv, pipelineId: uuid.New(), pipelineOptions: "", }, @@ -201,7 +225,7 @@ func Test_Process(t *testing.T) { { // Test case with calling processCode with canceling code processing. // As a result status into cache should be set as Status_STATUS_CANCELED. - name: "cancel", + name: "Cancel", createExecFile: true, code: "class HelloWorld {\n public static void main(String[] args) {\n while(true){}\n }\n}", cancelFunc: true, @@ -212,7 +236,7 @@ func Test_Process(t *testing.T) { args: args{ ctx: context.Background(), appEnv: appEnvs, - sdkEnv: sdkEnv, + sdkEnv: sdkJavaEnv, pipelineId: uuid.New(), pipelineOptions: "", }, @@ -220,10 +244,10 @@ func Test_Process(t *testing.T) { { // Test case with calling processCode without any error cases. // As a result status into cache should be set as Status_STATUS_FINISHED. - name: "processing complete successfully", + name: "Processing complete successfully on java sdk", createExecFile: true, cancelFunc: false, - code: "class HelloWorld {\n public static void main(String[] args) {\n System.out.println(\"Hello world!\");\n }\n}", + code: helloWordJava, expectedStatus: pb.Status_STATUS_FINISHED, expectedCompileOutput: "", expectedRunOutput: "Hello world!\n", @@ -231,7 +255,26 @@ func Test_Process(t *testing.T) { args: args{ ctx: context.Background(), appEnv: appEnvs, - sdkEnv: sdkEnv, + sdkEnv: sdkJavaEnv, + pipelineId: uuid.New(), + pipelineOptions: "", + }, + }, + { + // Test case with calling processCode method with incorrect go code. + // As a result status into cache should be set as Status_STATUS_PREPARATION_ERROR. + name: "Prepare step failed", + createExecFile: true, + code: incorrectGoHelloWord, + cancelFunc: false, + expectedStatus: pb.Status_STATUS_PREPARATION_ERROR, + expectedCompileOutput: nil, + expectedRunOutput: nil, + expectedRunError: nil, + args: args{ + ctx: context.Background(), + appEnv: appEnvs, + sdkEnv: &sdkGoEnv, pipelineId: uuid.New(), pipelineOptions: "", }, @@ -317,7 +360,7 @@ func TestGetProcessingOutput(t *testing.T) { { // Test case with calling GetProcessingOutput with pipelineId which doesn't contain run output. // As a result, want to receive an error. - name: "get run output with incorrect pipelineId", + name: "Get run output with incorrect pipelineId", args: args{ ctx: context.Background(), cacheService: cacheService, @@ -331,7 +374,7 @@ func TestGetProcessingOutput(t *testing.T) { { // Test case with calling GetProcessingOutput with pipelineId which contains incorrect run output. // As a result, want to receive an error. - name: "get run output with incorrect run output", + name: "Get run output with incorrect run output", args: args{ ctx: context.Background(), cacheService: cacheService, @@ -345,7 +388,7 @@ func TestGetProcessingOutput(t *testing.T) { { // Test case with calling GetProcessingOutput with pipelineId which contains run output. // As a result, want to receive an expected string. - name: "get run output with correct pipelineId", + name: "Get run output with correct pipelineId", args: args{ ctx: context.Background(), cacheService: cacheService, @@ -399,7 +442,7 @@ func TestGetProcessingStatus(t *testing.T) { { // Test case with calling GetProcessingStatus with pipelineId which doesn't contain status. // As a result, want to receive an error. - name: "get status with incorrect pipelineId", + name: "Get status with incorrect pipelineId", args: args{ ctx: context.Background(), cacheService: cacheService, @@ -412,7 +455,7 @@ func TestGetProcessingStatus(t *testing.T) { { // Test case with calling GetProcessingStatus with pipelineId which contains incorrect status value in cache. // As a result, want to receive an error. - name: "get status with incorrect cache value", + name: "Get status with incorrect cache value", args: args{ ctx: context.Background(), cacheService: cacheService, @@ -425,7 +468,7 @@ func TestGetProcessingStatus(t *testing.T) { { // Test case with calling GetProcessingStatus with pipelineId which contains status. // As a result, want to receive an expected status. - name: "get status with correct pipelineId", + name: "Get status with correct pipelineId", args: args{ ctx: context.Background(), cacheService: cacheService, @@ -475,7 +518,7 @@ func TestGetLastIndex(t *testing.T) { { // Test case with calling GetLastIndex with pipelineId which doesn't contain last index. // As a result, want to receive an error. - name: "get last index with incorrect pipelineId", + name: "Get last index with incorrect pipelineId", args: args{ ctx: context.Background(), cacheService: cacheService, @@ -489,7 +532,7 @@ func TestGetLastIndex(t *testing.T) { { // Test case with calling GetLastIndex with pipelineId which contains incorrect status value in cache. // As a result, want to receive an error. - name: "get last index with incorrect cache value", + name: "Get last index with incorrect cache value", args: args{ ctx: context.Background(), cacheService: cacheService, @@ -503,7 +546,7 @@ func TestGetLastIndex(t *testing.T) { { // Test case with calling GetLastIndex with pipelineId which contains last index. // As a result, want to receive an expected last index. - name: "get last index with correct pipelineId", + name: "Get last index with correct pipelineId", args: args{ ctx: context.Background(), cacheService: cacheService, @@ -559,7 +602,7 @@ func Test_getRunOrTestCmd(t *testing.T) { }{ { //Get cmd objects with set run executor - name: "get run cmd", + name: "Get run cmd", args: args{ isUnitTest: false, executor: &runEx, @@ -569,7 +612,7 @@ func Test_getRunOrTestCmd(t *testing.T) { }, { //Get cmd objects with set test executor - name: "get test cmd", + name: "Get test cmd", args: args{ isUnitTest: true, executor: &testEx, @@ -587,12 +630,25 @@ func Test_getRunOrTestCmd(t *testing.T) { } } -func setupBenchmarks(sdk pb.Sdk) { +func getSdkEnv(sdk pb.Sdk) (*environment.BeamEnvs, error) { + setupSDK(sdk) + appEnvs, err := environment.GetApplicationEnvsFromOsEnvs() + if err != nil { + return nil, err + } + sdkEnv, err := environment.ConfigureBeamEnvs(appEnvs.WorkingDir()) + if err != nil { + return nil, err + } + return sdkEnv, nil +} + +func setupSDK(sdk pb.Sdk) { err := os.MkdirAll(configFolder, fs.ModePerm) if err != nil { panic(err) } - filePath := filepath.Join(configFolder, sdk.String()+".json") + filePath := filepath.Join(configFolder, fmt.Sprintf("%s%s", sdk.String(), jsonExtension)) switch sdk { case pb.Sdk_SDK_JAVA: err = os.WriteFile(filePath, []byte(javaConfig), 0600) @@ -640,7 +696,7 @@ func prepareFiles(b *testing.B, pipelineId uuid.UUID, code string, sdk pb.Sdk) * } func Benchmark_ProcessJava(b *testing.B) { - setupBenchmarks(pb.Sdk_SDK_JAVA) + setupSDK(pb.Sdk_SDK_JAVA) defer teardownBenchmarks() appEnv, err := environment.GetApplicationEnvsFromOsEnvs() @@ -670,7 +726,7 @@ func Benchmark_ProcessJava(b *testing.B) { } func Benchmark_ProcessPython(b *testing.B) { - setupBenchmarks(pb.Sdk_SDK_PYTHON) + setupSDK(pb.Sdk_SDK_PYTHON) defer teardownBenchmarks() appEnv, err := environment.GetApplicationEnvsFromOsEnvs() @@ -683,24 +739,24 @@ func Benchmark_ProcessPython(b *testing.B) { } ctx := context.Background() - code := "if __name__ == \"__main__\":\n print(\"Hello world!\")\n" - + wordCountCode := "import argparse\nimport logging\nimport re\n\nimport apache_beam as beam\nfrom apache_beam.io import ReadFromText\nfrom apache_beam.io import WriteToText\nfrom apache_beam.options.pipeline_options import PipelineOptions\nfrom apache_beam.options.pipeline_options import SetupOptions\n\n\nclass WordExtractingDoFn(beam.DoFn):\n \"\"\"Parse each line of input text into words.\"\"\"\n def process(self, element):\n \"\"\"Returns an iterator over the words of this element.\n\n The element is a line of text. If the line is blank, note that, too.\n\n Args:\n element: the element being processed\n\n Returns:\n The processed element.\n \"\"\"\n return re.findall(r'[\\w\\']+', element, re.UNICODE)\n\n\ndef run(argv=None, save_main_session=True):\n \"\"\"Main entry point; defines and runs the wordcount pipeline.\"\"\"\n parser = argparse.ArgumentParser()\n parser.add_argument(\n '--input',\n dest='input',\n default='gs://dataflow-samples/shakespeare/kinglear.txt',\n help='Input file to process.')\n parser.add_argument(\n '--output',\n dest='output',\n required=True,\n help='Output file to write results to.')\n known_args, pipeline_args = parser.parse_known_args(argv)\n\n # We use the save_main_session option because one or more DoFn's in this\n # workflow rely on global context (e.g., a module imported at module level).\n pipeline_options = PipelineOptions(pipeline_args)\n pipeline_options.view_as(SetupOptions).save_main_session = save_main_session\n\n # The pipeline will be run on exiting the with block.\n with beam.Pipeline(options=pipeline_options) as p:\n\n # Read the text file[pattern] into a PCollection.\n lines = p | 'Read' >> ReadFromText(known_args.input)\n\n counts = (\n lines\n | 'Split' >> (beam.ParDo(WordExtractingDoFn()).with_output_types(str))\n | 'PairWithOne' >> beam.Map(lambda x: (x, 1))\n | 'GroupAndSum' >> beam.CombinePerKey(sum))\n\n # Format the counts into a PCollection of strings.\n def format_result(word, count):\n return '%s: %d' % (word, count)\n\n output = counts | 'Format' >> beam.MapTuple(format_result)\n\n # Write the output using a \"Write\" transform that has side effects.\n # pylint: disable=expression-not-assigned\n output | 'Write' >> WriteToText(known_args.output)\n\n\nif __name__ == '__main__':\n logging.getLogger().setLevel(logging.INFO)\n run()" + pipelineOptions := "--output t.txt" b.ResetTimer() for i := 0; i < b.N; i++ { b.StopTimer() pipelineId := uuid.New() - lc := prepareFiles(b, pipelineId, code, pb.Sdk_SDK_PYTHON) + lc := prepareFiles(b, pipelineId, wordCountCode, pb.Sdk_SDK_PYTHON) if err = utils.SetToCache(ctx, cacheService, pipelineId, cache.Canceled, false); err != nil { b.Fatal("error during set cancel flag to cache") } b.StartTimer() - Process(ctx, cacheService, lc, pipelineId, appEnv, sdkEnv, "") + Process(ctx, cacheService, lc, pipelineId, appEnv, sdkEnv, pipelineOptions) } } func Benchmark_ProcessGo(b *testing.B) { - setupBenchmarks(pb.Sdk_SDK_GO) + setupSDK(pb.Sdk_SDK_GO) defer teardownBenchmarks() appEnv, err := environment.GetApplicationEnvsFromOsEnvs() @@ -778,13 +834,13 @@ func Benchmark_GetLastIndex(b *testing.B) { } func Test_validateStep(t *testing.T) { - appEnvs, err := environment.GetApplicationEnvsFromOsEnvs() + javaSdkEnv, err := getSdkEnv(pb.Sdk_SDK_JAVA) if err != nil { panic(err) } - sdkEnv, err := environment.ConfigureBeamEnvs(appEnvs.WorkingDir()) - if err != nil { - panic(err) + incorrectSdkEnv := &environment.BeamEnvs{ + ApacheBeamSdk: pb.Sdk_SDK_UNSPECIFIED, + ExecutorConfig: nil, } type args struct { ctx context.Context @@ -807,13 +863,27 @@ func Test_validateStep(t *testing.T) { ctx: context.Background(), cacheService: cacheService, pipelineId: uuid.New(), - sdkEnv: sdkEnv, + sdkEnv: javaSdkEnv, pipelineLifeCycleCtx: context.Background(), validationResults: &sync.Map{}, cancelChannel: make(chan bool, 1), }, want: 3, - code: "class HelloWorld {\n public static void main(String[] args) {\n System.out.println(\"Hello world!\");\n }\n}", + code: helloWordJava, + }, + { + name: "Test validation step with incorrect sdkEnv", + args: args{ + ctx: context.Background(), + cacheService: cacheService, + pipelineId: uuid.New(), + sdkEnv: incorrectSdkEnv, + pipelineLifeCycleCtx: context.Background(), + validationResults: &sync.Map{}, + cancelChannel: make(chan bool, 1), + }, + want: 0, + code: helloWordJava, }, } for _, tt := range tests { @@ -834,17 +904,18 @@ func Test_validateStep(t *testing.T) { } func Test_prepareStep(t *testing.T) { - appEnvs, err := environment.GetApplicationEnvsFromOsEnvs() + javaSdkEnv, err := getSdkEnv(pb.Sdk_SDK_JAVA) if err != nil { panic(err) } - sdkEnv, err := environment.ConfigureBeamEnvs(appEnvs.WorkingDir()) - if err != nil { - panic(err) + incorrectSdkEnv := &environment.BeamEnvs{ + ApacheBeamSdk: pb.Sdk_SDK_UNSPECIFIED, + ExecutorConfig: nil, } validationResults := sync.Map{} validationResults.Store(validators.UnitTestValidatorName, false) validationResults.Store(validators.KatasValidatorName, false) + pipelineLifeCycleCtx, _ := context.WithTimeout(context.Background(), 1) type args struct { ctx context.Context cacheService cache.Cache @@ -855,10 +926,10 @@ func Test_prepareStep(t *testing.T) { cancelChannel chan bool } tests := []struct { - name string - args args - want *executors.Executor - code string + name string + args args + code string + expectedStatus pb.Status }{ { name: "Test preparer step working without an error", @@ -866,12 +937,41 @@ func Test_prepareStep(t *testing.T) { ctx: context.Background(), cacheService: cacheService, pipelineId: uuid.New(), - sdkEnv: sdkEnv, + sdkEnv: javaSdkEnv, pipelineLifeCycleCtx: context.Background(), validationResults: &validationResults, cancelChannel: make(chan bool, 1), }, - code: "class HelloWorld {\n public static void main(String[] args) {\n System.out.println(\"Hello world!\");\n }\n}", + code: helloWordJava, + expectedStatus: pb.Status_STATUS_COMPILING, + }, + { + name: "Test preparer step working with incorrect sdkEnv", + args: args{ + ctx: context.Background(), + cacheService: cacheService, + pipelineId: uuid.New(), + sdkEnv: incorrectSdkEnv, + pipelineLifeCycleCtx: context.Background(), + validationResults: &validationResults, + cancelChannel: make(chan bool, 1), + }, + code: "", + expectedStatus: pb.Status_STATUS_ERROR, + }, + { + name: "Error during expired context of the example", + args: args{ + ctx: context.Background(), + cacheService: cacheService, + pipelineId: uuid.New(), + sdkEnv: javaSdkEnv, + pipelineLifeCycleCtx: pipelineLifeCycleCtx, + validationResults: &validationResults, + cancelChannel: make(chan bool, 1), + }, + code: "", + expectedStatus: pb.Status_STATUS_RUN_TIMEOUT, }, } for _, tt := range tests { @@ -882,22 +982,25 @@ func Test_prepareStep(t *testing.T) { t.Fatalf("error during prepare folders: %s", err.Error()) } _ = lc.CreateSourceCodeFile(tt.code) - if got := prepareStep(tt.args.ctx, tt.args.cacheService, &lc.Paths, tt.args.pipelineId, tt.args.sdkEnv, tt.args.pipelineLifeCycleCtx, tt.args.validationResults, tt.args.cancelChannel); got == nil { - t.Errorf("prepareStep(): got nil instead of preparer executor") + prepareStep(tt.args.ctx, tt.args.cacheService, &lc.Paths, tt.args.pipelineId, tt.args.sdkEnv, tt.args.pipelineLifeCycleCtx, tt.args.validationResults, tt.args.cancelChannel) + status, _ := cacheService.GetValue(tt.args.ctx, tt.args.pipelineId, cache.Status) + if status != tt.expectedStatus { + t.Errorf("prepareStep: got status = %v, want %v", status, tt.expectedStatus) } }) } } func Test_compileStep(t *testing.T) { - appEnvs, err := environment.GetApplicationEnvsFromOsEnvs() + sdkJavaEnv, err := getSdkEnv(pb.Sdk_SDK_JAVA) if err != nil { panic(err) } - sdkEnv, err := environment.ConfigureBeamEnvs(appEnvs.WorkingDir()) + sdkPythonEnv, err := getSdkEnv(pb.Sdk_SDK_PYTHON) if err != nil { panic(err) } + pipelineLifeCycleCtx, _ := context.WithTimeout(context.Background(), 1) type args struct { ctx context.Context cacheService cache.Cache @@ -908,51 +1011,84 @@ func Test_compileStep(t *testing.T) { cancelChannel chan bool } tests := []struct { - name string - args args - want *executors.Executor - code string + name string + args args + code string + expectedStatus pb.Status }{ { - name: "Test compilation step working without an error", + name: "Test compilation step finishes successfully on java sdk", + args: args{ + ctx: context.Background(), + cacheService: cacheService, + pipelineId: uuid.New(), + sdkEnv: sdkJavaEnv, + isUnitTest: false, + pipelineLifeCycleCtx: context.Background(), + cancelChannel: make(chan bool, 1), + }, + code: helloWordJava, + expectedStatus: pb.Status_STATUS_EXECUTING, + }, + { + name: "Test compilation step finishes successfully on python sdk", args: args{ ctx: context.Background(), cacheService: cacheService, pipelineId: uuid.New(), - sdkEnv: sdkEnv, + sdkEnv: sdkPythonEnv, isUnitTest: false, pipelineLifeCycleCtx: context.Background(), cancelChannel: make(chan bool, 1), }, - want: nil, - code: "class HelloWorld {\n public static void main(String[] args) {\n System.out.println(\"Hello world!\");\n }\n}", + code: helloWordPython, + expectedStatus: pb.Status_STATUS_EXECUTING, + }, + { + name: "Error during expired context of the example", + args: args{ + ctx: context.Background(), + cacheService: cacheService, + pipelineId: uuid.New(), + sdkEnv: sdkJavaEnv, + isUnitTest: false, + pipelineLifeCycleCtx: pipelineLifeCycleCtx, + cancelChannel: make(chan bool, 1), + }, + code: helloWordJava, + expectedStatus: pb.Status_STATUS_RUN_TIMEOUT, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - lc, _ := fs_tool.NewLifeCycle(pb.Sdk_SDK_JAVA, tt.args.pipelineId, filepath.Join(os.Getenv("APP_WORK_DIR"), pipelinesFolder)) + lc, _ := fs_tool.NewLifeCycle(tt.args.sdkEnv.ApacheBeamSdk, tt.args.pipelineId, filepath.Join(os.Getenv("APP_WORK_DIR"), pipelinesFolder)) err := lc.CreateFolders() if err != nil { t.Fatalf("error during prepare folders: %s", err.Error()) } _ = lc.CreateSourceCodeFile(tt.code) - if got := compileStep(tt.args.ctx, tt.args.cacheService, &lc.Paths, tt.args.pipelineId, tt.args.sdkEnv, tt.args.isUnitTest, tt.args.pipelineLifeCycleCtx, tt.args.cancelChannel); got == nil { - t.Errorf("compileStep: got nil instead of compiler executor") + compileStep(tt.args.ctx, tt.args.cacheService, &lc.Paths, tt.args.pipelineId, tt.args.sdkEnv, tt.args.isUnitTest, tt.args.pipelineLifeCycleCtx, tt.args.cancelChannel) + status, _ := cacheService.GetValue(tt.args.ctx, tt.args.pipelineId, cache.Status) + if status != tt.expectedStatus { + t.Errorf("compileStep: got status = %v, want %v", status, tt.expectedStatus) } }) } } func Test_runStep(t *testing.T) { - appEnvs, err := environment.GetApplicationEnvsFromOsEnvs() + sdkJavaEnv, err := getSdkEnv(pb.Sdk_SDK_JAVA) if err != nil { panic(err) } - sdkEnv, err := environment.ConfigureBeamEnvs(appEnvs.WorkingDir()) + sdkPythonEnv, err := getSdkEnv(pb.Sdk_SDK_PYTHON) + if err != nil { + panic(err) + } + sdkGoEnv, err := getSdkEnv(pb.Sdk_SDK_GO) if err != nil { panic(err) } - sdkEnv.ApacheBeamSdk = pb.Sdk_SDK_PYTHON type args struct { ctx context.Context cacheService cache.Cache @@ -962,36 +1098,86 @@ func Test_runStep(t *testing.T) { pipelineOptions string pipelineLifeCycleCtx context.Context cancelChannel chan bool + createExecFile bool } tests := []struct { - name string - args args - code string + name string + args args + code string + expectedStatus pb.Status }{ { - name: "Test run step working without an error", + // Test case with calling runStep method on python sdk. + // cmd.Run return error during saving output. + // As a result, the pipeline status should be Status_STATUS_RUN_ERROR. + name: "Test run step working on python sdk", args: args{ ctx: context.Background(), cacheService: cacheService, - pipelineId: uuid.UUID{}, + pipelineId: uuid.New(), isUnitTest: false, - sdkEnv: sdkEnv, + sdkEnv: sdkPythonEnv, + pipelineOptions: "", + pipelineLifeCycleCtx: context.Background(), + cancelChannel: make(chan bool, 1), + createExecFile: true, + }, + code: helloWordPython, + expectedStatus: pb.Status_STATUS_RUN_ERROR, + }, + { + // Test case with calling runStep method on go sdk. + // cmd.Run return error due to missing executable file. + // As a result, the pipeline status should be Status_STATUS_RUN_ERROR. + name: "Test run step working on go sdk", + args: args{ + ctx: context.Background(), + cacheService: cacheService, + pipelineId: uuid.New(), + isUnitTest: true, + sdkEnv: sdkGoEnv, + pipelineOptions: "", + pipelineLifeCycleCtx: context.Background(), + cancelChannel: make(chan bool, 1), + createExecFile: true, + }, + code: helloWordGo, + expectedStatus: pb.Status_STATUS_RUN_ERROR, + }, + { + // Test case with calling runStep method without preparing files with code. + // As a result, the pipeline status should be Status_STATUS_ERROR. + name: "Test run step without preparing files with code", + args: args{ + ctx: context.Background(), + cacheService: cacheService, + pipelineId: uuid.UUID{}, + isUnitTest: true, + sdkEnv: sdkJavaEnv, pipelineOptions: "", pipelineLifeCycleCtx: context.Background(), cancelChannel: make(chan bool, 1), + createExecFile: false, }, - code: "if __name__ == \"__main__\":\n print(\"Hello world!\")\n", + code: helloWordJava, + expectedStatus: pb.Status_STATUS_ERROR, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - lc, _ := fs_tool.NewLifeCycle(pb.Sdk_SDK_PYTHON, tt.args.pipelineId, filepath.Join(os.Getenv("APP_WORK_DIR"), pipelinesFolder)) - err := lc.CreateFolders() - if err != nil { - t.Fatalf("error during prepare folders: %s", err.Error()) + lc, _ := fs_tool.NewLifeCycle(tt.args.sdkEnv.ApacheBeamSdk, tt.args.pipelineId, filepath.Join(os.Getenv("APP_WORK_DIR"), pipelinesFolder)) + if tt.args.createExecFile { + err := lc.CreateFolders() + if err != nil { + t.Fatalf("error during prepare folders: %s", err.Error()) + } + _ = lc.CreateSourceCodeFile(tt.code) } - _ = lc.CreateSourceCodeFile(tt.code) runStep(tt.args.ctx, tt.args.cacheService, &lc.Paths, tt.args.pipelineId, tt.args.isUnitTest, tt.args.sdkEnv, tt.args.pipelineOptions, tt.args.pipelineLifeCycleCtx, tt.args.cancelChannel) + status, _ := cacheService.GetValue(tt.args.ctx, tt.args.pipelineId, cache.Status) + if status != tt.expectedStatus { + t.Errorf("runStep() got status = %v, want %v", status, tt.expectedStatus) + } }) } } @@ -1004,3 +1190,354 @@ func syncMapLen(syncMap *sync.Map) int { }) return length } + +func TestGetGraph(t *testing.T) { + ctx := context.Background() + pipelineId1 := uuid.New() + graph := "GRAPH" + err := cacheService.SetValue(ctx, pipelineId1, cache.Graph, graph) + if err != nil { + return + } + pipelineId2 := uuid.New() + err = cacheService.SetValue(ctx, pipelineId2, cache.Graph, 1) + if err != nil { + return + } + type args struct { + ctx context.Context + cacheService cache.Cache + key uuid.UUID + errorTitle string + } + tests := []struct { + name string + args args + want string + wantErr bool + }{ + { + name: "Get graph when key exist in cache", + args: args{ + ctx: context.Background(), + cacheService: cacheService, + key: pipelineId1, + errorTitle: "error", + }, + want: graph, + wantErr: false, + }, + { + name: "Get graph when key doesn't exist in cache", + args: args{ + ctx: context.Background(), + cacheService: cacheService, + key: uuid.New(), + errorTitle: "error", + }, + want: "", + wantErr: true, + }, + { + name: "Get graph when value from cache by key couldn't be converted to a string", + args: args{ + ctx: context.Background(), + cacheService: cacheService, + key: pipelineId2, + errorTitle: "error", + }, + want: "", + wantErr: true, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got, err := GetGraph(tt.args.ctx, tt.args.cacheService, tt.args.key, tt.args.errorTitle) + if (err != nil) != tt.wantErr { + t.Errorf("GetGraph error = %v, wantErr %v", err, tt.wantErr) + return + } + if got != tt.want { + t.Errorf("GetGraph got = %v, want %v", got, tt.want) + } + }) + } +} + +func Test_processSetupError(t *testing.T) { + client, mock := redismock.NewClientMock() + pipelineId := uuid.New() + errorMessage := "MOCK_ERROR" + type args struct { + err error + pipelineId uuid.UUID + cacheService cache.Cache + ctxWithTimeout context.Context + } + tests := []struct { + name string + mocks func() + args args + wantErr bool + }{ + { + name: "Error during HSet operation", + mocks: func() { + mock.ExpectHSet(pipelineId.String(), "MOCK_VALUE").SetErr(fmt.Errorf(errorMessage)) + }, + args: args{ + err: fmt.Errorf(errorMessage), + pipelineId: pipelineId, + cacheService: &redis.Cache{ + Client: client, + }, + ctxWithTimeout: nil, + }, + wantErr: true, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tt.mocks() + if err := processSetupError(tt.args.err, tt.args.pipelineId, tt.args.cacheService, tt.args.ctxWithTimeout); (err != nil) != tt.wantErr { + t.Errorf("processSetupError() error = %v, wantErr %v", err, tt.wantErr) + } + }) + } +} + +func Test_processErrorWithSavingOutput(t *testing.T) { + client, mock := redismock.NewClientMock() + pipelineId := uuid.New() + errorMessage := "MOCK_ERROR" + subKey := cache.RunOutput + type args struct { + ctx context.Context + err error + errorOutput []byte + pipelineId uuid.UUID + subKey cache.SubKey + cacheService cache.Cache + errorTitle string + newStatus pb.Status + } + tests := []struct { + name string + mocks func() + args args + wantErr bool + }{ + { + name: "Error during HSet operation", + mocks: func() { + mock.ExpectHSet(pipelineId.String(), subKey).SetErr(fmt.Errorf(errorMessage)) + }, + args: args{ + ctx: context.Background(), + err: fmt.Errorf(errorMessage), + errorOutput: nil, + pipelineId: pipelineId, + subKey: subKey, + cacheService: &redis.Cache{Client: client}, + errorTitle: "", + newStatus: 0, + }, + wantErr: true, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tt.mocks() + if err := processErrorWithSavingOutput(tt.args.ctx, tt.args.err, tt.args.errorOutput, tt.args.pipelineId, tt.args.subKey, tt.args.cacheService, tt.args.errorTitle, tt.args.newStatus); (err != nil) != tt.wantErr { + t.Errorf("processErrorWithSavingOutput() error = %v, wantErr %v", err, tt.wantErr) + } + }) + } +} + +func Test_processRunError(t *testing.T) { + client, mock := redismock.NewClientMock() + pipelineId := uuid.New() + errorMessage := "MOCK_ERROR" + subKey := cache.RunError + errorChannel := make(chan error, 1) + errorChannel <- fmt.Errorf(errorMessage) + type args struct { + ctx context.Context + errorChannel chan error + errorOutput []byte + pipelineId uuid.UUID + cacheService cache.Cache + stopReadLogsChannel chan bool + finishReadLogsChannel chan bool + } + tests := []struct { + name string + mocks func() + args args + wantErr bool + }{ + { + name: "Error during HSet operation", + mocks: func() { + mock.ExpectHSet(pipelineId.String(), subKey).SetErr(fmt.Errorf(errorMessage)) + }, + args: args{ + ctx: context.Background(), + errorChannel: errorChannel, + errorOutput: nil, + pipelineId: pipelineId, + cacheService: &redis.Cache{Client: client}, + stopReadLogsChannel: nil, + finishReadLogsChannel: nil, + }, + wantErr: true, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tt.mocks() + if err := processRunError(tt.args.ctx, tt.args.errorChannel, tt.args.errorOutput, tt.args.pipelineId, tt.args.cacheService, tt.args.stopReadLogsChannel, tt.args.finishReadLogsChannel); (err != nil) != tt.wantErr { + t.Errorf("processRunError() error = %v, wantErr %v", err, tt.wantErr) + } + }) + } +} + +func Test_processCompileSuccess(t *testing.T) { + client, mock := redismock.NewClientMock() + pipelineId := uuid.New() + output := "output" + cacheMock := &redis.Cache{Client: client} + marshalLogs, _ := json.Marshal(cache.Logs) + marshalCompileOutput, _ := json.Marshal(cache.CompileOutput) + marshalRunOutput, _ := json.Marshal(cache.RunOutput) + marshalRunError, _ := json.Marshal(cache.RunError) + outputMarshal, _ := json.Marshal(output) + marshalEmptyString, _ := json.Marshal("") + type args struct { + ctx context.Context + output []byte + pipelineId uuid.UUID + cacheService cache.Cache + } + tests := []struct { + name string + mocks func() + args args + wantErr bool + }{ + { + name: "Error during set value to CompileOutput subKey", + mocks: func() { + }, + args: args{ + ctx: context.Background(), + output: []byte(output), + pipelineId: pipelineId, + cacheService: cacheMock, + }, + wantErr: true, + }, + { + name: "Error during set value to RunOutput subKey", + mocks: func() { + mock.ExpectHSet(pipelineId.String(), marshalCompileOutput, outputMarshal).SetVal(1) + }, + args: args{ + ctx: context.Background(), + output: []byte(output), + pipelineId: pipelineId, + cacheService: cacheMock, + }, + wantErr: true, + }, + { + name: "Error during set value to RunError subKey", + mocks: func() { + mock.ExpectHSet(pipelineId.String(), marshalCompileOutput, outputMarshal).SetVal(1) + mock.ExpectHSet(pipelineId.String(), marshalRunOutput, marshalEmptyString).SetVal(1) + }, + args: args{ + ctx: context.Background(), + output: []byte(output), + pipelineId: pipelineId, + cacheService: cacheMock, + }, + wantErr: true, + }, + { + name: "Error during set value to Logs subKey", + mocks: func() { + mock.ExpectHSet(pipelineId.String(), marshalCompileOutput, outputMarshal).SetVal(1) + mock.ExpectHSet(pipelineId.String(), marshalRunOutput, marshalEmptyString).SetVal(1) + mock.ExpectHSet(pipelineId.String(), marshalRunError, marshalEmptyString).SetVal(1) + }, + args: args{ + ctx: context.Background(), + output: []byte(output), + pipelineId: pipelineId, + cacheService: cacheMock, + }, + wantErr: true, + }, + { + name: "Error during set value to Graph subKey", + mocks: func() { + mock.ExpectHSet(pipelineId.String(), marshalCompileOutput, outputMarshal).SetVal(1) + mock.ExpectHSet(pipelineId.String(), marshalRunOutput, marshalEmptyString).SetVal(1) + mock.ExpectHSet(pipelineId.String(), marshalRunError, marshalEmptyString).SetVal(1) + mock.ExpectHSet(pipelineId.String(), marshalLogs, marshalEmptyString).SetVal(1) + }, + args: args{ + ctx: context.Background(), + output: []byte(output), + pipelineId: pipelineId, + cacheService: cacheMock, + }, + wantErr: true, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tt.mocks() + if err := processCompileSuccess(tt.args.ctx, tt.args.output, tt.args.pipelineId, tt.args.cacheService); (err != nil) != tt.wantErr { + t.Errorf("processCompileSuccess() error = %v, wantErr %v", err, tt.wantErr) + } + }) + } +} + +func Test_readGraphFile(t *testing.T) { + pipelineLifeCycleCtx, _ := context.WithTimeout(context.Background(), 1*time.Second) + type args struct { + pipelineLifeCycleCtx context.Context + backgroundCtx context.Context + cacheService cache.Cache + graphFilePath string + pipelineId uuid.UUID + } + tests := []struct { + name string + args args + }{ + { + name: "Successfully saving the prepared graph to the cache", + args: args{ + pipelineLifeCycleCtx: pipelineLifeCycleCtx, + backgroundCtx: context.Background(), + cacheService: cacheService, + graphFilePath: graphFilePath, + pipelineId: uuid.New(), + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + readGraphFile(tt.args.pipelineLifeCycleCtx, tt.args.backgroundCtx, tt.args.cacheService, tt.args.graphFilePath, tt.args.pipelineId) + if v, _ := cacheService.GetValue(tt.args.backgroundCtx, tt.args.pipelineId, cache.Graph); v == nil { + t.Errorf("readGraphFile() error: the graph was not cached") + } + }) + } +} diff --git a/playground/backend/internal/setup_tools/builder/setup_builder.go b/playground/backend/internal/setup_tools/builder/setup_builder.go index 51031fd98305..ef14d1316cd0 100644 --- a/playground/backend/internal/setup_tools/builder/setup_builder.go +++ b/playground/backend/internal/setup_tools/builder/setup_builder.go @@ -59,7 +59,7 @@ func Preparer(paths *fs_tool.LifeCyclePaths, sdkEnv *environment.BeamEnvs, valRe WithPreparer(). WithSdkPreparers(prep). ExecutorBuilder - return &builder, err + return &builder, nil } // Compiler return executor with set args for compiler From 50bb5c958c686948fd05cf24d7236e4fcf09c36a Mon Sep 17 00:00:00 2001 From: Pavel Avilov Date: Thu, 3 Mar 2022 22:57:15 +0300 Subject: [PATCH 29/68] Merge pull request #16912 from [BEAM-13878] [Playground] Increase test coverage for the fs_tool package * Increase test coverage for the fs_tool package * Rename folder * Remove useless variable * Update test names --- .../backend/internal/fs_tool/fs_test.go | 63 ++++++++++--- .../backend/internal/fs_tool/go_fs_test.go | 2 +- .../backend/internal/fs_tool/java_fs_test.go | 17 +++- .../internal/fs_tool/path_checker_test.go | 93 +++++++++++++++++++ .../internal/fs_tool/python_fs_test.go | 2 +- 5 files changed, 160 insertions(+), 17 deletions(-) create mode 100644 playground/backend/internal/fs_tool/path_checker_test.go diff --git a/playground/backend/internal/fs_tool/fs_test.go b/playground/backend/internal/fs_tool/fs_test.go index 8ff4b0d456e2..df499bb3fc69 100644 --- a/playground/backend/internal/fs_tool/fs_test.go +++ b/playground/backend/internal/fs_tool/fs_test.go @@ -31,10 +31,21 @@ const ( destinationDir = "destinationDir" testFileMode = 0755 pipelinesFolder = "executable_files" + fileName = "file.txt" + emptyFolder = "emptyFolder" ) +func TestMain(m *testing.M) { + err := prepareFiles() + if err != nil { + panic(fmt.Errorf("error during test setup: %s", err.Error())) + } + defer teardownFiles() + m.Run() +} + func prepareFiles() error { - err := os.Mkdir(sourceDir, testFileMode) + err := os.MkdirAll(filepath.Join(sourceDir, emptyFolder), testFileMode) if err != nil { return err } @@ -42,7 +53,7 @@ func prepareFiles() error { if err != nil { return err } - filePath := filepath.Join(sourceDir, "file.txt") + filePath := filepath.Join(sourceDir, fileName) _, err = os.Create(filePath) return err } @@ -67,11 +78,6 @@ func teardownFolders(baseFileFolder string) error { } func TestLifeCycle_CopyFile(t *testing.T) { - if err := prepareFiles(); err != nil { - t.Fatalf("Error during preparing files for test: %s", err) - } - defer teardownFiles() - type fields struct { folderGlobs []string Paths LifeCyclePaths @@ -88,7 +94,7 @@ func TestLifeCycle_CopyFile(t *testing.T) { wantErr bool }{ { - name: "file doesn't exist", + name: "File doesn't exist", fields: fields{ folderGlobs: nil, }, @@ -100,17 +106,29 @@ func TestLifeCycle_CopyFile(t *testing.T) { wantErr: true, }, { - name: "file exists", + name: "File exists", fields: fields{ folderGlobs: nil, }, args: args{ - fileName: "file.txt", + fileName: fileName, sourceDir: sourceDir, destinationDir: destinationDir, }, wantErr: false, }, + { + name: "Copy directory instead of file", + fields: fields{ + folderGlobs: nil, + }, + args: args{ + fileName: emptyFolder, + sourceDir: sourceDir, + destinationDir: destinationDir, + }, + wantErr: true, + }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { @@ -177,7 +195,7 @@ func TestLifeCycle_CreateSourceCodeFile(t *testing.T) { wantErr bool }{ { - name: "source file folder path doesn't exist", + name: "Source file folder path doesn't exist", fields: fields{ Paths: LifeCyclePaths{ AbsoluteSourceFileFolderPath: "src", @@ -185,7 +203,7 @@ func TestLifeCycle_CreateSourceCodeFile(t *testing.T) { }, wantErr: true, }, { - name: "source file folder path exists", + name: "Source file folder path exists", fields: fields{ Paths: LifeCyclePaths{ AbsoluteSourceFileFolderPath: filepath.Join(baseFileFolder, "src"), @@ -343,6 +361,27 @@ func TestNewLifeCycle(t *testing.T) { }, }, }, + { + name: "SCIO LifeCycle", + args: args{ + sdk: pb.Sdk_SDK_SCIO, + pipelineId: pipelineId, + pipelinesFolder: pipelinesFolder, + }, + want: &LifeCycle{ + folderGlobs: []string{baseFileFolder}, + Paths: LifeCyclePaths{ + SourceFileName: fmt.Sprintf("%s%s", pipelineId.String(), scioExecutableFileExtension), + AbsoluteSourceFileFolderPath: baseFileFolder, + AbsoluteSourceFilePath: filepath.Join(baseFileFolder, fmt.Sprintf("%s%s", pipelineId.String(), scioExecutableFileExtension)), + ExecutableFileName: fmt.Sprintf("%s%s", pipelineId.String(), scioExecutableFileExtension), + AbsoluteExecutableFileFolderPath: baseFileFolder, + AbsoluteExecutableFilePath: filepath.Join(baseFileFolder, fmt.Sprintf("%s%s", pipelineId.String(), scioExecutableFileExtension)), + AbsoluteBaseFolderPath: baseFileFolder, + AbsoluteLogFilePath: filepath.Join(baseFileFolder, logFileName), + }, + }, + }, { name: "Unavailable SDK", args: args{ diff --git a/playground/backend/internal/fs_tool/go_fs_test.go b/playground/backend/internal/fs_tool/go_fs_test.go index a0211751d16c..8021e1c906b6 100644 --- a/playground/backend/internal/fs_tool/go_fs_test.go +++ b/playground/backend/internal/fs_tool/go_fs_test.go @@ -42,7 +42,7 @@ func Test_newGoLifeCycle(t *testing.T) { { // Test case with calling newGoLifeCycle method with correct pipelineId and workingDir. // As a result, want to receive an expected go life cycle. - name: "newGoLifeCycle", + name: "NewGoLifeCycle", args: args{ pipelineId: pipelineId, pipelinesFolder: filepath.Join(workingDir, pipelinesFolder), diff --git a/playground/backend/internal/fs_tool/java_fs_test.go b/playground/backend/internal/fs_tool/java_fs_test.go index d51e7c031846..ac2cfb6970c1 100644 --- a/playground/backend/internal/fs_tool/java_fs_test.go +++ b/playground/backend/internal/fs_tool/java_fs_test.go @@ -43,7 +43,7 @@ func Test_newJavaLifeCycle(t *testing.T) { { // Test case with calling newJavaLifeCycle method with correct pipelineId and workingDir. // As a result, want to receive an expected java life cycle. - name: "newJavaLifeCycle", + name: "NewJavaLifeCycle", args: args{ pipelineId: pipelineId, pipelinesFolder: filepath.Join(workingDir, pipelinesFolder), @@ -96,10 +96,21 @@ func Test_executableName(t *testing.T) { want string wantErr bool }{ + { + // Test case with calling sourceFileName method with empty directory. + // As a result, want to receive an error. + name: "Directory is empty", + prepare: func() {}, + args: args{ + executableFolder: filepath.Join(workDir, pipelinesFolder, pipelineId.String(), "bin"), + }, + want: "", + wantErr: true, + }, { // Test case with calling sourceFileName method with correct pipelineId and workingDir. // As a result, want to receive a name that should be executed - name: "get executable name", + name: "Get executable name", prepare: func() { compiled := filepath.Join(workDir, pipelinesFolder, pipelineId.String(), compiledFolderName) filePath := filepath.Join(compiled, "temp.class") @@ -117,7 +128,7 @@ func Test_executableName(t *testing.T) { { // Test case with calling sourceFileName method with wrong directory. // As a result, want to receive an error. - name: "directory doesn't exist", + name: "Directory doesn't exist", prepare: func() {}, args: args{ executableFolder: filepath.Join(workDir, pipelineId.String()), diff --git a/playground/backend/internal/fs_tool/path_checker_test.go b/playground/backend/internal/fs_tool/path_checker_test.go new file mode 100644 index 000000000000..5a7a4e28bd93 --- /dev/null +++ b/playground/backend/internal/fs_tool/path_checker_test.go @@ -0,0 +1,93 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package fs_tool + +import ( + "fmt" + "path/filepath" + "testing" +) + +func TestWrongExtension_Error(t *testing.T) { + errorMessage := "MOCK_ERROR" + type fields struct { + error string + } + tests := []struct { + name string + fields fields + want string + }{ + { + name: "Get error of the WrongExtension", + fields: fields{error: errorMessage}, + want: fmt.Sprintf("File has wrong extension: %v", errorMessage), + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + e := &WrongExtension{ + error: tt.fields.error, + } + if got := e.Error(); got != tt.want { + t.Errorf("Error() = %v, want %v", got, tt.want) + } + }) + } +} + +func TestCheckPathIsValid(t *testing.T) { + type args struct { + args []interface{} + } + tests := []struct { + name string + args args + want bool + wantErr bool + }{ + { + name: "File not exist", + args: args{args: []interface{}{"filePath.txt", ".txt"}}, + want: false, + wantErr: true, + }, + { + name: "Incorrect extension", + args: args{args: []interface{}{filepath.Join(sourceDir, fileName), JavaSourceFileExtension}}, + want: false, + wantErr: true, + }, + { + name: "CheckPathIsValid worked successfully", + args: args{args: []interface{}{filepath.Join(sourceDir, fileName), ".txt"}}, + want: true, + wantErr: false, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got, err := CheckPathIsValid(tt.args.args...) + if (err != nil) != tt.wantErr { + t.Errorf("CheckPathIsValid() error = %v, wantErr %v", err, tt.wantErr) + return + } + if got != tt.want { + t.Errorf("CheckPathIsValid() got = %v, want %v", got, tt.want) + } + }) + } +} diff --git a/playground/backend/internal/fs_tool/python_fs_test.go b/playground/backend/internal/fs_tool/python_fs_test.go index 98ee39e9805d..c881d4640c54 100644 --- a/playground/backend/internal/fs_tool/python_fs_test.go +++ b/playground/backend/internal/fs_tool/python_fs_test.go @@ -40,7 +40,7 @@ func Test_newPythonLifeCycle(t *testing.T) { { // Test case with calling newPythonLifeCycle method with correct pipelineId and workingDir. // As a result, want to receive an expected python life cycle. - name: "newPythonLifeCycle", + name: "NewPythonLifeCycle", args: args{ pipelineId: pipelineId, pipelinesFolder: filepath.Join(workingDir, pipelinesFolder), From 494b4273036c14fda004c0ca8a088aa111a63dc4 Mon Sep 17 00:00:00 2001 From: Pavel Avilov Date: Thu, 3 Mar 2022 22:57:49 +0300 Subject: [PATCH 30/68] Merge pull request #16946 from [BEAM-13873] [Playground] Increase test coverage for the environment package * Increase test coverage for the environment package * Update test names * Refactoring code * Add bucket name to method --- .../internal/environment/application_test.go | 56 +++++- .../backend/internal/environment/beam_test.go | 41 +++- .../environment/environment_service_test.go | 181 ++++++++++++++++-- 3 files changed, 248 insertions(+), 30 deletions(-) diff --git a/playground/backend/internal/environment/application_test.go b/playground/backend/internal/environment/application_test.go index 92f2aeb1ec8d..f81e4a7ae40e 100644 --- a/playground/backend/internal/environment/application_test.go +++ b/playground/backend/internal/environment/application_test.go @@ -33,7 +33,7 @@ func TestNetworkEnvs_Address(t *testing.T) { want string }{ { - name: "ip and port concatenated through ':'", + name: "Ip and port concatenated through ':'", fields: fields{ip: defaultIp, port: defaultPort}, want: fmt.Sprintf("%s:%d", defaultIp, defaultPort), }, @@ -63,7 +63,7 @@ func TestCacheEnvs_CacheType(t *testing.T) { want string }{ { - name: "all success", + name: "Get cache type", fields: fields{ cacheType: "MOCK_CACHE_TYPE", address: "MOCK_ADDRESS", @@ -98,7 +98,7 @@ func TestCacheEnvs_Address(t *testing.T) { want string }{ { - name: "all success", + name: "Get cache address", fields: fields{ cacheType: "MOCK_CACHE_TYPE", address: "MOCK_ADDRESS", @@ -133,7 +133,7 @@ func TestCacheEnvs_KeyExpirationTime(t *testing.T) { want time.Duration }{ { - name: "all success", + name: "Get expiration time for cache keys", fields: fields{ cacheType: "MOCK_CACHE_TYPE", address: "MOCK_ADDRESS", @@ -168,7 +168,7 @@ func TestApplicationEnvs_WorkingDir(t *testing.T) { want string }{ { - name: "all success", + name: "Get working directory", fields: fields{ workingDir: "MOCK_WORKING_DIR", cacheEnvs: &CacheEnvs{}, @@ -203,7 +203,7 @@ func TestApplicationEnvs_CacheEnvs(t *testing.T) { want *CacheEnvs }{ { - name: "all success", + name: "Get cache environment variables", fields: fields{ workingDir: "MOCK_WORKING_DIR", cacheEnvs: &CacheEnvs{}, @@ -238,7 +238,7 @@ func TestApplicationEnvs_PipelineExecuteTimeout(t *testing.T) { want time.Duration }{ { - name: "all success", + name: "Get pipeline execute timeout", fields: fields{ workingDir: "MOCK_WORKING_DIR", cacheEnvs: &CacheEnvs{}, @@ -277,7 +277,7 @@ func TestApplicationEnvs_LaunchSite(t *testing.T) { { // Test case with calling LaunchSite method. // As a result, want to receive an expected launch site. - name: "get launch site", + name: "Get launch site", fields: fields{ workingDir: "", cacheEnvs: &CacheEnvs{}, @@ -318,7 +318,7 @@ func TestApplicationEnvs_GoogleProjectId(t *testing.T) { { // Test case with calling GoogleProjectId method. // As a result, want to receive an expected project id. - name: "get google project id", + name: "Get google project id", fields: fields{ workingDir: "", cacheEnvs: &CacheEnvs{}, @@ -358,7 +358,7 @@ func TestApplicationEnvs_PipelinesFolder(t *testing.T) { { // Test case with calling PipelinesFolder method. // As a result, want to receive an expected name of pipelines folder. - name: "get google project id", + name: "Get name of pipeline folder", fields: fields{ workingDir: "", cacheEnvs: &CacheEnvs{}, @@ -382,3 +382,39 @@ func TestApplicationEnvs_PipelinesFolder(t *testing.T) { }) } } + +func TestNetworkEnvs_Protocol(t *testing.T) { + protocol := "HTTP" + type fields struct { + ip string + port int + protocol string + } + tests := []struct { + name string + fields fields + want string + }{ + { + name: "Get server protocol", + fields: fields{ + ip: "", + port: 0, + protocol: protocol, + }, + want: protocol, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + serverEnvs := &NetworkEnvs{ + ip: tt.fields.ip, + port: tt.fields.port, + protocol: tt.fields.protocol, + } + if got := serverEnvs.Protocol(); got != tt.want { + t.Errorf("Protocol() = %v, want %v", got, tt.want) + } + }) + } +} diff --git a/playground/backend/internal/environment/beam_test.go b/playground/backend/internal/environment/beam_test.go index 53b515b6c4cc..cf6f58bfce0d 100644 --- a/playground/backend/internal/environment/beam_test.go +++ b/playground/backend/internal/environment/beam_test.go @@ -33,7 +33,7 @@ func TestBeamEnvs_PreparedModDir(t *testing.T) { want string }{ { - name: "get path to prepared directory of the go.mod", + name: "Get path to prepared directory of the go.mod", fields: fields{ ApacheBeamSdk: 0, ExecutorConfig: nil, @@ -55,3 +55,42 @@ func TestBeamEnvs_PreparedModDir(t *testing.T) { }) } } + +func TestBeamEnvs_NumOfParallelJobs(t *testing.T) { + numOfParallelJobs := 2 + type fields struct { + ApacheBeamSdk playground.Sdk + ExecutorConfig *ExecutorConfig + preparedModDir string + numOfParallelJobs int + } + tests := []struct { + name string + fields fields + want int + }{ + { + name: "Get the number of parallel jobs", + fields: fields{ + ApacheBeamSdk: 0, + ExecutorConfig: nil, + preparedModDir: "", + numOfParallelJobs: numOfParallelJobs, + }, + want: numOfParallelJobs, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + b := &BeamEnvs{ + ApacheBeamSdk: tt.fields.ApacheBeamSdk, + ExecutorConfig: tt.fields.ExecutorConfig, + preparedModDir: tt.fields.preparedModDir, + numOfParallelJobs: tt.fields.numOfParallelJobs, + } + if got := b.NumOfParallelJobs(); got != tt.want { + t.Errorf("NumOfParallelJobs() = %v, want %v", got, tt.want) + } + }) + } +} diff --git a/playground/backend/internal/environment/environment_service_test.go b/playground/backend/internal/environment/environment_service_test.go index aef7186f9d24..3008e4fa8ffc 100644 --- a/playground/backend/internal/environment/environment_service_test.go +++ b/playground/backend/internal/environment/environment_service_test.go @@ -16,18 +16,23 @@ package environment import ( - playground "beam.apache.org/playground/backend/internal/api/v1" + pb "beam.apache.org/playground/backend/internal/api/v1" "fmt" "io/fs" "os" "path/filepath" "reflect" "testing" + "time" ) const ( javaConfig = "{\n \"compile_cmd\": \"javac\",\n \"run_cmd\": \"java\",\n \"test_cmd\": \"java\",\n \"compile_args\": [\n \"-d\",\n \"bin\",\n \"-classpath\"\n ],\n \"run_args\": [\n \"-cp\",\n \"bin:\"\n ],\n \"test_args\": [\n \"-cp\",\n \"bin:\",\n \"org.junit.runner.JUnitCore\"\n ]\n}" + goConfig = "{\n \"compile_cmd\": \"go\",\n \"run_cmd\": \"\",\n \"test_cmd\": \"go\",\n \"compile_args\": [\n \"build\",\n \"-o\",\n \"bin\"\n ],\n \"run_args\": [\n ],\n \"test_args\": [\n \"test\",\n \"-v\"\n ]\n}\n" + pythonConfig = "{\n \"compile_cmd\": \"\",\n \"run_cmd\": \"python3\",\n \"test_cmd\": \"pytest\",\n \"compile_args\": [],\n \"run_args\": [],\n \"test_args\": []\n}\n" + scioConfig = "{\n \"compile_cmd\": \"\",\n \"run_cmd\": \"sbt\",\n \"test_cmd\": \"sbt\",\n \"compile_args\": [],\n \"run_args\": [\n \"runMain\"\n ],\n \"test_args\": []\n}\n" defaultProjectId = "" + dirPermission = 0600 ) var executorConfig *ExecutorConfig @@ -46,8 +51,14 @@ func setup() error { if err != nil { return err } - filePath := filepath.Join(configFolderName, defaultSdk.String()+jsonExt) - err = os.WriteFile(filePath, []byte(javaConfig), 0600) + javaConfigPath := filepath.Join(configFolderName, defaultSdk.String()+jsonExt) + err = os.WriteFile(javaConfigPath, []byte(javaConfig), dirPermission) + goConfigPath := filepath.Join(configFolderName, pb.Sdk_SDK_GO.String()+jsonExt) + err = os.WriteFile(goConfigPath, []byte(goConfig), dirPermission) + pythonConfigPath := filepath.Join(configFolderName, pb.Sdk_SDK_PYTHON.String()+jsonExt) + err = os.WriteFile(pythonConfigPath, []byte(pythonConfig), dirPermission) + scioConfigPath := filepath.Join(configFolderName, pb.Sdk_SDK_SCIO.String()+jsonExt) + err = os.WriteFile(scioConfigPath, []byte(scioConfig), dirPermission) if err != nil { return err } @@ -90,7 +101,7 @@ func TestNewEnvironment(t *testing.T) { name string want *Environment }{ - {name: "create env service with default envs", want: &Environment{ + {name: "Create env service with default envs", want: &Environment{ NetworkEnvs: *NewNetworkEnvs(defaultIp, defaultPort, defaultProtocol), BeamSdkEnvs: *NewBeamEnvs(defaultSdk, executorConfig, preparedModDir, 0), ApplicationEnvs: *NewApplicationEnvs("/app", defaultLaunchSite, defaultProjectId, defaultPipelinesFolder, &CacheEnvs{defaultCacheType, defaultCacheAddress, defaultCacheKeyExpirationTime}, defaultPipelineExecuteTimeout, defaultBucketName), @@ -118,26 +129,26 @@ func Test_getSdkEnvsFromOsEnvs(t *testing.T) { wantErr bool }{ { - name: "not specified beam sdk key in os envs", - want: NewBeamEnvs(playground.Sdk_SDK_UNSPECIFIED, nil, preparedModDir, defaultNumOfParallelJobs), + name: "Not specified beam sdk key in os envs", + want: NewBeamEnvs(pb.Sdk_SDK_UNSPECIFIED, nil, preparedModDir, defaultNumOfParallelJobs), envsToSet: map[string]string{}, wantErr: false, }, { - name: "default beam envs", + name: "Default beam envs", want: NewBeamEnvs(defaultSdk, executorConfig, preparedModDir, defaultNumOfParallelJobs), envsToSet: map[string]string{beamSdkKey: "SDK_JAVA"}, wantErr: false, }, { - name: "specific sdk key in os envs", + name: "Specific sdk key in os envs", want: NewBeamEnvs(defaultSdk, executorConfig, preparedModDir, defaultNumOfParallelJobs), envsToSet: map[string]string{beamSdkKey: "SDK_JAVA"}, wantErr: false, }, { - name: "wrong sdk key in os envs", - want: NewBeamEnvs(playground.Sdk_SDK_UNSPECIFIED, nil, preparedModDir, defaultNumOfParallelJobs), + name: "Wrong sdk key in os envs", + want: NewBeamEnvs(pb.Sdk_SDK_UNSPECIFIED, nil, preparedModDir, defaultNumOfParallelJobs), envsToSet: map[string]string{beamSdkKey: "SDK_J"}, wantErr: false, }, @@ -168,16 +179,16 @@ func Test_getNetworkEnvsFromOsEnvs(t *testing.T) { wantErr bool }{ { - name: "default values", + name: "Default values", want: NewNetworkEnvs(defaultIp, defaultPort, defaultProtocol), }, { - name: "values from os envs", + name: "Values from os envs", want: NewNetworkEnvs("12.12.12.21", 1234, "TCP"), envsToSet: map[string]string{serverIpKey: "12.12.12.21", serverPortKey: "1234", protocolTypeKey: "TCP"}, }, { - name: "not int port in os env, should be default", + name: "Not int port in os env, should be default", want: nil, envsToSet: map[string]string{serverIpKey: "12.12.12.21", serverPortKey: "1a34"}, wantErr: true, @@ -202,6 +213,8 @@ func Test_getNetworkEnvsFromOsEnvs(t *testing.T) { } func Test_getApplicationEnvsFromOsEnvs(t *testing.T) { + hour := "1h" + convertedTime, _ := time.ParseDuration(hour) tests := []struct { name string want *ApplicationEnvs @@ -209,16 +222,40 @@ func Test_getApplicationEnvsFromOsEnvs(t *testing.T) { envsToSet map[string]string }{ { - name: "working dir is provided", + name: "Working dir is provided", want: NewApplicationEnvs("/app", defaultLaunchSite, defaultProjectId, defaultPipelinesFolder, &CacheEnvs{defaultCacheType, defaultCacheAddress, defaultCacheKeyExpirationTime}, defaultPipelineExecuteTimeout, defaultBucketName), wantErr: false, envsToSet: map[string]string{workingDirKey: "/app", launchSiteKey: defaultLaunchSite, projectIdKey: defaultProjectId}, }, { - name: "working dir isn't provided", + name: "Working dir isn't provided", want: nil, wantErr: true, }, + { + name: "CacheKeyExpirationTimeKey is set with the correct value", + want: NewApplicationEnvs("/app", defaultLaunchSite, defaultProjectId, defaultPipelinesFolder, &CacheEnvs{defaultCacheType, defaultCacheAddress, convertedTime}, defaultPipelineExecuteTimeout, defaultBucketName), + wantErr: false, + envsToSet: map[string]string{workingDirKey: "/app", cacheKeyExpirationTimeKey: hour}, + }, + { + name: "CacheKeyExpirationTimeKey is set with the incorrect value", + want: NewApplicationEnvs("/app", defaultLaunchSite, defaultProjectId, defaultPipelinesFolder, &CacheEnvs{defaultCacheType, defaultCacheAddress, defaultCacheKeyExpirationTime}, defaultPipelineExecuteTimeout, defaultBucketName), + wantErr: false, + envsToSet: map[string]string{workingDirKey: "/app", cacheKeyExpirationTimeKey: "1"}, + }, + { + name: "CacheKeyExpirationTimeKey is set with the correct value", + want: NewApplicationEnvs("/app", defaultLaunchSite, defaultProjectId, defaultPipelinesFolder, &CacheEnvs{defaultCacheType, defaultCacheAddress, defaultCacheKeyExpirationTime}, convertedTime, defaultBucketName), + wantErr: false, + envsToSet: map[string]string{workingDirKey: "/app", pipelineExecuteTimeoutKey: hour}, + }, + { + name: "PipelineExecuteTimeoutKey is set with the incorrect value", + want: NewApplicationEnvs("/app", defaultLaunchSite, defaultProjectId, defaultPipelinesFolder, &CacheEnvs{defaultCacheType, defaultCacheAddress, defaultCacheKeyExpirationTime}, defaultPipelineExecuteTimeout, defaultBucketName), + wantErr: false, + envsToSet: map[string]string{workingDirKey: "/app", pipelineExecuteTimeoutKey: "1"}, + }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { @@ -242,7 +279,7 @@ func Test_getApplicationEnvsFromOsEnvs(t *testing.T) { func Test_createExecutorConfig(t *testing.T) { type args struct { - apacheBeamSdk playground.Sdk + apacheBeamSdk pb.Sdk configPath string } tests := []struct { @@ -252,7 +289,7 @@ func Test_createExecutorConfig(t *testing.T) { wantErr bool }{ { - name: "create executor configuration from json file", + name: "Create executor configuration from json file", args: args{apacheBeamSdk: defaultSdk, configPath: filepath.Join(configFolderName, defaultSdk.String()+jsonExt)}, want: executorConfig, wantErr: false, @@ -283,13 +320,13 @@ func Test_getConfigFromJson(t *testing.T) { wantErr bool }{ { - name: "get object from json", + name: "Get object from json", args: args{filepath.Join(configFolderName, defaultSdk.String()+jsonExt)}, want: NewExecutorConfig("javac", "java", "java", []string{"-d", "bin", "-classpath"}, []string{"-cp", "bin:"}, []string{"-cp", "bin:", "org.junit.runner.JUnitCore"}), wantErr: false, }, { - name: "error if wrong json path", + name: "Error if wrong json path", args: args{filepath.Join("wrong_folder", defaultSdk.String()+jsonExt)}, want: nil, wantErr: true, @@ -308,3 +345,109 @@ func Test_getConfigFromJson(t *testing.T) { }) } } + +func TestConfigureBeamEnvs(t *testing.T) { + workingDir := "./" + modDir := "/modDir" + goExecutorConfig := NewExecutorConfig( + "go", + "", + "go", + []string{"build", "-o", "bin"}, + []string{}, + []string{"test", "-v"}, + ) + pythonExecutorConfig := NewExecutorConfig( + "", + "python3", + "pytest", + []string{}, + []string{}, + []string{}, + ) + scioExecutorConfig := NewExecutorConfig( + "", + "sbt", + "sbt", + []string{}, + []string{"runMain"}, + []string{}, + ) + type args struct { + workingDir string + } + tests := []struct { + name string + args args + want *BeamEnvs + wantErr bool + envsToSet map[string]string + }{ + { + name: "PREPARED_MOD_DIR is not specified in the environment for GO sdk", + args: args{workingDir: workingDir}, + want: nil, + wantErr: true, + envsToSet: map[string]string{beamSdkKey: "SDK_GO"}, + }, + { + name: "BeamSdkKey set to GO sdk", + args: args{workingDir: workingDir}, + want: NewBeamEnvs(pb.Sdk_SDK_GO, goExecutorConfig, modDir, defaultNumOfParallelJobs), + wantErr: false, + envsToSet: map[string]string{beamSdkKey: "SDK_GO", preparedModDirKey: modDir}, + }, + { + name: "Error during creating executable config", + args: args{workingDir: "/app"}, + want: nil, + wantErr: true, + envsToSet: map[string]string{beamSdkKey: "SDK_PYTHON"}, + }, + { + name: "BeamSdkKey set to Python sdk", + want: NewBeamEnvs(pb.Sdk_SDK_PYTHON, pythonExecutorConfig, modDir, defaultNumOfParallelJobs), + wantErr: false, + envsToSet: map[string]string{beamSdkKey: "SDK_PYTHON"}, + }, + { + name: "BeamSdkKey set to SCIO sdk", + want: NewBeamEnvs(pb.Sdk_SDK_SCIO, scioExecutorConfig, modDir, defaultNumOfParallelJobs), + wantErr: false, + envsToSet: map[string]string{beamSdkKey: "SDK_SCIO"}, + }, + { + name: "NumOfParallelJobsKey is set with a positive number", + want: NewBeamEnvs(pb.Sdk_SDK_PYTHON, pythonExecutorConfig, modDir, 1), + wantErr: false, + envsToSet: map[string]string{beamSdkKey: "SDK_PYTHON", numOfParallelJobsKey: "1"}, + }, + { + name: "NumOfParallelJobsKey is set with a negative number", + want: NewBeamEnvs(pb.Sdk_SDK_PYTHON, pythonExecutorConfig, modDir, defaultNumOfParallelJobs), + wantErr: false, + envsToSet: map[string]string{beamSdkKey: "SDK_PYTHON", numOfParallelJobsKey: "-1"}, + }, + { + name: "NumOfParallelJobsKey is set with incorrect value", + want: NewBeamEnvs(pb.Sdk_SDK_PYTHON, pythonExecutorConfig, modDir, defaultNumOfParallelJobs), + wantErr: false, + envsToSet: map[string]string{beamSdkKey: "SDK_PYTHON", numOfParallelJobsKey: "incorrectValue"}, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + if err := setOsEnvs(tt.envsToSet); err != nil { + t.Fatalf("couldn't setup os env") + } + got, err := ConfigureBeamEnvs(tt.args.workingDir) + if (err != nil) != tt.wantErr { + t.Errorf("ConfigureBeamEnvs() error = %v, wantErr %v", err, tt.wantErr) + return + } + if !reflect.DeepEqual(got, tt.want) { + t.Errorf("ConfigureBeamEnvs() got = %v, want %v", got, tt.want) + } + }) + } +} From 88a427611619410516f230b69b15e0664cedf315 Mon Sep 17 00:00:00 2001 From: Aydar Farrakhov Date: Mon, 28 Feb 2022 17:08:54 +0300 Subject: [PATCH 31/68] [BEAM-13999] playground - support vertical orientation for graph --- .../graph/graph_builder/canvas_drawer.dart | 11 +- .../graph/graph_builder/graph_builder.dart | 25 ++-- .../graph_builder/painters/edge_painter.dart | 118 +++++++++++++++--- .../graph_builder/painters/graph_painter.dart | 18 ++- .../lib/modules/output/components/graph.dart | 15 ++- .../output/components/output_area.dart | 22 +++- 6 files changed, 172 insertions(+), 37 deletions(-) diff --git a/playground/frontend/lib/modules/graph/graph_builder/canvas_drawer.dart b/playground/frontend/lib/modules/graph/graph_builder/canvas_drawer.dart index df870aa190a4..59cbda7bb848 100644 --- a/playground/frontend/lib/modules/graph/graph_builder/canvas_drawer.dart +++ b/playground/frontend/lib/modules/graph/graph_builder/canvas_drawer.dart @@ -104,7 +104,7 @@ class CanvasDrawer { canvas.drawCircle(Offset(x1, y1), radius, linePaint); } - drawArrow(double x1, double y1) { + drawRightArrow(double x1, double y1) { final path = Path(); path.moveTo(x1 - kArrowSize, y1 - kArrowSize); path.lineTo(x1 + kArrowSize, y1); @@ -113,6 +113,15 @@ class CanvasDrawer { canvas.drawPath(path, linePaint); } + drawBottomArrow(double x1, double y1) { + final path = Path(); + path.moveTo(x1 - kArrowSize, y1); + path.lineTo(x1, y1 + kArrowSize); + path.lineTo(x1 + kArrowSize, y1); + path.lineTo(x1 - kArrowSize, y1); + canvas.drawPath(path, linePaint); + } + drawRect( double left, double top, diff --git a/playground/frontend/lib/modules/graph/graph_builder/graph_builder.dart b/playground/frontend/lib/modules/graph/graph_builder/graph_builder.dart index 1c78b4f968e7..53901aaa9500 100644 --- a/playground/frontend/lib/modules/graph/graph_builder/graph_builder.dart +++ b/playground/frontend/lib/modules/graph/graph_builder/graph_builder.dart @@ -60,7 +60,7 @@ abstract class GraphBuilder { void parseNextLine(String line); - GraphPainter getPainter() { + GraphPainter getPainter(GraphDirection direction) { final List nodeElements = elements .where((element) => element.type == NodeType.node) .toList() @@ -80,14 +80,21 @@ abstract class GraphBuilder { ); return GraphPainter( + direction: direction, elementsPainter: elements .where((element) => element.type == NodeType.node) - .map((element) => NodeElementPainter( - element: element as Node, - row: nodeToCellMap[element.name]!.row, - column: nodeToCellMap[element.name]!.column, - )) - .toList(), + .map((element) { + final cell = nodeToCellMap[element.name]!; + final row = + direction == GraphDirection.horizontal ? cell.row : cell.column; + final column = + direction == GraphDirection.horizontal ? cell.column : cell.row; + return NodeElementPainter( + element: element as Node, + row: row, + column: column, + ); + }).toList(), edges: edges.map((e) => EdgePainter(e)).toList(), ); } @@ -237,8 +244,8 @@ class PythonGraphBuilder extends GraphBuilder { if (elementsMap[name] != null) { return; } - final label = - name.replaceFirst(kPythonDefaultCollectionLabel, kPythonCollectionLabel); + final label = name.replaceFirst( + kPythonDefaultCollectionLabel, kPythonCollectionLabel); Node node = Node(label: label, depth: 1, name: name); elementsMap[name] = node; elements.add(node); diff --git a/playground/frontend/lib/modules/graph/graph_builder/painters/edge_painter.dart b/playground/frontend/lib/modules/graph/graph_builder/painters/edge_painter.dart index 63be832f6cb5..d67610530104 100644 --- a/playground/frontend/lib/modules/graph/graph_builder/painters/edge_painter.dart +++ b/playground/frontend/lib/modules/graph/graph_builder/painters/edge_painter.dart @@ -20,22 +20,43 @@ import 'dart:math'; import 'package:collection/collection.dart'; import 'package:playground/constants/sizes.dart'; import 'package:playground/modules/graph/graph_builder/canvas_drawer.dart'; +import 'package:playground/modules/graph/graph_builder/painters/graph_painter.dart'; import 'package:playground/modules/graph/graph_builder/painters/node_painter.dart'; import 'package:playground/modules/graph/models/graph.dart'; +const kEdgeSpacing = 2 * kXlSpacing; + class EdgePainter { final Edge edge; EdgePainter(this.edge); void paint( - CanvasDrawer drawer, - Map elementsMap, - Map rowStarts, - Map columnStarts, - Map rowSizes, - Map columnSizes, - ) { + CanvasDrawer drawer, + Map elementsMap, + Map rowStarts, + Map columnStarts, + Map rowSizes, + Map columnSizes, + GraphDirection direction, + ) { + if (direction == GraphDirection.vertical) { + _drawVertical( + drawer, elementsMap, rowStarts, columnStarts, rowSizes, columnSizes); + } else { + _drawHorizontal( + drawer, elementsMap, rowStarts, columnStarts, rowSizes, columnSizes); + } + } + + _drawHorizontal( + CanvasDrawer drawer, + Map elementsMap, + Map rowStarts, + Map columnStarts, + Map rowSizes, + Map columnSizes, + ) { final startNode = elementsMap[edge.startId]!; final endNode = elementsMap[edge.endId]!; final startColumn = startNode.column; @@ -48,16 +69,16 @@ class EdgePainter { var y = startNode.top! + startNode.size!.height / 2; drawer.drawCircle(x, y, 4.0); movePoints.add(Point(x, y)); - // 1. Go to the closest border - x = columnStarts[startColumn]! + columnSizes[startColumn]! + 2 * kXlSpacing; + // 1. Go to the closest border (right) + x = columnStarts[startColumn]! + columnSizes[startColumn]! + kEdgeSpacing; movePoints.add(Point(x, y)); // 2. Go to the correct row - y = rowStarts[endRow]! + rowSizes[endRow]! + 2 * kXlSpacing; + y = rowStarts[endRow]! + rowSizes[endRow]! + kEdgeSpacing; movePoints.add(Point(x, y)); // 3. Go to the correct column - x = columnStarts[endColumn]! - 2 * kXlSpacing; + x = columnStarts[endColumn]! - kEdgeSpacing; movePoints.add(Point(x, y)); // 4. Go to the middle of the row @@ -87,14 +108,83 @@ class EdgePainter { optimizedMovePoints.add(element); }); - drawer.drawArrow( + drawer.drawRightArrow( optimizedMovePoints[0].x + kXlSpacing, optimizedMovePoints[0].y); - optimizedMovePoints.forEachIndexed((index, point) { + _drawLine(drawer, optimizedMovePoints); + } + + _drawVertical( + CanvasDrawer drawer, + Map elementsMap, + Map rowStarts, + Map columnStarts, + Map rowSizes, + Map columnSizes, + ) { + final startNode = elementsMap[edge.startId]!; + final endNode = elementsMap[edge.endId]!; + final startRow = startNode.row; + final endColumn = endNode.column; + final endRow = endNode.row; + + final List> movePoints = []; + + var x = startNode.left! + startNode.size!.width / 2; + var y = startNode.top! + rowSizes[startRow]!; + drawer.drawCircle(x, y, 4.0); + movePoints.add(Point(x, y)); + // 1. Go to the closest border (bottom) + y = rowStarts[startRow]! + rowSizes[startRow]! + kEdgeSpacing; + movePoints.add(Point(x, y)); + + // 2. Go to the correct column + x = columnStarts[endColumn]! + columnSizes[endColumn]! + kEdgeSpacing; + movePoints.add(Point(x, y)); + + // 3. Go to the correct row + y = rowStarts[endRow]! - kEdgeSpacing; + movePoints.add(Point(x, y)); + + // 4. Go to the middle of the column + x = columnStarts[endColumn]! + endNode.size!.width / 2; + movePoints.add(Point(x, y)); + + // 5. Go to the element + y = rowStarts[endRow]!; + movePoints.add(Point(x, y)); + drawer.drawCircle(x, y, 4.0); + + List> optimizedMovePoints = []; + + movePoints.forEachIndexed((index, element) { + if (index == 0 || index == movePoints.length - 1) { + optimizedMovePoints.add(element); + return; + } + if (movePoints[index - 1].x == movePoints[index].x && + movePoints[index].x == movePoints[index + 1].x) { + return; + } + if (movePoints[index - 1].y == movePoints[index].y && + movePoints[index].y == movePoints[index + 1].y) { + return; + } + optimizedMovePoints.add(element); + }); + + drawer.drawBottomArrow( + optimizedMovePoints[0].x, optimizedMovePoints[0].y + kXlSpacing); + + _drawLine(drawer, optimizedMovePoints); + } + + _drawLine(CanvasDrawer drawer, List> points) { + points.forEachIndexed((index, point) { if (index == 0) { return; } - final prevPoint = optimizedMovePoints[index - 1]; + final prevPoint = points[index - 1]; if (edge.isPrimary) { drawer.drawLine(prevPoint.x, prevPoint.y, point.x, point.y); } else { diff --git a/playground/frontend/lib/modules/graph/graph_builder/painters/graph_painter.dart b/playground/frontend/lib/modules/graph/graph_builder/painters/graph_painter.dart index 7e846c4542e1..81d563531630 100644 --- a/playground/frontend/lib/modules/graph/graph_builder/painters/graph_painter.dart +++ b/playground/frontend/lib/modules/graph/graph_builder/painters/graph_painter.dart @@ -25,9 +25,12 @@ import 'package:playground/modules/graph/graph_builder/painters/edge_painter.dar import 'package:playground/modules/graph/graph_builder/painters/node_painter.dart'; import 'package:playground/modules/graph/models/graph.dart'; +enum GraphDirection { vertical, horizontal } + class GraphPainter { final List elementsPainter; final List edges; + final GraphDirection direction; final Map elementsMap = {}; final Map rowSizes = {}; final Map columnSizes = {}; @@ -43,7 +46,11 @@ class GraphPainter { return Size(width, height); } - GraphPainter({required this.elementsPainter, required this.edges}) { + GraphPainter({ + required this.elementsPainter, + required this.edges, + required this.direction, + }) { for (var element in elementsPainter) { elementsMap[element.element.name] = element; } @@ -80,7 +87,14 @@ class GraphPainter { }); for (var element in edges) { element.paint( - drawer, elementsMap, rowStarts, columnStarts, rowSizes, columnSizes); + drawer, + elementsMap, + rowStarts, + columnStarts, + rowSizes, + columnSizes, + direction, + ); } } } diff --git a/playground/frontend/lib/modules/output/components/graph.dart b/playground/frontend/lib/modules/output/components/graph.dart index 62a95968f450..01fcfbd4b08e 100644 --- a/playground/frontend/lib/modules/output/components/graph.dart +++ b/playground/frontend/lib/modules/output/components/graph.dart @@ -42,11 +42,13 @@ class GraphCustomPainter extends CustomPainter { class GraphTab extends StatefulWidget { final String graph; final SDK sdk; + final GraphDirection direction; const GraphTab({ Key? key, required this.graph, required this.sdk, + required this.direction, }) : super(key: key); @override @@ -59,17 +61,20 @@ class _GraphTabState extends State { @override void initState() { if (widget.graph.isNotEmpty) { - graphPainter = - GraphBuilder.parseDot(widget.graph, widget.sdk)?.getPainter(); + graphPainter = GraphBuilder.parseDot(widget.graph, widget.sdk) + ?.getPainter(widget.direction); } super.initState(); } @override void didUpdateWidget(GraphTab oldWidget) { - if (widget.graph.isNotEmpty && oldWidget.graph != widget.graph) { - graphPainter = - GraphBuilder.parseDot(widget.graph, widget.sdk)?.getPainter(); + final graphChanged = + widget.graph.isNotEmpty && oldWidget.graph != widget.graph; + final directionChanged = widget.direction != oldWidget.direction; + if (graphChanged || directionChanged) { + graphPainter = GraphBuilder.parseDot(widget.graph, widget.sdk) + ?.getPainter(widget.direction); } if (widget.graph.isEmpty) { graphPainter = null; diff --git a/playground/frontend/lib/modules/output/components/output_area.dart b/playground/frontend/lib/modules/output/components/output_area.dart index 8ab9763110f9..2bc9af86b9aa 100644 --- a/playground/frontend/lib/modules/output/components/output_area.dart +++ b/playground/frontend/lib/modules/output/components/output_area.dart @@ -17,8 +17,11 @@ */ import 'package:flutter/material.dart'; +import 'package:playground/modules/graph/graph_builder/painters/graph_painter.dart'; import 'package:playground/modules/output/components/graph.dart'; import 'package:playground/modules/output/components/output_result.dart'; +import 'package:playground/modules/output/models/output_placement.dart'; +import 'package:playground/modules/output/models/output_placement_state.dart'; import 'package:playground/pages/playground/states/playground_state.dart'; import 'package:provider/provider.dart'; @@ -36,24 +39,25 @@ class OutputArea extends StatelessWidget { Widget build(BuildContext context) { return Container( color: Theme.of(context).backgroundColor, - child: Consumer( - builder: (context, state, child) { + child: Consumer2( + builder: (context, playgroundState, placementState, child) { return TabBarView( controller: tabController, physics: const NeverScrollableScrollPhysics(), children: [ OutputResult( - text: state.result?.output ?? '', + text: playgroundState.result?.output ?? '', isSelected: tabController.index == 0, ), OutputResult( - text: state.result?.log ?? '', + text: playgroundState.result?.log ?? '', isSelected: tabController.index == 1, ), if (showGraph) GraphTab( - graph: state.result?.graph ?? '', - sdk: state.sdk, + graph: playgroundState.result?.graph ?? '', + sdk: playgroundState.sdk, + direction: _getGraphDirection(placementState.placement), ), ], ); @@ -61,4 +65,10 @@ class OutputArea extends StatelessWidget { ), ); } + + GraphDirection _getGraphDirection(OutputPlacement placement) { + return placement == OutputPlacement.bottom + ? GraphDirection.horizontal + : GraphDirection.vertical; + } } From 9325fe8d9a87243bd612bb52810535ac4d34a40a Mon Sep 17 00:00:00 2001 From: Brian Hulette Date: Thu, 3 Mar 2022 12:16:08 -0800 Subject: [PATCH 32/68] [BEAM-13951] Update mass_comment.py list of Run commands (#16889) * BEAM-13951: Sort run command list * BEAM-13951: Update list * fixup! BEAM-13951: Update list --- release/src/main/scripts/mass_comment.py | 118 ++++++++++++++++++----- 1 file changed, 94 insertions(+), 24 deletions(-) diff --git a/release/src/main/scripts/mass_comment.py b/release/src/main/scripts/mass_comment.py index 866196b69920..ffced67aadc5 100644 --- a/release/src/main/scripts/mass_comment.py +++ b/release/src/main/scripts/mass_comment.py @@ -27,42 +27,112 @@ import requests from datetime import datetime +# This list can be found by querying the Jenkins API, see BEAM-13951 COMMENTS_TO_ADD = [ - "Run Release Gradle Build", - "Run Go PostCommit", - "Run Java PostCommit", - "Run Java Flink PortableValidatesRunner Batch", - "Run Java Flink PortableValidatesRunner Streaming", + "Run CommunityMetrics PreCommit", + "Run Dataflow Runner Nexmark Tests", + "Run Dataflow Runner V2 Java 11 Nexmark Tests", + "Run Dataflow Runner V2 Java 17 Nexmark Tests", + "Run Dataflow Runner V2 Nexmark Tests", + "Run Dataflow Streaming ValidatesRunner", + "Run Dataflow ValidatesRunner Java 11", + "Run Dataflow ValidatesRunner Java 17", "Run Dataflow ValidatesRunner", + "Run Direct Runner Nexmark Tests", + "Run Direct ValidatesRunner Java 11", + "Run Direct ValidatesRunner Java 17", + "Run Direct ValidatesRunner in Java 11", + "Run Direct ValidatesRunner", + "Run Flink Runner Nexmark Tests", + "Run Flink ValidatesRunner Java 11", "Run Flink ValidatesRunner", - "Run Samza ValidatesRunner", - "Run Spark ValidatesRunner", - "Run Java Spark PortableValidatesRunner Batch", - "Run Python Spark ValidatesRunner", - "Run Python Dataflow ValidatesContainer", - "Run Python Dataflow ValidatesRunner", - "Run Python Flink ValidatesRunner", - "Run Twister2 ValidatesRunner", - "Run Python 3.6 PostCommit", - "Run Python 3.7 PostCommit", - "Run Python 3.8 PostCommit", - "Run SQL PostCommit", + "Run Go Flink ValidatesRunner", + "Run Go PostCommit", "Run Go PreCommit", + "Run Go Samza ValidatesRunner", + "Run Go Spark ValidatesRunner", + "Run GoPortable PreCommit", + "Run Java 11 Examples on Dataflow Runner V2", + "Run Java 17 Examples on Dataflow Runner V2", + "Run Java Dataflow V2 ValidatesRunner Streaming", + "Run Java Dataflow V2 ValidatesRunner", + "Run Java Examples on Dataflow Runner V2", + "Run Java Examples_Direct", + "Run Java Examples_Flink", + "Run Java Examples_Spark", + "Run Java Flink PortableValidatesRunner Streaming", + "Run Java Portability examples on Dataflow with Java 11", + "Run Java PostCommit", "Run Java PreCommit", + "Run Java Samza PortableValidatesRunner", + "Run Java Spark PortableValidatesRunner Batch", + "Run Java Spark v2 PortableValidatesRunner Streaming", + "Run Java Spark v3 PortableValidatesRunner Streaming", + "Run Java examples on Dataflow Java 11", + "Run Java examples on Dataflow Java 17", + "Run Java examples on Dataflow with Java 11", "Run Java_Examples_Dataflow PreCommit", - "Run JavaPortabilityApi PreCommit", + "Run Java_Examples_Dataflow_Java11 PreCommit", + "Run Java_Examples_Dataflow_Java17 PreCommit", + "Run Java_PVR_Flink_Batch PreCommit", + "Run Java_PVR_Flink_Docker PreCommit", + "Run Javadoc PostCommit", + "Run Jpms Dataflow Java 11 PostCommit", + "Run Jpms Dataflow Java 17 PostCommit", + "Run Jpms Direct Java 11 PostCommit", + "Run Jpms Direct Java 17 PostCommit", + "Run Jpms Flink Java 11 PostCommit", + "Run Jpms Spark Java 11 PostCommit", + "Run PortableJar_Flink PostCommit", + "Run PortableJar_Spark PostCommit", "Run Portable_Python PreCommit", - "Run PythonLint PreCommit", + "Run PostCommit_Java_Dataflow", + "Run PostCommit_Java_DataflowV2", + "Run PostCommit_Java_Hadoop_Versions", + "Run Python 3.6 PostCommit", + "Run Python 3.7 PostCommit", + "Run Python 3.8 PostCommit", + "Run Python 3.9 PostCommit", + "Run Python Dataflow V2 ValidatesRunner", + "Run Python Dataflow ValidatesContainer", + "Run Python Dataflow ValidatesRunner", + "Run Python Examples_Dataflow", + "Run Python Examples_Direct", + "Run Python Examples_Flink", + "Run Python Examples_Spark", + "Run Python Flink ValidatesRunner", "Run Python PreCommit", - "Run Python DockerBuild PreCommit", + "Run Python Samza ValidatesRunner", + "Run Python Spark ValidatesRunner", + "Run PythonDocker PreCommit", + "Run PythonDocs PreCommit", + "Run PythonFormatter PreCommit", + "Run PythonLint PreCommit", + "Run Python_PVR_Flink PreCommit", + "Run RAT PreCommit", + "Run Release Gradle Build", + "Run SQL PostCommit", + "Run SQL PreCommit", + "Run SQL_Java11 PreCommit", + "Run SQL_Java17 PreCommit", + "Run Samza ValidatesRunner", + "Run Spark Runner Nexmark Tests", + "Run Spark StructuredStreaming ValidatesRunner", + "Run Spark ValidatesRunner Java 11", + "Run Spark ValidatesRunner", + "Run Spotless PreCommit", + "Run Twister2 ValidatesRunner", + "Run Typescript PreCommit", + "Run ULR Loopback ValidatesRunner", + "Run Whitespace PreCommit", "Run XVR_Direct PostCommit", "Run XVR_Flink PostCommit", + "Run XVR_JavaUsingPython_Dataflow PostCommit", + "Run XVR_PythonUsingJavaSQL_Dataflow PostCommit", + "Run XVR_PythonUsingJava_Dataflow PostCommit", + "Run XVR_Samza PostCommit", "Run XVR_Spark PostCommit", "Run XVR_Spark3 PostCommit", - "Run Python Examples_Direct", - "Run Python Examples_Spark", - "Run Python Examples_Flink", - "Run Python Examples_Dataflow", ] From ff6b0d1dd447f0f3a362bb515fef91cd7a3dbdb2 Mon Sep 17 00:00:00 2001 From: brucearctor <5032356+brucearctor@users.noreply.github.com> Date: Thu, 3 Mar 2022 14:34:29 -0800 Subject: [PATCH 33/68] [BEAM-10652] Allow Clustering without Partition in BigQuery (#16578) * [BEAM-10652] removed check that blocked clustering without partitioning * [BEAM-10652] allow clustering without requiring partition * newline * added needed null * remove testClusteringThrowsWithoutPartitioning * update clustering * formatting * now compiles * passes spotless * update doc * focus on single test * spotless * run all ITs * spotless * testing with time partitioning * checking * set clustering independant of partitioning * remove timepart from it * spotless * removed test * added TODO * removed block of unneded code/comment * remove override to v3 coder * Spotless cleanup * re-add override to v3 coder * spotless * adding checksum ( wrong value ) * added needed query var * use tableName as var * DATASET NAME * project name in query * update query * change tests * remove unneeded imports * remove rest of forgotten * add rows * 16000 bytes * bigint * streaming test * spotless * methods * end stream * stream method and naming * nostream * streaming * streamingoptions * without streaming example * string column instead of date -- related to BEAM-13753 * mor strings * spotless * revert, only DEFAULT and FILE_LOADS --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 11 +- .../beam/sdk/io/gcp/bigquery/WriteTables.java | 7 +- .../io/gcp/bigquery/BigQueryClusteringIT.java | 185 ++++++++++++++++++ .../io/gcp/bigquery/BigQueryIOWriteTest.java | 9 - 4 files changed, 191 insertions(+), 21 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryClusteringIT.java diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 97a9bfa1595c..9786d90a272e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -2254,9 +2254,9 @@ public Write withClustering(Clustering clustering) { /** * Allows writing to clustered tables when {@link #to(SerializableFunction)} or {@link * #to(DynamicDestinations)} is used. The returned {@link TableDestination} objects should - * specify the time partitioning and clustering fields per table. If writing to a single table, - * use {@link #withClustering(Clustering)} instead to pass a {@link Clustering} instance that - * specifies the static clustering fields to use. + * specify the clustering fields per table. If writing to a single table, use {@link + * #withClustering(Clustering)} instead to pass a {@link Clustering} instance that specifies the + * static clustering fields to use. * *

Setting this option enables use of {@link TableDestinationCoderV3} which encodes * clustering information. The updated coder is compatible with non-clustered tables, so can be @@ -2687,11 +2687,6 @@ public WriteResult expand(PCollection input) { "The supplied getTableFunction object can directly set TimePartitioning." + " There is no need to call BigQueryIO.Write.withTimePartitioning."); } - if (getClustering() != null && getClustering().getFields() != null) { - checkArgument( - getJsonTimePartitioning() != null, - "Clustering fields can only be set when TimePartitioning is set."); - } DynamicDestinations dynamicDestinations = getDynamicDestinations(); if (dynamicDestinations == null) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java index 3a40b59284b1..737ab4ff41de 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java @@ -485,10 +485,9 @@ private PendingJob startLoad( } if (timePartitioning != null) { loadConfig.setTimePartitioning(timePartitioning); - // only set clustering if timePartitioning is set - if (clustering != null) { - loadConfig.setClustering(clustering); - } + } + if (clustering != null) { + loadConfig.setClustering(clustering); } if (kmsKey != null) { loadConfig.setDestinationEncryptionConfiguration( diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryClusteringIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryClusteringIT.java new file mode 100644 index 000000000000..67777b265885 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryClusteringIT.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.gcp.bigquery; + +import com.google.api.services.bigquery.Bigquery; +import com.google.api.services.bigquery.model.Clustering; +import com.google.api.services.bigquery.model.Table; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import java.math.BigInteger; +import java.util.Arrays; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.gcp.testing.BigqueryClient; +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.ExperimentalOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.TestPipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.ValueInSingleWindow; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Integration test that clusters sample data in BigQuery. */ +@RunWith(JUnit4.class) +public class BigQueryClusteringIT { + private static final Long EXPECTED_BYTES = 16000L; + private static final BigInteger EXPECTED_ROWS = new BigInteger("1000"); + private static final String WEATHER_SAMPLES_TABLE = + "clouddataflow-readonly:samples.weather_stations"; + private static final String DATASET_NAME = "BigQueryClusteringIT"; + private static final Clustering CLUSTERING = + new Clustering().setFields(Arrays.asList("station_number")); + private static final TableSchema SCHEMA = + new TableSchema() + .setFields( + Arrays.asList( + new TableFieldSchema().setName("station_number").setType("INTEGER"), + new TableFieldSchema().setName("date").setType("DATE"))); + + private Bigquery bqClient; + private BigQueryClusteringITOptions options; + + @Before + public void setUp() { + PipelineOptionsFactory.register(BigQueryClusteringITOptions.class); + options = TestPipeline.testingPipelineOptions().as(BigQueryClusteringITOptions.class); + options.setTempLocation(options.getTempRoot() + "/temp-it/"); + bqClient = BigqueryClient.getNewBigqueryClient(options.getAppName()); + } + + /** Customized PipelineOptions for BigQueryClustering Integration Test. */ + public interface BigQueryClusteringITOptions + extends TestPipelineOptions, ExperimentalOptions, BigQueryOptions { + @Description("Table to read from, specified as " + ":.") + @Default.String(WEATHER_SAMPLES_TABLE) + String getBqcInput(); + + void setBqcInput(String value); + } + + static class KeepStationNumberAndConvertDate extends DoFn { + @ProcessElement + public void processElement(ProcessContext c) { + String day = (String) c.element().get("day"); + String month = (String) c.element().get("month"); + String year = (String) c.element().get("year"); + + TableRow row = new TableRow(); + row.set("station_number", c.element().get("station_number")); + row.set("date", String.format("%s-%s-%s", year, month, day)); + c.output(row); + } + } + + static class ClusteredDestinations extends DynamicDestinations { + private final String tableName; + + public ClusteredDestinations(String tableName) { + this.tableName = tableName; + } + + @Override + public @Nullable Coder getDestinationCoder() { + return TableDestinationCoderV3.of(); + } + + @Override + public TableDestination getDestination(ValueInSingleWindow element) { + return new TableDestination( + String.format("%s.%s", DATASET_NAME, tableName), null, null, CLUSTERING); + } + + @Override + public TableDestination getTable(TableDestination destination) { + return destination; + } + + @Override + public TableSchema getSchema(TableDestination destination) { + return SCHEMA; + } + } + + @Test + public void testE2EBigQueryClusteringNoPartitionTableFunction() throws Exception { + String tableName = "weather_stations_clustered_table_function_" + System.currentTimeMillis(); + + Pipeline p = Pipeline.create(options); + + p.apply(BigQueryIO.readTableRows().from(options.getBqcInput())) + .apply(ParDo.of(new KeepStationNumberAndConvertDate())) + .apply( + BigQueryIO.writeTableRows() + .to( + (ValueInSingleWindow vsw) -> + new TableDestination( + String.format("%s.%s", DATASET_NAME, tableName), + null, + null, + CLUSTERING)) + .withClustering(CLUSTERING) + .withSchema(SCHEMA) + .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) + .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE) + .withMethod(BigQueryIO.Write.Method.DEFAULT)); + + p.run().waitUntilFinish(); + + Table table = bqClient.tables().get(options.getProject(), DATASET_NAME, tableName).execute(); + + Assert.assertEquals(CLUSTERING, table.getClustering()); + Assert.assertEquals(EXPECTED_BYTES, table.getNumBytes()); + Assert.assertEquals(EXPECTED_ROWS, table.getNumRows()); + } + + @Test + public void testE2EBigQueryClusteringNoPartitionDynamicDestinations() throws Exception { + String tableName = + "weather_stations_clustered_dynamic_destinations_" + System.currentTimeMillis(); + + Pipeline p = Pipeline.create(options); + + p.apply(BigQueryIO.readTableRows().from(options.getBqcInput())) + .apply(ParDo.of(new KeepStationNumberAndConvertDate())) + .apply( + BigQueryIO.writeTableRows() + .to(new ClusteredDestinations(tableName)) + .withClustering(CLUSTERING) + .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) + .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE) + .withMethod(BigQueryIO.Write.Method.FILE_LOADS)); + + p.run().waitUntilFinish(); + + Table table = bqClient.tables().get(options.getProject(), DATASET_NAME, tableName).execute(); + + Assert.assertEquals(CLUSTERING, table.getClustering()); + Assert.assertEquals(EXPECTED_ROWS, table.getNumRows()); + Assert.assertEquals(EXPECTED_BYTES, table.getNumBytes()); + } +} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java index 7b2df2524174..5bd82e7b128a 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java @@ -534,15 +534,6 @@ public void testClusteringStorageApi() throws Exception { } } - @Test(expected = IllegalArgumentException.class) - public void testClusteringThrowsWithoutPartitioning() throws Exception { - if (useStorageApi || !useStreaming) { - throw new IllegalArgumentException(); - } - p.enableAbandonedNodeEnforcement(false); - testTimePartitioningClustering(Method.STREAMING_INSERTS, false, true); - } - @Test public void testClusteringTableFunction() throws Exception { TableRow row1 = new TableRow().set("date", "2018-01-01").set("number", "1"); From bc2a897643eeca270ff91717684e3500fa4e642f Mon Sep 17 00:00:00 2001 From: Daniel Oliveira Date: Thu, 3 Mar 2022 14:40:04 -0800 Subject: [PATCH 34/68] [BEAM-13857] Add K:V flags for expansion service jars and addresses to Go ITs. (#16908) Adds functionality for running jars to the Go integration test framework, and uses this functionality to implement handling of K:V flags for providing expansion service jars and addresses to the test framework. This means that tests can simply get the address of an expansion service with the appropriate label, and this feature will handle running a jar if necessary, or just using the passed in endpoint otherwise. --- sdks/go/test/integration/expansions.go | 119 ++++++++++++ sdks/go/test/integration/expansions_test.go | 179 ++++++++++++++++++ sdks/go/test/integration/flags.go | 86 ++++++++- .../go/test/integration/internal/jars/jars.go | 64 +++++++ .../go/test/integration/internal/jars/proc.go | 22 +++ .../integration/internal/jars/proc_unix.go | 39 ++++ .../integration/internal/jars/run_nonunix.go | 54 ++++++ .../integration/internal/jars/run_unix.go | 65 +++++++ .../test/integration/internal/ports/ports.go | 30 +++ 9 files changed, 657 insertions(+), 1 deletion(-) create mode 100644 sdks/go/test/integration/expansions.go create mode 100644 sdks/go/test/integration/expansions_test.go create mode 100644 sdks/go/test/integration/internal/jars/jars.go create mode 100644 sdks/go/test/integration/internal/jars/proc.go create mode 100644 sdks/go/test/integration/internal/jars/proc_unix.go create mode 100644 sdks/go/test/integration/internal/jars/run_nonunix.go create mode 100644 sdks/go/test/integration/internal/jars/run_unix.go create mode 100644 sdks/go/test/integration/internal/ports/ports.go diff --git a/sdks/go/test/integration/expansions.go b/sdks/go/test/integration/expansions.go new file mode 100644 index 000000000000..2c9c54a62658 --- /dev/null +++ b/sdks/go/test/integration/expansions.go @@ -0,0 +1,119 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package integration + +import ( + "fmt" + "strconv" + "time" + + "github.com/apache/beam/sdks/v2/go/test/integration/internal/jars" + "github.com/apache/beam/sdks/v2/go/test/integration/internal/ports" +) + +// ExpansionServices is a struct used for getting addresses and starting expansion services, based +// on the --expansion_jar and --expansion_addr flags in this package. The main reason to use this +// instead of accessing the flags directly is to let it handle jar startup and shutdown. +// +// Usage +// +// Create an ExpansionServices object in TestMain with NewExpansionServices. Then use GetAddr for +// every expansion service needed for the test. Call Shutdown on it before finishing TestMain (or +// simply defer a call to it). +// +// ExpansionServices is not concurrency safe, and so a single instance should not be used within +// multiple individual tests, due to the possibility of those tests being run concurrently. It is +// recommended to only use ExpansionServices in TestMain to avoid this. +// +// Example: +// var retCode int +// defer func() { os.Exit(retCode) }() // Defer os.Exit so it happens after other defers. +// services := integration.NewExpansionServices() +// defer func() { services.Shutdown() }() +// addr, err := services.GetAddr("example") +// if err != nil { +// retCode = 1 +// panic(err) +// } +// expansionAddr = addr // Save address to a package-level variable used by tests. +// retCode = ptest.MainRet(m) +type ExpansionServices struct { + addrs map[string]string + jars map[string]string + procs []jars.Process + // Callback for running jars, stored this way for testing purposes. + run func(time.Duration, string, ...string) (jars.Process, error) + waitTime time.Duration // Time to sleep after running jar. Tests can adjust this. +} + +// NewExpansionServices creates and initializes an ExpansionServices instance. +func NewExpansionServices() *ExpansionServices { + return &ExpansionServices{ + addrs: GetExpansionAddrs(), + jars: GetExpansionJars(), + procs: make([]jars.Process, 0), + run: jars.Run, + waitTime: 3 * time.Second, + } +} + +// GetAddr gets the address for the expansion service with the given label. The label corresponds to +// the labels used in the --expansion_jar and --expansion_addr flags. If an expansion service is +// provided as a jar, then that jar will be run to retrieve the address, and the jars are not +// guaranteed to be shut down unless Shutdown is called. +// +// Note: If this function starts a jar, it waits a few seconds for it to initialize. Do not use +// this function if the possibility of a few seconds of latency is not acceptable. +func (es *ExpansionServices) GetAddr(label string) (string, error) { + // Always default to existing address before running a jar. + if addr, ok := es.addrs[label]; ok { + return addr, nil + } + jar, ok := es.jars[label] + if !ok { + err := fmt.Errorf("no --expansion_jar or --expansion_addr flag provided with label \"%s\"", label) + return "", fmt.Errorf("expansion service labeled \"%s\" not found: %w", label, err) + } + + // Start jar on open port. + port, err := ports.GetOpenTCP() + if err != nil { + return "", fmt.Errorf("cannot get open port for expansion service labeled \"%s\": %w", label, err) + } + portStr := strconv.Itoa(port) + + // Run jar and cache its info. + proc, err := es.run(*ExpansionTimeout, jar, portStr) + if err != nil { + return "", fmt.Errorf("cannot run jar for expansion service labeled \"%s\": %w", label, err) + } + time.Sleep(es.waitTime) // Wait a bit for the jar to start. + es.procs = append(es.procs, proc) + addr := "localhost:" + portStr + es.addrs[label] = addr + return addr, nil +} + +// Shutdown shuts down any jars started by the ExpansionServices struct and should get called if it +// was used at all. +func (es *ExpansionServices) Shutdown() { + for _, p := range es.procs { + p.Kill() + } + es.jars = nil + es.addrs = nil + es.procs = nil +} diff --git a/sdks/go/test/integration/expansions_test.go b/sdks/go/test/integration/expansions_test.go new file mode 100644 index 000000000000..99878d0623fd --- /dev/null +++ b/sdks/go/test/integration/expansions_test.go @@ -0,0 +1,179 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package integration + +import ( + "fmt" + "testing" + "time" + + _ "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/dataflow" + _ "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/flink" + _ "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/samza" + _ "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/spark" + "github.com/apache/beam/sdks/v2/go/test/integration/internal/jars" + "github.com/google/go-cmp/cmp" + "github.com/google/go-cmp/cmp/cmpopts" +) + +type testProcess struct { + killed bool + jar string +} + +func (p *testProcess) Kill() error { + p.killed = true + return nil +} + +func failRun(_ time.Duration, _ string, _ ...string) (jars.Process, error) { + return nil, fmt.Errorf("unexpectedly running a jar, failing") +} + +func succeedRun(_ time.Duration, jar string, _ ...string) (jars.Process, error) { + return &testProcess{jar: jar}, nil +} + +// TestExpansionServices_GetAddr_Addresses tests calling GetAddr on provided addresses. +func TestExpansionServices_GetAddr_Addresses(t *testing.T) { + addrsMap := map[string]string{ + "label1": "testAddr1", + "label2": "testAddr2", + "label3": "testAddr3", + } + jarsMap := map[string]string{ + "label2": "jarFilepath2", + } + es := &ExpansionServices{ + addrs: addrsMap, + jars: jarsMap, + procs: make([]jars.Process, 0), + run: failRun, + waitTime: 0, + } + + // Ensure we get the same map we put in, and that addresses take priority over jars if + // both are given for the same label. + for label, wantAddr := range addrsMap { + gotAddr, err := es.GetAddr(label) + if err != nil { + t.Errorf("unexpected error when getting address for \"%v\": %v", label, err) + continue + } + if gotAddr != wantAddr { + t.Errorf("incorrect address for \"%v\", want %v, got %v", label, wantAddr, gotAddr) + } + } + // Check that nonexistent labels fail. + if _, err := es.GetAddr("nonexistent_label"); err == nil { + t.Errorf("did not receive error when calling GetAddr with nonexistent label") + } +} + +// TestExpansionServices_GetAddr_Jars tests calling GetAddr on provided jars. +func TestExpansionServices_GetAddr_Jars(t *testing.T) { + addrsMap := map[string]string{} + jarsMap := map[string]string{ + "label1": "jarFilepath1", + "label2": "jarFilepath2", + "label3": "jarFilepath3", + } + es := &ExpansionServices{ + addrs: addrsMap, + jars: jarsMap, + procs: make([]jars.Process, 0), + run: succeedRun, + waitTime: 0, + } + + // Call GetAddr on each jar twice, checking that the addresses remain consistent. + gotMap := make(map[string]string) + for label := range jarsMap { + gotAddr, err := es.GetAddr(label) + if err != nil { + t.Errorf("unexpected error when getting address for \"%v\": %v", label, err) + continue + } + gotMap[label] = gotAddr + } + for label, gotAddr := range gotMap { + secondAddr, err := es.GetAddr(label) + if err != nil { + t.Errorf("unexpected error when getting address for \"%v\": %v", label, err) + continue + } + if secondAddr != gotAddr { + t.Errorf("getAddr returned different address when called twice for \"%v\", "+ + "attempt 1: %v, attempt 2: %v", label, gotAddr, secondAddr) + } + } + // Check that all jars were run. + gotJars := make([]string, 0) + for _, proc := range es.procs { + testProc := proc.(*testProcess) + gotJars = append(gotJars, testProc.jar) + } + wantJars := make([]string, 0) + for _, jar := range jarsMap { + wantJars = append(wantJars, jar) + } + lessFunc := func(a, b string) bool { return a < b } + if diff := cmp.Diff(wantJars, gotJars, cmpopts.SortSlices(lessFunc)); diff != "" { + t.Errorf("processes in ExpansionServices does not match jars that should be running: diff(-want,+got):\n%v", diff) + } +} + +// TestExpansionServices_Shutdown tests that a shutdown correctly kills all jars started by an +// ExpansionServices. +func TestExpansionServices_Shutdown(t *testing.T) { + addrsMap := map[string]string{} + jarsMap := map[string]string{ + "label1": "jarFilepath1", + "label2": "jarFilepath2", + "label3": "jarFilepath3", + } + es := &ExpansionServices{ + addrs: addrsMap, + jars: jarsMap, + procs: make([]jars.Process, 0), + run: succeedRun, + waitTime: 0, + } + // Call getAddr on each label to run jars. + for label := range addrsMap { + _, err := es.GetAddr(label) + if err != nil { + t.Errorf("unexpected error when getting address for \"%v\": %v", label, err) + continue + } + } + + // Shutdown and confirm that jars are killed and addresses can no longer be retrieved. + procs := es.procs + es.Shutdown() + for _, proc := range procs { + testProc := proc.(*testProcess) + if !testProc.killed { + t.Errorf("process for jar %v was not killed on Shutdown()", testProc.jar) + } + } + for label := range addrsMap { + _, err := es.GetAddr(label) + if err == nil { + t.Errorf("calling GetAddr after Shutdown did not return an error for \"%v\"", label) + } + } +} diff --git a/sdks/go/test/integration/flags.go b/sdks/go/test/integration/flags.go index d1b56b2a9ad5..6275578900ee 100644 --- a/sdks/go/test/integration/flags.go +++ b/sdks/go/test/integration/flags.go @@ -15,7 +15,11 @@ package integration -import "flag" +import ( + "flag" + "fmt" + "strings" +) // The following flags are flags used in one or more integration tests, and that // may be used by scripts that execute "go test ./sdks/go/test/integration/...". @@ -53,4 +57,84 @@ var ( KafkaJarTimeout = flag.String("kafka_jar_timeout", "10m", "Sets an auto-shutdown timeout to the Kafka cluster. "+ "Requires the timeout command to be present in Path, unless the value is set to \"\".") + + // ExpansionJars contains elements in the form "label:jar" describing jar + // filepaths for expansion services to use in integration tests, and the + // corresponding labels. Once provided through this flag, those jars can + // be used in tests via the ExpansionServices struct. + ExpansionJars stringSlice + + // ExpansionAddrs contains elements in the form "label:address" describing + // endpoints for expansion services to use in integration tests, and the + // corresponding labels. Once provided through this flag, those addresses + // can be used in tests via the ExpansionServices struct. + ExpansionAddrs stringSlice + + // ExpansionTimeout attempts to apply an auto-shutdown timeout to any + // expansion services started by integration tests. + ExpansionTimeout = flag.Duration("expansion_timeout", 0, + "Sets an auto-shutdown timeout to any started expansion services. "+ + "Requires the timeout command to be present in Path, unless the value is set to 0.") ) + +func init() { + flag.Var(&ExpansionJars, "expansion_jar", + "Define jar locations for expansion services. Each entry consists of "+ + "two values, an arbitrary label and a jar filepath, separated by a "+ + "\":\", in the form \"label:jar\". Jars provided through this flag "+ + "can be started by tests.") + flag.Var(&ExpansionAddrs, "expansion_addr", + "Define addresses for expansion services. Each entry consists of "+ + "two values, an arbitrary label and an address, separated by a "+ + "\":\", in the form \"label:address\". Addresses provided through "+ + "this flag can be used as expansion addresses by tests.") +} + +// GetExpansionJars gets all the jars given to --expansion_jar as a map of label to jar location. +func GetExpansionJars() map[string]string { + ret := make(map[string]string) + for _, jar := range ExpansionJars { + splits := strings.SplitN(jar, ":", 2) + ret[splits[0]] = splits[1] + } + return ret +} + +// GetExpansionAddrs gets all the addresses given to --expansion_addr as a map of label to address. +func GetExpansionAddrs() map[string]string { + ret := make(map[string]string) + for _, addr := range ExpansionAddrs { + splits := strings.SplitN(addr, ":", 2) + ret[splits[0]] = splits[1] + } + return ret +} + +// stringSlice is a flag.Value implementation for string slices, that allows +// multiple strings to be assigned to one flag by specifying multiple instances +// of the flag. +// +// Example: +// var myFlags stringSlice +// flag.Var(&myFlags, "my_flag", "A list of flags") +// With the example above, the slice can be set to contain ["foo", "bar"]: +// cmd -my_flag foo -my_flag bar +type stringSlice []string + +// String implements the String method of flag.Value. This outputs the value +// of the flag as a string. +func (s *stringSlice) String() string { + return fmt.Sprintf("%v", *s) +} + +// Set implements the Set method of flag.Value. This stores a string input to +// the flag into a stringSlice representation. +func (s *stringSlice) Set(value string) error { + *s = append(*s, value) + return nil +} + +// Get returns the instance itself. +func (s stringSlice) Get() interface{} { + return s +} diff --git a/sdks/go/test/integration/internal/jars/jars.go b/sdks/go/test/integration/internal/jars/jars.go new file mode 100644 index 000000000000..9ac70a274615 --- /dev/null +++ b/sdks/go/test/integration/internal/jars/jars.go @@ -0,0 +1,64 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package jars contains functionality for running jars for integration tests. The main entry point +// for running a jar is the Run function. The Process interface is used to interact with the running +// jars, and most importantly for shutting down the jars once finished with them. +package jars + +import ( + "fmt" + "os/exec" + "time" +) + +type runCallback func(dur time.Duration, jar string, args ...string) (Process, error) + +var runner runCallback // Saves which behavior to use when Run is called. + +func init() { + runner = getRunner() +} + +// getRunner is used to determine the appropriate behavior for run during initialization time, +// based on the OS and installed binaries of the system. This is returned as a runCallback which +// can be called whenever Run is called. If an error prevents Run from being used at all (for +// example, Java is not installed), then the runCallback will return that error. +func getRunner() runCallback { + // First check if we can even run jars. + _, err := exec.LookPath("java") + if err != nil { + err := fmt.Errorf("cannot run jar: 'java' command not installed: %w", err) + return func(_ time.Duration, _ string, _ ...string) (Process, error) { + return nil, err + } + } + + // Defer to OS-specific logic for checking for presence of timeout command. + return getTimeoutRunner() +} + +// Run runs a jar given an optional duration, a path to the jar, and any desired arguments to the +// jar. It returns a Process object which can be used to shut down the jar once finished. +// +// The dur parameter is a duration for the timeout command which can be used to automatically kill +// the jar after a set duration, in order to avoid resource leakage. Timeout is described here: +// https://man7.org/linux/man-pages/man1/timeout.1.html. Durations will be translated from +// time.Duration to a string based on the number of minutes. If a duration is provided but the +// system is unable to use the timeout is unable to use the timeout command, this function will +// return an error. To indicate that a duration isn't needed, pass in 0. +func Run(dur time.Duration, jar string, args ...string) (Process, error) { + return runner(dur, jar, args...) +} diff --git a/sdks/go/test/integration/internal/jars/proc.go b/sdks/go/test/integration/internal/jars/proc.go new file mode 100644 index 000000000000..fa12beeead9a --- /dev/null +++ b/sdks/go/test/integration/internal/jars/proc.go @@ -0,0 +1,22 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package jars + +// Process is an interface to allow wrapping os.Process with alternate behavior +// depending on OS. +type Process interface { + Kill() error +} diff --git a/sdks/go/test/integration/internal/jars/proc_unix.go b/sdks/go/test/integration/internal/jars/proc_unix.go new file mode 100644 index 000000000000..c9f2d9d05bea --- /dev/null +++ b/sdks/go/test/integration/internal/jars/proc_unix.go @@ -0,0 +1,39 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Match build constraints of imported package golang.org/x/sys/unix. +//go:build aix || darwin || dragonfly || freebsd || linux || netbsd || openbsd || solaris || zos +// +build aix darwin dragonfly freebsd linux netbsd openbsd solaris zos + +package jars + +import ( + "golang.org/x/sys/unix" + "os" +) + +// UnixProcess wraps os.Process and changes the Kill function to perform a more +// graceful shutdown, mainly for compatibility with the timeout command. +type UnixProcess struct { + proc *os.Process // The os.Process for the running jar. +} + +// Kill gracefully shuts down the process. It is recommended to use this +// instead of directly killing the process. +func (p *UnixProcess) Kill() error { + // Avoid using SIGKILL. If the jar is wrapped in the timeout command + // SIGKILL will kill the timeout and leave the jar running. + return p.proc.Signal(unix.SIGTERM) +} diff --git a/sdks/go/test/integration/internal/jars/run_nonunix.go b/sdks/go/test/integration/internal/jars/run_nonunix.go new file mode 100644 index 000000000000..189f2ab27209 --- /dev/null +++ b/sdks/go/test/integration/internal/jars/run_nonunix.go @@ -0,0 +1,54 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Exclude build constraints of package golang.org/x/sys/unix. +//go:build !(aix || darwin || dragonfly || freebsd || linux || netbsd || openbsd || solaris || zos) +// +build !aix,!darwin,!dragonfly,!freebsd,!linux,!netbsd,!openbsd,!solaris,!zos + +package jars + +import ( + "fmt" + "runtime" + "time" +) + +// getTimeoutRunner is an OS-specific branch for determining what behavior to use for Run. This +// non-unix version does not handle timeout durations. +func getTimeoutRunner() runCallback { + // Wrap run with error handling for OS that does not support timeout duration. + return func(dur time.Duration, jar string, args ...string) (*Process, error) { + // Currently, we hard-fail here if a duration is provided but timeout is unsupported. If + // we ever decide to soft-fail instead, this is the code to change. + if dur != 0 { + return nil, fmt.Errorf("cannot run jar: duration parameter provided but timeouts are unsupported on os %s", runtime.GOOS) + } + return run(jar, args...) + } +} + +// run simply starts up a jar and returns the cmd.Process. +func run(jar string, args ...string) (Process, error) { + var cmdArr []string + cmdArr = append(cmdArr, "java", "-jar", jar) + cmdArr = append(cmdArr, args...) + + cmd := exec.Command(cmdArr[0], cmdArr[1:]...) + err := cmd.Start() + if err != nil { + return nil, err + } + return cmd.Process, nil +} diff --git a/sdks/go/test/integration/internal/jars/run_unix.go b/sdks/go/test/integration/internal/jars/run_unix.go new file mode 100644 index 000000000000..2a401b888ec8 --- /dev/null +++ b/sdks/go/test/integration/internal/jars/run_unix.go @@ -0,0 +1,65 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Match build constraints of imported package golang.org/x/sys/unix. +//go:build aix || darwin || dragonfly || freebsd || linux || netbsd || openbsd || solaris || zos +// +build aix darwin dragonfly freebsd linux netbsd openbsd solaris zos + +package jars + +import ( + "fmt" + "os/exec" + "time" +) + +// getTimeoutRunner is an OS-specific branch for determining what behavior to use for Run. This +// Unix specific version handles timeout. +func getTimeoutRunner() runCallback { + _, err := exec.LookPath("timeout") + if err != nil { + // Wrap run with Unix-specific error handling for missing timeout command. + return func(dur time.Duration, jar string, args ...string) (Process, error) { + // Currently, we hard-fail here if a duration is provided but timeout is unsupported. If + // we ever decide to soft-fail instead, this is the code to change. + if dur != 0 { + return nil, fmt.Errorf("cannot run jar: duration parameter provided but 'timeout' command not installed: %w", err) + } + return run(dur, jar, args...) + } + } + + // Path for a supported timeout, just use the default run function. + return run +} + +// run starts up a jar, and wraps it in "timeout" only if a duration is provided. Processes are +// returned wrapped as Unix processes that provide graceful shutdown for unix specifically. +func run(dur time.Duration, jar string, args ...string) (Process, error) { + var cmdArr []string + if dur != 0 { + durStr := fmt.Sprintf("%.2fm", dur.Minutes()) + cmdArr = append(cmdArr, "timeout", durStr) + } + cmdArr = append(cmdArr, "java", "-jar", jar) + cmdArr = append(cmdArr, args...) + + cmd := exec.Command(cmdArr[0], cmdArr[1:]...) + err := cmd.Start() + if err != nil { + return nil, err + } + return &UnixProcess{proc: cmd.Process}, nil +} diff --git a/sdks/go/test/integration/internal/ports/ports.go b/sdks/go/test/integration/internal/ports/ports.go new file mode 100644 index 000000000000..895824edf776 --- /dev/null +++ b/sdks/go/test/integration/internal/ports/ports.go @@ -0,0 +1,30 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package ports contains utilities for handling ports needed for integration +// tests. +package ports + +import "net" + +// GetOpenTCP gets an open TCP port and returns it, or an error on failure. +func GetOpenTCP() (int, error) { + listener, err := net.Listen("tcp", ":0") + if err != nil { + return 0, err + } + defer listener.Close() + return listener.Addr().(*net.TCPAddr).Port, nil +} From 37f571a779c2bc3c639c42677eb54e3462bb6f8d Mon Sep 17 00:00:00 2001 From: sp029619 Date: Tue, 1 Mar 2022 08:44:08 -0600 Subject: [PATCH 35/68] BEAM-14011 fix s3 filesystem multipart copy --- CHANGES.md | 2 +- .../beam/sdk/io/aws/s3/S3FileSystem.java | 10 +-- .../beam/sdk/io/aws2/s3/S3FileSystem.java | 32 +++++--- .../beam/sdk/io/aws2/s3/S3FileSystemTest.java | 82 ++++++++++++++----- 4 files changed, 87 insertions(+), 39 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 4c4e411393f8..d391a24816b8 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -79,7 +79,7 @@ ## Bugfixes -* Fixed X (Java/Python) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)). +* Fix S3 copy for large objects (Java) ([BEAM-14011](https://issues.apache.org/jira/browse/BEAM-14011)) ## Known Issues diff --git a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3FileSystem.java b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3FileSystem.java index 024d2861c331..9b52c7c731ab 100644 --- a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3FileSystem.java +++ b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3FileSystem.java @@ -484,8 +484,8 @@ CompleteMultipartUploadResult multipartCopy( InitiateMultipartUploadRequest initiateUploadRequest = new InitiateMultipartUploadRequest(destinationPath.getBucket(), destinationPath.getKey()) .withStorageClass(config.getS3StorageClass()) - .withObjectMetadata(sourceObjectMetadata); - initiateUploadRequest.setSSECustomerKey(config.getSSECustomerKey()); + .withObjectMetadata(sourceObjectMetadata) + .withSSECustomerKey(config.getSSECustomerKey()); InitiateMultipartUploadResult initiateUploadResult = amazonS3.get().initiateMultipartUpload(initiateUploadRequest); @@ -512,7 +512,6 @@ CompleteMultipartUploadResult multipartCopy( eTags.add(copyPartResult.getPartETag()); } else { long bytePosition = 0; - Integer uploadBufferSizeBytes = config.getS3UploadBufferSizeBytes(); // Amazon parts are 1-indexed, not zero-indexed. for (int partNumber = 1; bytePosition < objectSize; partNumber++) { final CopyPartRequest copyPartRequest = @@ -524,14 +523,15 @@ CompleteMultipartUploadResult multipartCopy( .withUploadId(uploadId) .withPartNumber(partNumber) .withFirstByte(bytePosition) - .withLastByte(Math.min(objectSize - 1, bytePosition + uploadBufferSizeBytes - 1)); + .withLastByte( + Math.min(objectSize - 1, bytePosition + MAX_COPY_OBJECT_SIZE_BYTES - 1)); copyPartRequest.setSourceSSECustomerKey(config.getSSECustomerKey()); copyPartRequest.setDestinationSSECustomerKey(config.getSSECustomerKey()); CopyPartResult copyPartResult = amazonS3.get().copyPart(copyPartRequest); eTags.add(copyPartResult.getPartETag()); - bytePosition += uploadBufferSizeBytes; + bytePosition += MAX_COPY_OBJECT_SIZE_BYTES; } } diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/s3/S3FileSystem.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/s3/S3FileSystem.java index 38defa471b78..dcff568e1952 100644 --- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/s3/S3FileSystem.java +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/s3/S3FileSystem.java @@ -98,7 +98,7 @@ class S3FileSystem extends FileSystem { // Amazon S3 API: You can create a copy of your object up to 5 GB in a single atomic operation // Ref. https://docs.aws.amazon.com/AmazonS3/latest/dev/CopyingObjectsExamples.html - private static final long MAX_COPY_OBJECT_SIZE_BYTES = 5_368_709_120L; + @VisibleForTesting static final long MAX_COPY_OBJECT_SIZE_BYTES = 5_368_709_120L; // S3 API, delete-objects: "You may specify up to 1000 keys." private static final int MAX_DELETE_OBJECTS_PER_REQUEST = 1000; @@ -468,7 +468,8 @@ CopyObjectResponse atomicCopy( throws SdkServiceException { CopyObjectRequest copyObjectRequest = CopyObjectRequest.builder() - .copySource(sourcePath.getBucket() + "/" + sourcePath.getKey()) + .sourceBucket(sourcePath.getBucket()) + .sourceKey(sourcePath.getKey()) .destinationBucket(destinationPath.getBucket()) .destinationKey(destinationPath.getKey()) .metadata(objectHead.metadata()) @@ -499,6 +500,7 @@ CompleteMultipartUploadResponse multipartCopy( .ssekmsKeyId(config.getSSEKMSKeyId()) .sseCustomerKey(config.getSSECustomerKey().getKey()) .sseCustomerAlgorithm(config.getSSECustomerKey().getAlgorithm()) + .sseCustomerKeyMD5(config.getSSECustomerKey().getMD5()) .build(); CreateMultipartUploadResponse createMultipartUploadResponse = @@ -515,15 +517,18 @@ CompleteMultipartUploadResponse multipartCopy( if (objectSize == 0) { final UploadPartCopyRequest uploadPartCopyRequest = UploadPartCopyRequest.builder() - .bucket(sourcePath.getBucket()) - .key(sourcePath.getKey()) - .copySource(sourcePath.getBucket() + "/" + sourcePath.getKey()) + .destinationBucket(destinationPath.getBucket()) + .destinationKey(destinationPath.getKey()) + .sourceBucket(sourcePath.getBucket()) + .sourceKey(sourcePath.getKey()) .uploadId(uploadId) .partNumber(1) .sseCustomerKey(config.getSSECustomerKey().getKey()) .sseCustomerAlgorithm(config.getSSECustomerKey().getAlgorithm()) + .sseCustomerKeyMD5(config.getSSECustomerKey().getMD5()) .copySourceSSECustomerKey(config.getSSECustomerKey().getKey()) .copySourceSSECustomerAlgorithm(config.getSSECustomerKey().getAlgorithm()) + .copySourceSSECustomerKeyMD5(config.getSSECustomerKey().getMD5()) .build(); copyPartResult = s3Client.get().uploadPartCopy(uploadPartCopyRequest).copyPartResult(); @@ -531,32 +536,35 @@ CompleteMultipartUploadResponse multipartCopy( completedParts.add(completedPart); } else { long bytePosition = 0; - Integer uploadBufferSizeBytes = config.getS3UploadBufferSizeBytes(); // Amazon parts are 1-indexed, not zero-indexed. for (int partNumber = 1; bytePosition < objectSize; partNumber++) { final UploadPartCopyRequest uploadPartCopyRequest = UploadPartCopyRequest.builder() - .bucket(sourcePath.getBucket()) - .key(sourcePath.getKey()) - .copySource(destinationPath.getBucket() + "/" + sourcePath.getKey()) + .destinationBucket(destinationPath.getBucket()) + .destinationKey(destinationPath.getKey()) + .sourceBucket(sourcePath.getBucket()) + .sourceKey(sourcePath.getKey()) .uploadId(uploadId) .partNumber(partNumber) .copySourceRange( String.format( "bytes=%s-%s", bytePosition, - Math.min(objectSize - 1, bytePosition + uploadBufferSizeBytes - 1))) + Math.min(objectSize - 1, bytePosition + MAX_COPY_OBJECT_SIZE_BYTES - 1))) .sseCustomerKey(config.getSSECustomerKey().getKey()) .sseCustomerAlgorithm(config.getSSECustomerKey().getAlgorithm()) + .sseCustomerKeyMD5(config.getSSECustomerKey().getMD5()) .copySourceSSECustomerKey(config.getSSECustomerKey().getKey()) .copySourceSSECustomerAlgorithm(config.getSSECustomerKey().getAlgorithm()) + .copySourceSSECustomerKeyMD5(config.getSSECustomerKey().getMD5()) .build(); copyPartResult = s3Client.get().uploadPartCopy(uploadPartCopyRequest).copyPartResult(); - completedPart = CompletedPart.builder().partNumber(1).eTag(copyPartResult.eTag()).build(); + completedPart = + CompletedPart.builder().partNumber(partNumber).eTag(copyPartResult.eTag()).build(); completedParts.add(completedPart); - bytePosition += uploadBufferSizeBytes; + bytePosition += MAX_COPY_OBJECT_SIZE_BYTES; } } CompletedMultipartUpload completedMultipartUpload = diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/s3/S3FileSystemTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/s3/S3FileSystemTest.java index e56b405d6eff..123597a3e36e 100644 --- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/s3/S3FileSystemTest.java +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/s3/S3FileSystemTest.java @@ -25,7 +25,6 @@ import static org.apache.beam.sdk.io.aws2.s3.S3TestUtils.s3OptionsWithPathStyleAccessEnabled; import static org.apache.beam.sdk.io.aws2.s3.S3TestUtils.s3OptionsWithSSECustomerKey; import static org.apache.beam.sdk.io.aws2.s3.S3TestUtils.toMd5; -import static org.apache.beam.sdk.io.fs.CreateOptions.StandardCreateOptions.builder; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; @@ -33,6 +32,7 @@ import static org.junit.Assert.assertEquals; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.argThat; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.ArgumentMatchers.notNull; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; @@ -68,6 +68,8 @@ import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.S3Configuration; import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.CompletedMultipartUpload; +import software.amazon.awssdk.services.s3.model.CompletedPart; import software.amazon.awssdk.services.s3.model.CopyObjectRequest; import software.amazon.awssdk.services.s3.model.CopyObjectResponse; import software.amazon.awssdk.services.s3.model.CopyPartResult; @@ -272,7 +274,8 @@ private void testAtomicCopy(S3FileSystem s3FileSystem, SSECustomerKey sseCustome CopyObjectResponse copyObjectResponse = builder.build(); CopyObjectRequest copyObjectRequest = CopyObjectRequest.builder() - .copySource(sourcePath.getBucket() + "/" + sourcePath.getKey()) + .sourceBucket(sourcePath.getBucket()) + .sourceKey(sourcePath.getKey()) .destinationBucket(destinationPath.getBucket()) .destinationBucket(destinationPath.getKey()) .sseCustomerKey(sseCustomerKey.getKey()) @@ -305,21 +308,14 @@ public void testMultipartCopyWithS3Options() throws IOException { } private void testMultipartCopy(S3FileSystemConfiguration config) throws IOException { - testMultipartCopy( - buildMockedS3FileSystem(config), - config.getSSECustomerKey(), - config.getS3UploadBufferSizeBytes()); + testMultipartCopy(buildMockedS3FileSystem(config), config.getSSECustomerKey()); } private void testMultipartCopy(S3Options options) throws IOException { - testMultipartCopy( - buildMockedS3FileSystem(options), - options.getSSECustomerKey(), - options.getS3UploadBufferSizeBytes()); + testMultipartCopy(buildMockedS3FileSystem(options), options.getSSECustomerKey()); } - private void testMultipartCopy( - S3FileSystem s3FileSystem, SSECustomerKey sseCustomerKey, long s3UploadBufferSizeBytes) + private void testMultipartCopy(S3FileSystem s3FileSystem, SSECustomerKey sseCustomerKey) throws IOException { S3ResourceId sourcePath = S3ResourceId.fromUri(s3FileSystem.getScheme() + "://bucket/from"); S3ResourceId destinationPath = S3ResourceId.fromUri(s3FileSystem.getScheme() + "://bucket/to"); @@ -346,7 +342,7 @@ private void testMultipartCopy( HeadObjectResponse.Builder headObjectResponseBuilder = HeadObjectResponse.builder() - .contentLength((long) (s3UploadBufferSizeBytes * 1.5)) + .contentLength((long) (S3FileSystem.MAX_COPY_OBJECT_SIZE_BYTES * 1.5)) .contentEncoding("read-seek-efficient"); if (sseCustomerKeyMd5 != null) { headObjectResponseBuilder.sseCustomerKeyMD5(sseCustomerKeyMd5); @@ -358,6 +354,36 @@ private void testMultipartCopy( sseCustomerKeyMd5, headObjectResponse); + UploadPartCopyRequest uploadPartCopyRequest1 = + UploadPartCopyRequest.builder() + .destinationBucket(destinationPath.getBucket()) + .destinationKey(destinationPath.getKey()) + .sourceBucket(sourcePath.getBucket()) + .sourceKey(sourcePath.getKey()) + .uploadId(createMultipartUploadResponse.uploadId()) + .partNumber(1) + .copySourceRange( + String.format("bytes=0-%s", S3FileSystem.MAX_COPY_OBJECT_SIZE_BYTES - 1)) + .sseCustomerKey(sseCustomerKey == null ? null : sseCustomerKey.getKey()) + .sseCustomerAlgorithm(sseCustomerKey == null ? null : sseCustomerKey.getAlgorithm()) + .sseCustomerKeyMD5(sseCustomerKeyMd5) + .copySourceSSECustomerKey(sseCustomerKey == null ? null : sseCustomerKey.getKey()) + .copySourceSSECustomerAlgorithm( + sseCustomerKey == null ? null : sseCustomerKey.getAlgorithm()) + .copySourceSSECustomerKeyMD5(sseCustomerKeyMd5) + .build(); + + UploadPartCopyRequest uploadPartCopyRequest2 = + uploadPartCopyRequest1 + .toBuilder() + .partNumber(2) + .copySourceRange( + String.format( + "bytes=%s-%s", + S3FileSystem.MAX_COPY_OBJECT_SIZE_BYTES, + headObjectResponse.contentLength() - 1)) + .build(); + CopyPartResult copyPartResult1 = CopyPartResult.builder().eTag("etag-1").build(); CopyPartResult copyPartResult2 = CopyPartResult.builder().eTag("etag-2").build(); UploadPartCopyResponse.Builder uploadPartCopyResponseBuilder1 = @@ -370,19 +396,33 @@ private void testMultipartCopy( } UploadPartCopyResponse uploadPartCopyResponse1 = uploadPartCopyResponseBuilder1.build(); UploadPartCopyResponse uploadPartCopyResponse2 = uploadPartCopyResponseBuilder2.build(); - UploadPartCopyRequest uploadPartCopyRequest = - UploadPartCopyRequest.builder().sseCustomerKey(sseCustomerKey.getKey()).build(); - when(s3FileSystem.getS3Client().uploadPartCopy(any(UploadPartCopyRequest.class))) - .thenReturn(uploadPartCopyResponse1) + when(s3FileSystem.getS3Client().uploadPartCopy(eq(uploadPartCopyRequest1))) + .thenReturn(uploadPartCopyResponse1); + when(s3FileSystem.getS3Client().uploadPartCopy(eq(uploadPartCopyRequest2))) .thenReturn(uploadPartCopyResponse2); - assertEquals( - sseCustomerKeyMd5, - s3FileSystem.getS3Client().uploadPartCopy(uploadPartCopyRequest).sseCustomerKeyMD5()); s3FileSystem.multipartCopy(sourcePath, destinationPath, headObjectResponse); verify(s3FileSystem.getS3Client(), times(1)) - .completeMultipartUpload(any(CompleteMultipartUploadRequest.class)); + .completeMultipartUpload( + eq( + CompleteMultipartUploadRequest.builder() + .bucket(destinationPath.getBucket()) + .key(destinationPath.getKey()) + .uploadId(createMultipartUploadResponse.uploadId()) + .multipartUpload( + CompletedMultipartUpload.builder() + .parts( + CompletedPart.builder() + .partNumber(1) + .eTag(copyPartResult1.eTag()) + .build(), + CompletedPart.builder() + .partNumber(2) + .eTag(copyPartResult2.eTag()) + .build()) + .build()) + .build())); } @Test From 346d2e296c18d510ac72ec2f5a7fa6f63abbe745 Mon Sep 17 00:00:00 2001 From: Aydar Zainutdinov Date: Fri, 4 Mar 2022 19:29:10 +0300 Subject: [PATCH 36/68] Merge pull request #16842 from [BEAM-13932][Playground] Container's user privileges * [BEAM-13932][Playground] Change Dockerfiles * [BEAM-13932][Playground] Update proxy and permissions for the container's user * [BEAM-13932][Playground] Update permissions for the container's user for scio --- playground/backend/containers/go/Dockerfile | 12 ++++----- playground/backend/containers/go/build.gradle | 4 --- .../backend/containers/go/entrypoint.sh | 9 +++++++ playground/backend/containers/go/proxy.sh | 24 ------------------ playground/backend/containers/java/Dockerfile | 13 +++++----- .../backend/containers/java/build.gradle | 4 --- .../backend/containers/java/entrypoint.sh | 9 +++++++ playground/backend/containers/java/proxy.sh | 24 ------------------ .../backend/containers/python/Dockerfile | 12 +++++---- .../backend/containers/python/build.gradle | 4 --- .../backend/containers/python/entrypoint.sh | 10 ++++++++ playground/backend/containers/python/proxy.sh | 25 ------------------- playground/backend/containers/scio/Dockerfile | 12 +++++---- .../backend/containers/scio/build.gradle | 4 --- .../backend/containers/scio/entrypoint.sh | 9 +++++++ playground/backend/containers/scio/proxy.sh | 24 ------------------ .../infrastructure/proxy/allow_list_proxy.py | 6 +++-- 17 files changed, 68 insertions(+), 137 deletions(-) delete mode 100644 playground/backend/containers/go/proxy.sh delete mode 100644 playground/backend/containers/java/proxy.sh delete mode 100644 playground/backend/containers/python/proxy.sh delete mode 100644 playground/backend/containers/scio/proxy.sh diff --git a/playground/backend/containers/go/Dockerfile b/playground/backend/containers/go/Dockerfile index b7243db14cdc..0c9c0d7fc039 100644 --- a/playground/backend/containers/go/Dockerfile +++ b/playground/backend/containers/go/Dockerfile @@ -70,16 +70,16 @@ ENV HTTP_PROXY="http://127.0.0.1:8081" ENV HTTPS_PROXY="http://127.0.0.1:8081" COPY entrypoint.sh / -COPY proxy.sh / -# Run proxy -RUN sh /proxy.sh # Create a user group `appgroup` and a user `appuser` RUN groupadd --gid 20000 appgroup \ && useradd --uid 20000 --gid appgroup --shell /bin/bash --create-home appuser -# Chown all the files to the app user. -RUN chown -R appuser:appgroup /opt/playground/backend/ && chown -R appuser:appgroup /opt/playground/prepared_folder/ \ - && chmod +x /entrypoint.sh + +RUN mkdir -p /opt/playground/backend/executable_files/ + +# Chown all required files to the `appuser`. +RUN chown -R appuser:appgroup /opt/playground/backend/executable_files/ \ + && chmod -R 777 /usr/local/share/ca-certificates/extra/ && chmod -R 777 /etc/ssl/certs && chmod +x /entrypoint.sh # Switch to appuser USER appuser diff --git a/playground/backend/containers/go/build.gradle b/playground/backend/containers/go/build.gradle index c175fd7e4ee6..0fc3a6bf40e3 100644 --- a/playground/backend/containers/go/build.gradle +++ b/playground/backend/containers/go/build.gradle @@ -43,10 +43,6 @@ task copyDockerfileDependencies(type: Copy) { from 'entrypoint.sh' into 'build/' } - copy { - from 'proxy.sh' - into 'build/' - } copy { from '../../../infrastructure/proxy/allow_list.py' into 'build/' diff --git a/playground/backend/containers/go/entrypoint.sh b/playground/backend/containers/go/entrypoint.sh index 988302c2a714..63d77a6fef04 100755 --- a/playground/backend/containers/go/entrypoint.sh +++ b/playground/backend/containers/go/entrypoint.sh @@ -14,4 +14,13 @@ # See the License for the specific language governing permissions and # limitations under the License. +nohup /opt/mitmproxy/mitmdump -s /opt/mitmproxy/allow_list_proxy.py -p 8081 & +while [ ! -f /home/appuser/.mitmproxy/mitmproxy-ca.pem ] ; +do + sleep 2 +done +openssl x509 -in /home/appuser/.mitmproxy/mitmproxy-ca.pem -inform PEM -out /home/appuser/.mitmproxy/mitmproxy-ca.crt +cp /home/appuser/.mitmproxy/mitmproxy-ca.crt /usr/local/share/ca-certificates/extra/ +update-ca-certificates + /opt/playground/backend/server_go_backend diff --git a/playground/backend/containers/go/proxy.sh b/playground/backend/containers/go/proxy.sh deleted file mode 100644 index 73a0bfb33ffd..000000000000 --- a/playground/backend/containers/go/proxy.sh +++ /dev/null @@ -1,24 +0,0 @@ -#!/bin/bash -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -nohup /opt/mitmproxy/mitmdump -s /opt/mitmproxy/allow_list_proxy.py -p 8081 & -while [ ! -f /root/.mitmproxy/mitmproxy-ca.pem ] ; -do - sleep 2 -done -openssl x509 -in /root/.mitmproxy/mitmproxy-ca.pem -inform PEM -out /root/.mitmproxy/mitmproxy-ca.crt -cp /root/.mitmproxy/mitmproxy-ca.crt /usr/local/share/ca-certificates/extra/ -update-ca-certificates diff --git a/playground/backend/containers/java/Dockerfile b/playground/backend/containers/java/Dockerfile index 603cca4b8cb7..cacaa0b811d2 100644 --- a/playground/backend/containers/java/Dockerfile +++ b/playground/backend/containers/java/Dockerfile @@ -88,16 +88,17 @@ ENV HTTP_PROXY="http://127.0.0.1:8081" ENV HTTPS_PROXY="http://127.0.0.1:8081" COPY entrypoint.sh / -COPY proxy.sh / -# Run proxy -RUN sh /proxy.sh # Create a user group `appgroup` and a user `appuser` RUN groupadd --gid 20000 appgroup \ && useradd --uid 20000 --gid appgroup --shell /bin/bash --create-home appuser -# Chown all the files to the app user. -RUN chown -R appuser:appgroup /opt/playground/backend/ && chown -R appuser:appgroup /opt/apache/beam/jars/ \ - && chmod +x /entrypoint.sh + +RUN mkdir -p /opt/playground/backend/executable_files/ + +# Chown all required files to the `appuser`. +RUN chown -R appuser:appgroup /opt/playground/backend/executable_files/ && chmod -R 775 /opt/apache/beam/jars/ \ + && chmod -R 777 /usr/local/share/ca-certificates/extra/ && chmod -R 777 /usr/local/openjdk-8/jre/lib/security/ \ + && chmod -R 777 /etc/ssl/certs && chmod +x /entrypoint.sh # Switch to appuser USER appuser diff --git a/playground/backend/containers/java/build.gradle b/playground/backend/containers/java/build.gradle index 5d449a000e98..9d1e22c56f0e 100644 --- a/playground/backend/containers/java/build.gradle +++ b/playground/backend/containers/java/build.gradle @@ -43,10 +43,6 @@ task copyDockerfileDependencies(type: Copy) { from 'entrypoint.sh' into 'build/' } - copy { - from 'proxy.sh' - into 'build/' - } copy { from '../../../infrastructure/proxy/allow_list.py' into 'build/' diff --git a/playground/backend/containers/java/entrypoint.sh b/playground/backend/containers/java/entrypoint.sh index 146281d06400..87305f61055a 100755 --- a/playground/backend/containers/java/entrypoint.sh +++ b/playground/backend/containers/java/entrypoint.sh @@ -14,4 +14,13 @@ # See the License for the specific language governing permissions and # limitations under the License. +nohup /opt/mitmproxy/mitmdump -s /opt/mitmproxy/allow_list_proxy.py -p 8081 & +while [ ! -f /home/appuser/.mitmproxy/mitmproxy-ca.pem ] ; +do + sleep 2 +done +openssl x509 -in /home/appuser/.mitmproxy/mitmproxy-ca.pem -inform PEM -out /home/appuser/.mitmproxy/mitmproxy-ca.crt +cp /home/appuser/.mitmproxy/mitmproxy-ca.crt /usr/local/share/ca-certificates/extra/ +update-ca-certificates + /opt/playground/backend/server_java_backend diff --git a/playground/backend/containers/java/proxy.sh b/playground/backend/containers/java/proxy.sh deleted file mode 100644 index 73a0bfb33ffd..000000000000 --- a/playground/backend/containers/java/proxy.sh +++ /dev/null @@ -1,24 +0,0 @@ -#!/bin/bash -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -nohup /opt/mitmproxy/mitmdump -s /opt/mitmproxy/allow_list_proxy.py -p 8081 & -while [ ! -f /root/.mitmproxy/mitmproxy-ca.pem ] ; -do - sleep 2 -done -openssl x509 -in /root/.mitmproxy/mitmproxy-ca.pem -inform PEM -out /root/.mitmproxy/mitmproxy-ca.crt -cp /root/.mitmproxy/mitmproxy-ca.crt /usr/local/share/ca-certificates/extra/ -update-ca-certificates diff --git a/playground/backend/containers/python/Dockerfile b/playground/backend/containers/python/Dockerfile index 65e28fac1ec0..152a6c931975 100644 --- a/playground/backend/containers/python/Dockerfile +++ b/playground/backend/containers/python/Dockerfile @@ -64,15 +64,17 @@ ENV HTTP_PROXY="http://127.0.0.1:8081" ENV HTTPS_PROXY="http://127.0.0.1:8081" COPY entrypoint.sh / -COPY proxy.sh / -# Run proxy -RUN sh /proxy.sh # Create a user group `appgroup` and a user `appuser` RUN groupadd --gid 20000 appgroup \ && useradd --uid 20000 --gid appgroup --shell /bin/bash --create-home appuser -# Chown all the files to the app user. -RUN chown -R appuser:appgroup /opt/playground/backend/ && chmod +x /entrypoint.sh + +RUN mkdir -p /opt/playground/backend/executable_files/ + +# Chown all required files to the `appuser`. +RUN chown -R appuser:appgroup /opt/playground/backend/executable_files/ \ + && chmod 777 /usr/local/lib/python3.7/site-packages/certifi/cacert.pem \ + && chmod -R 777 /usr/local/share/ca-certificates/extra/ && chmod -R 777 /etc/ssl/certs && chmod +x /entrypoint.sh # Switch to appuser USER appuser diff --git a/playground/backend/containers/python/build.gradle b/playground/backend/containers/python/build.gradle index 9378eabdcce2..6fa49a311624 100644 --- a/playground/backend/containers/python/build.gradle +++ b/playground/backend/containers/python/build.gradle @@ -43,10 +43,6 @@ task copyDockerfileDependencies(type: Copy) { from 'entrypoint.sh' into 'build/' } - copy { - from 'proxy.sh' - into 'build/' - } copy { from '../../../infrastructure/proxy/allow_list.py' into 'build/' diff --git a/playground/backend/containers/python/entrypoint.sh b/playground/backend/containers/python/entrypoint.sh index b8cfeb467cbf..56768c356295 100755 --- a/playground/backend/containers/python/entrypoint.sh +++ b/playground/backend/containers/python/entrypoint.sh @@ -14,4 +14,14 @@ # See the License for the specific language governing permissions and # limitations under the License. +nohup /opt/mitmproxy/mitmdump -s /opt/mitmproxy/allow_list_proxy.py -p 8081 & +while [ ! -f /home/appuser/.mitmproxy/mitmproxy-ca.pem ] ; +do + sleep 2 +done +openssl x509 -in /home/appuser/.mitmproxy/mitmproxy-ca.pem -inform PEM -out /home/appuser/.mitmproxy/mitmproxy-ca.crt +cp /home/appuser/.mitmproxy/mitmproxy-ca.crt /usr/local/share/ca-certificates/extra/ +update-ca-certificates +cat /home/appuser/.mitmproxy/mitmproxy-ca.pem >> /usr/local/lib/python3.7/site-packages/certifi/cacert.pem + /opt/playground/backend/server_python_backend diff --git a/playground/backend/containers/python/proxy.sh b/playground/backend/containers/python/proxy.sh deleted file mode 100644 index 36ba22848242..000000000000 --- a/playground/backend/containers/python/proxy.sh +++ /dev/null @@ -1,25 +0,0 @@ -#!/bin/bash -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -nohup /opt/mitmproxy/mitmdump -s /opt/mitmproxy/allow_list_proxy.py -p 8081 & -while [ ! -f /root/.mitmproxy/mitmproxy-ca.pem ] ; -do - sleep 2 -done -openssl x509 -in /root/.mitmproxy/mitmproxy-ca.pem -inform PEM -out /root/.mitmproxy/mitmproxy-ca.crt -cp /root/.mitmproxy/mitmproxy-ca.crt /usr/local/share/ca-certificates/extra/ -update-ca-certificates -cat /root/.mitmproxy/mitmproxy-ca.pem >> /usr/local/lib/python3.7/site-packages/certifi/cacert.pem diff --git a/playground/backend/containers/scio/Dockerfile b/playground/backend/containers/scio/Dockerfile index 8031b1fdb273..4466cfd657f8 100644 --- a/playground/backend/containers/scio/Dockerfile +++ b/playground/backend/containers/scio/Dockerfile @@ -68,15 +68,17 @@ ENV HTTP_PROXY="http://127.0.0.1:8081" ENV HTTPS_PROXY="http://127.0.0.1:8081" COPY entrypoint.sh / -COPY proxy.sh / -# Run proxy -RUN sh /proxy.sh # Create a user group `appgroup` and a user `appuser` RUN groupadd --gid 20000 appgroup \ && useradd --uid 20000 --gid appgroup --shell /bin/bash --create-home appuser -# Chown all the files to the app user. -RUN chown -R appuser:appgroup /opt/playground/backend/ && chmod +x /entrypoint.sh + +RUN mkdir -p /opt/playground/backend/executable_files/ + +# Chown all required files to the `appuser`. +RUN chown -R appuser:appgroup /opt/playground/backend/executable_files/ \ + && chmod -R 777 /usr/local/share/ca-certificates/extra/ && chmod -R 777 /usr/local/openjdk-8/jre/lib/security/ \ + && chmod -R 777 /etc/ssl/certs && chmod +x /entrypoint.sh # Switch to appuser USER appuser diff --git a/playground/backend/containers/scio/build.gradle b/playground/backend/containers/scio/build.gradle index 011705990bd6..8b96f045ac99 100644 --- a/playground/backend/containers/scio/build.gradle +++ b/playground/backend/containers/scio/build.gradle @@ -43,10 +43,6 @@ task copyDockerfileDependencies(type: Copy) { from 'entrypoint.sh' into 'build/' } - copy { - from 'proxy.sh' - into 'build/' - } copy { from '../../../infrastructure/proxy/allow_list.py' into 'build/' diff --git a/playground/backend/containers/scio/entrypoint.sh b/playground/backend/containers/scio/entrypoint.sh index fd2ef3d618e6..b3e6d55ae507 100755 --- a/playground/backend/containers/scio/entrypoint.sh +++ b/playground/backend/containers/scio/entrypoint.sh @@ -14,4 +14,13 @@ # See the License for the specific language governing permissions and # limitations under the License. +nohup /opt/mitmproxy/mitmdump -s /opt/mitmproxy/allow_list_proxy.py -p 8081 & +while [ ! -f /home/appuser/.mitmproxy/mitmproxy-ca.pem ] ; +do + sleep 2 +done +openssl x509 -in /home/appuser/.mitmproxy/mitmproxy-ca.pem -inform PEM -out /home/appuser/.mitmproxy/mitmproxy-ca.crt +cp /home/appuser/.mitmproxy/mitmproxy-ca.crt /usr/local/share/ca-certificates/extra/ +update-ca-certificates + /opt/playground/backend/server_scio_backend diff --git a/playground/backend/containers/scio/proxy.sh b/playground/backend/containers/scio/proxy.sh deleted file mode 100644 index 73a0bfb33ffd..000000000000 --- a/playground/backend/containers/scio/proxy.sh +++ /dev/null @@ -1,24 +0,0 @@ -#!/bin/bash -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -nohup /opt/mitmproxy/mitmdump -s /opt/mitmproxy/allow_list_proxy.py -p 8081 & -while [ ! -f /root/.mitmproxy/mitmproxy-ca.pem ] ; -do - sleep 2 -done -openssl x509 -in /root/.mitmproxy/mitmproxy-ca.pem -inform PEM -out /root/.mitmproxy/mitmproxy-ca.crt -cp /root/.mitmproxy/mitmproxy-ca.crt /usr/local/share/ca-certificates/extra/ -update-ca-certificates diff --git a/playground/infrastructure/proxy/allow_list_proxy.py b/playground/infrastructure/proxy/allow_list_proxy.py index ee04bc35014b..5e71fea54d5c 100644 --- a/playground/infrastructure/proxy/allow_list_proxy.py +++ b/playground/infrastructure/proxy/allow_list_proxy.py @@ -21,10 +21,12 @@ def request(flow: http.HTTPFlow) -> None: allowed_bucket = flow.request.pretty_host == GCS_HOST and \ - flow.request.path.split("/")[1] in ALLOWED_BUCKET_LIST + (flow.request.path.split("/")[1] in ALLOWED_BUCKET_LIST or \ + flow.request.path.split("/")[4] in ALLOWED_BUCKET_LIST) allowed_host = flow.request.pretty_host in ALLOWED_LIST if not (allowed_bucket or allowed_host): flow.response = http.Response.make( status_code=403, content="Making requests to the hosts that are not listed " - "in the allowed list is forbidden.") + "in the allowed list is forbidden. " + "host:" + flow.request.pretty_host + ", path: " + flow.request.path) From f73e4757fb350c3a2cc16d1d546640e8d3a31326 Mon Sep 17 00:00:00 2001 From: Brian Hulette Date: Fri, 4 Mar 2022 10:14:02 -0800 Subject: [PATCH 37/68] Doc updates and blog post for 2.37.0 (#16887) * Doc updates and blog post for 2.37.0 * Add BEAM-13980 to known issues * Update dates * Drop known issue (fix cherrypicked) * Add license * Add missing # --- CHANGES.md | 2 +- website/www/site/config.toml | 2 +- .../www/site/content/en/blog/beam-2.37.0.md | 123 ++++++++++++++++++ .../site/content/en/get-started/downloads.md | 8 ++ 4 files changed, 133 insertions(+), 2 deletions(-) create mode 100644 website/www/site/content/en/blog/beam-2.37.0.md diff --git a/CHANGES.md b/CHANGES.md index d391a24816b8..c3eadeddae4c 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -85,7 +85,7 @@ * ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)). -# [2.37.0] - Unreleased, Branch Cut +# [2.37.0] - 2022-03-04 ## Highlights * Java 17 support for Dataflow ([BEAM-12240](https://issues.apache.org/jira/browse/BEAM-12240)). diff --git a/website/www/site/config.toml b/website/www/site/config.toml index cabaa7c3d1e9..e9d927ea84d0 100644 --- a/website/www/site/config.toml +++ b/website/www/site/config.toml @@ -104,7 +104,7 @@ github_project_repo = "https://github.com/apache/beam" [params] description = "Apache Beam is an open source, unified model and set of language-specific SDKs for defining and executing data processing workflows, and also data ingestion and integration flows, supporting Enterprise Integration Patterns (EIPs) and Domain Specific Languages (DSLs). Dataflow pipelines simplify the mechanics of large-scale batch and streaming data processing and can run on a number of runtimes like Apache Flink, Apache Spark, and Google Cloud Dataflow (a cloud service). Beam also brings DSL in different languages, allowing users to easily implement their data integration processes." -release_latest = "2.36.0" +release_latest = "2.37.0" # The repository and branch where the files live in Github or Colab. This is used # to serve and stage from your local branch, but publish to the master branch. # e.g. https://github.com/{{< param branch_repo >}}/path/to/notebook.ipynb diff --git a/website/www/site/content/en/blog/beam-2.37.0.md b/website/www/site/content/en/blog/beam-2.37.0.md new file mode 100644 index 000000000000..965995dc1a34 --- /dev/null +++ b/website/www/site/content/en/blog/beam-2.37.0.md @@ -0,0 +1,123 @@ +--- +title: "Apache Beam 2.37.0" +date: 2022-03-04 08:30:00 -0800 +categories: + - blog +authors: + - bhulette +--- + + + +We are happy to present the new 2.37.0 release of Beam. +This release includes both improvements and new functionality. +See the [download page](/get-started/downloads/#2370-2022-03-04) for this release. + + + +For more information on changes in 2.37.0 check out the [detailed release notes](https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12319527&version=12351168). + +## Highlights +* Java 17 support for Dataflow ([BEAM-12240](https://issues.apache.org/jira/browse/BEAM-12240)). + * Users using Dataflow Runner V2 may see issues with state cache due to inaccurate object sizes ([BEAM-13695](https://issues.apache.org/jira/browse/BEAM-13695)). + * ZetaSql is currently unsupported ([issue](https://github.com/google/zetasql/issues/89)). +* Python 3.9 support in Apache Beam ([BEAM-12000](https://issues.apache.org/jira/browse/BEAM-12000)). + * Dataflow support for Python 3.9 is expected to be available with 2.37.0, + but may not be fully available yet when the release is announced ([BEAM-13864](https://issues.apache.org/jira/browse/BEAM-13864)). + * Users of Dataflow Runner V2 can run Python 3.9 pipelines with 2.37.0 release right away. + +## I/Os + +* Go SDK now has wrappers for the following Cross Language Transforms from Java, along with automatic expansion service startup for each. + * JDBCIO ([BEAM-13293](https://issues.apache.org/jira/browse/BEAM-13293)). + * Debezium ([BEAM-13761](https://issues.apache.org/jira/browse/BEAM-13761)). + * BeamSQL ([BEAM-13683](https://issues.apache.org/jira/browse/BEAM-13683)). + * BiqQuery ([BEAM-13732](https://issues.apache.org/jira/browse/BEAM-13732)). + * KafkaIO now also has automatic expansion service startup. ([BEAM-13821](https://issues.apache.org/jira/browse/BEAM-13821)). + +## New Features / Improvements + +* DataFrame API now supports pandas 1.4.x ([BEAM-13605](https://issues.apache.org/jira/browse/BEAM-13605)). +* Go SDK DoFns can now observe trigger panes directly ([BEAM-13757](https://issues.apache.org/jira/browse/BEAM-13757)). + +### Known Issues + +* See a full list of open [issues that affect](https://issues.apache.org/jira/issues/?jql=project%20%3D%20BEAM%20AND%20affectedVersion%20%3D%202.37.0%20ORDER%20BY%20priority%20DESC%2C%20updated%20DESC) this version. + +## List of Contributors + +According to git shortlog, the following people contributed to the 2.37.0 release. Thank you to all contributors! + +Aizhamal Nurmamat kyzy +Alexander +Alexander Chermenin +Alexandr Zhuravlev +Alexey Romanenko +Anand Inguva +andoni-guzman +andreukus +Andy Ye +Artur Khanin +Aydar Farrakhov +Aydar Zainutdinov +AydarZaynutdinov +Benjamin Gonzalez +Brian Hulette +Chamikara Jayalath +Daniel Oliveira +Danny McCormick +daria-malkova +daria.malkova +darshan-sj +David Huntsperger +dprieto91 +emily +Etienne Chauchot +Fernando Morales +Heejong Lee +Ismaël Mejía +Jack McCluskey +Jan Lukavský +johnjcasey +Kamil Breguła +kellen +Kenneth Knowles +kileys +Kyle Weaver +Luke Cwik +Marcin Kuthan +Marco Robles +Matt Rudary +Miguel Hernandez +Milena Bukal +Moritz Mack +Mostafa Aghajani +Ning Kang +Pablo Estrada +Pavel Avilov +Reuven Lax +Ritesh Ghorse +Robert Bradshaw +Robert Burke +Sam Whittle +Sandy Chapman +Sergey Kalinin +Thiago Nunes +thorbjorn444 +Tim Robertson +Tomo Suzuki +Valentyn Tymofieiev +Victor +Victor Chen +Vitaly Ivanov +Yichi Zhang diff --git a/website/www/site/content/en/get-started/downloads.md b/website/www/site/content/en/get-started/downloads.md index 412ef5820073..895efb801654 100644 --- a/website/www/site/content/en/get-started/downloads.md +++ b/website/www/site/content/en/get-started/downloads.md @@ -88,6 +88,14 @@ versions denoted `0.x.y`. ## Releases +### 2.37.0 (2022-03-04) +Official [source code download](https://www.apache.org/dyn/closer.cgi/beam/2.37.0/apache-beam-2.37.0-source-release.zip). +[SHA-512](https://downloads.apache.org/beam/2.37.0/apache-beam-2.37.0-source-release.zip.sha512). +[signature](https://downloads.apache.org/beam/2.37.0/apache-beam-2.37.0-source-release.zip.asc). + +[Release notes](https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12319527&version=12351168) +[Blog post](/blog/beam-2.37.0). + ### 2.36.0 (2022-02-07) Official [source code download](https://www.apache.org/dyn/closer.cgi/beam/2.36.0/apache-beam-2.36.0-source-release.zip). [SHA-512](https://downloads.apache.org/beam/2.36.0/apache-beam-2.36.0-source-release.zip.sha512). From 0d79ddb57688de81a08137011830bb0a8993ab49 Mon Sep 17 00:00:00 2001 From: David Cavazos Date: Fri, 4 Mar 2022 12:16:27 -0800 Subject: [PATCH 38/68] Remove resolved issue in docs + update class path on sample (#17018) --- .../examples/snippets/transforms/elementwise/pardo.py | 2 +- .../en/documentation/transforms/python/elementwise/pardo.md | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo.py index c54d05e39911..fd8dca134d2d 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo.py @@ -96,7 +96,7 @@ def pardo_dofn_methods(test=None): class DoFnMethods(beam.DoFn): def __init__(self): print('__init__') - self.window = beam.window.GlobalWindow() + self.window = beam.transforms.window.GlobalWindow() def setup(self): print('setup') diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/pardo.md b/website/www/site/content/en/documentation/transforms/python/elementwise/pardo.md index efa64ab5d909..dc12acda1807 100644 --- a/website/www/site/content/en/documentation/transforms/python/elementwise/pardo.md +++ b/website/www/site/content/en/documentation/transforms/python/elementwise/pardo.md @@ -141,8 +141,6 @@ Output: > *Known issues:* > -> * [[BEAM-7885]](https://issues.apache.org/jira/browse/BEAM-7885) -> `DoFn.setup()` doesn't run for streaming jobs running in the `DirectRunner`. > * [[BEAM-7340]](https://issues.apache.org/jira/browse/BEAM-7340) > `DoFn.teardown()` metrics are lost. From 8636862fad94878d624786977b2051402e745bb8 Mon Sep 17 00:00:00 2001 From: Ning Kang Date: Fri, 4 Mar 2022 12:32:39 -0800 Subject: [PATCH 39/68] [BEAM-14016] Fixed flaky postcommit test (#17009) Fixed SpannerWriteIntegrationTest.test_spanner_update by fixing the metric exporter usage in spannerio. --- .../io/gcp/experimental/spannerio.py | 27 ++++++++++++------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/sdks/python/apache_beam/io/gcp/experimental/spannerio.py b/sdks/python/apache_beam/io/gcp/experimental/spannerio.py index e36fbad09214..39a3a27aaf89 100644 --- a/sdks/python/apache_beam/io/gcp/experimental/spannerio.py +++ b/sdks/python/apache_beam/io/gcp/experimental/spannerio.py @@ -156,7 +156,7 @@ transactions sorted by table and primary key. WriteToSpanner transforms starts with the grouping into batches. The first step -in this process is to make the make the mutation groups of the WriteMutation +in this process is to make the mutation groups of the WriteMutation objects and then filtering them into batchable and unbatchable mutation groups. There are three batching parameters (max_number_cells, max_number_rows & max_batch_size_bytes). We calculated th mutation byte size from the method @@ -1202,9 +1202,12 @@ def __init__(self, spanner_configuration): monitoring_infos.SPANNER_PROJECT_ID: spanner_configuration.project, monitoring_infos.SPANNER_DATABASE_ID: spanner_configuration.database, } - self.service_metric = None + # table_id to metrics + self.service_metrics = {} - def _table_metric(self, table_id): + def _register_table_metric(self, table_id): + if table_id in self.service_metrics: + return database_id = self._spanner_configuration.database project_id = self._spanner_configuration.project resource = resource_identifiers.SpannerTable( @@ -1217,7 +1220,7 @@ def _table_metric(self, table_id): service_call_metric = ServiceCallMetric( request_count_urn=monitoring_infos.API_REQUEST_COUNT_URN, base_labels=labels) - return service_call_metric + self.service_metrics[table_id] = service_call_metric def setup(self): spanner_client = Client(self._spanner_configuration.project) @@ -1226,13 +1229,16 @@ def setup(self): self._spanner_configuration.database, pool=self._spanner_configuration.pool) + def start_bundle(self): + self.service_metrics = {} + def process(self, element): self.batches.inc() try: with self._db_instance.batch() as b: for m in element: table_id = m.kwargs['table'] - self.service_metric = self._table_metric(table_id) + self._register_table_metric(table_id) if m.operation == WriteMutation._OPERATION_DELETE: batch_func = b.delete @@ -1247,14 +1253,17 @@ def process(self, element): else: raise ValueError("Unknown operation action: %s" % m.operation) batch_func(**m.kwargs) - - self.service_metric.call('ok') except (ClientError, GoogleAPICallError) as e: - self.service_metric.call(str(e.code.value)) + for service_metric in self.service_metrics.values(): + service_metric.call(str(e.code.value)) raise except HttpError as e: - self.service_metric.call(str(e)) + for service_metric in self.service_metrics.values(): + service_metric.call(str(e)) raise + else: + for service_metric in self.service_metrics.values(): + service_metric.call('ok') @with_input_types(typing.Union[MutationGroup, _Mutator]) From 00dd9fb2fb00ff634e110e674cee8db544f47df5 Mon Sep 17 00:00:00 2001 From: github-actions Date: Fri, 4 Mar 2022 07:00:38 -0500 Subject: [PATCH 40/68] [BEAM-13925] months in date constructor are 0 indexed --- scripts/ci/pr-bot/processNewPrs.ts | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/scripts/ci/pr-bot/processNewPrs.ts b/scripts/ci/pr-bot/processNewPrs.ts index 1f178900e4f5..34494ce7c54b 100644 --- a/scripts/ci/pr-bot/processNewPrs.ts +++ b/scripts/ci/pr-bot/processNewPrs.ts @@ -46,8 +46,12 @@ function needsProcessed(pull: any, prState: typeof Pr): boolean { ); return false; } - let firstPrToProcess = new Date(2022, 3, 2, 20); - if (new Date(pull.created_at) < firstPrToProcess) { + const firstPrToProcess = new Date(2022, 2, 2, 20); + const createdAt = new Date(pull.created_at); + if (createdAt < firstPrToProcess) { + console.log( + `Skipping PR ${pull.number} because it was created at ${createdAt}, before the first pr to process date of ${firstPrToProcess}` + ); return false; } if (prState.remindAfterTestsPass && prState.remindAfterTestsPass.length > 0) { @@ -157,6 +161,8 @@ async function processPull( return; } + console.log(`Processing PR ${pull.number}`); + let checkState = await getChecksStatus(REPO_OWNER, REPO, pull.head.sha); if (prState.remindAfterTestsPass && prState.remindAfterTestsPass.length > 0) { From b402d5e4ec25deb4deb2e2cb4e1ca8bd65b66bee Mon Sep 17 00:00:00 2001 From: Andy Ye Date: Fri, 4 Mar 2022 16:29:50 -0600 Subject: [PATCH 41/68] [BEAM-13947] Add split() and rsplit(), non-deferred column operations on categorical columns (#16677) * Add split/rsplit; Need to refactor regex * Support Regex; Refactor tests * Remove debugger * fix grammar * Fix passing regex arg * Reorder imports * Address PR comments; Simplify kwargs * Simplify getting columns for split_cat * Update doctests to skip expand=True operations * Fix missing doctest --- sdks/python/apache_beam/dataframe/frames.py | 71 ++++++++++- .../apache_beam/dataframe/frames_test.py | 112 ++++++++++++++++++ .../dataframe/pandas_doctests_test.py | 29 ++++- 3 files changed, 203 insertions(+), 9 deletions(-) diff --git a/sdks/python/apache_beam/dataframe/frames.py b/sdks/python/apache_beam/dataframe/frames.py index 00961a6f1b2f..965329a34ecf 100644 --- a/sdks/python/apache_beam/dataframe/frames.py +++ b/sdks/python/apache_beam/dataframe/frames.py @@ -4765,13 +4765,72 @@ def get_dummies(self, **kwargs): requires_partition_by=partitionings.Arbitrary(), preserves_partition_by=partitionings.Arbitrary())) - split = frame_base.wont_implement_method( - pd.core.strings.StringMethods, 'split', - reason='non-deferred-columns') + def _split_helper(self, rsplit=False, **kwargs): + expand = kwargs.get('expand', False) - rsplit = frame_base.wont_implement_method( - pd.core.strings.StringMethods, 'rsplit', - reason='non-deferred-columns') + if not expand: + # Not creating separate columns + proxy = self._expr.proxy() + if not rsplit: + func = lambda s: pd.concat([proxy, s.str.split(**kwargs)]) + else: + func = lambda s: pd.concat([proxy, s.str.rsplit(**kwargs)]) + else: + # Creating separate columns, so must be more strict on dtype + dtype = self._expr.proxy().dtype + if not isinstance(dtype, pd.CategoricalDtype): + method_name = 'rsplit' if rsplit else 'split' + raise frame_base.WontImplementError( + f"{method_name}() of non-categorical type is not supported because " + "the type of the output column depends on the data. Please use " + "pd.CategoricalDtype with explicit categories.", + reason="non-deferred-columns") + + # Split the categories + split_cats = dtype.categories.str.split(**kwargs) + + # Count the number of new columns to create for proxy + max_splits = len(max(split_cats, key=len)) + proxy = pd.DataFrame(columns=range(max_splits)) + + def func(s): + if not rsplit: + result = s.str.split(**kwargs) + else: + result = s.str.rsplit(**kwargs) + result[~result.isna()].replace(np.nan, value=None) + return result + + return frame_base.DeferredFrame.wrap( + expressions.ComputedExpression( + 'split', + func, + [self._expr], + proxy=proxy, + requires_partition_by=partitionings.Arbitrary(), + preserves_partition_by=partitionings.Arbitrary())) + + @frame_base.with_docs_from(pd.core.strings.StringMethods) + @frame_base.args_to_kwargs(pd.core.strings.StringMethods) + @frame_base.populate_defaults(pd.core.strings.StringMethods) + def split(self, **kwargs): + """ + Like other non-deferred methods, dtype must be CategoricalDtype. + One exception is when ``expand`` is ``False``. Because we are not + creating new columns at construction time, dtype can be `str`. + """ + return self._split_helper(rsplit=False, **kwargs) + + @frame_base.with_docs_from(pd.core.strings.StringMethods) + @frame_base.args_to_kwargs(pd.core.strings.StringMethods) + @frame_base.populate_defaults(pd.core.strings.StringMethods) + def rsplit(self, **kwargs): + """ + Like other non-deferred methods, dtype must be CategoricalDtype. + One exception is when ``expand`` is ``False``. Because we are not + creating new columns at construction time, dtype can be `str`. + """ + return self._split_helper(rsplit=True, **kwargs) ELEMENTWISE_STRING_METHODS = [ diff --git a/sdks/python/apache_beam/dataframe/frames_test.py b/sdks/python/apache_beam/dataframe/frames_test.py index ca4f561c9ab7..61ae056a7fa4 100644 --- a/sdks/python/apache_beam/dataframe/frames_test.py +++ b/sdks/python/apache_beam/dataframe/frames_test.py @@ -14,6 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. +import re import unittest import numpy as np @@ -2356,6 +2357,117 @@ def test_sample_with_weights_distribution(self): expected = num_samples * target_prob self.assertTrue(expected / 3 < result < expected * 2, (expected, result)) + def test_split_pandas_examples_no_expand(self): + # if expand=False (default), then no need to cast dtype to be + # CategoricalDtype. + s = pd.Series([ + "this is a regular sentence", + "https://docs.python.org/3/tutorial/index.html", + np.nan + ]) + result = self._evaluate(lambda s: s.str.split(), s) + self.assert_frame_data_equivalent(result, s.str.split()) + + result = self._evaluate(lambda s: s.str.rsplit(), s) + self.assert_frame_data_equivalent(result, s.str.rsplit()) + + result = self._evaluate(lambda s: s.str.split(n=2), s) + self.assert_frame_data_equivalent(result, s.str.split(n=2)) + + result = self._evaluate(lambda s: s.str.rsplit(n=2), s) + self.assert_frame_data_equivalent(result, s.str.rsplit(n=2)) + + result = self._evaluate(lambda s: s.str.split(pat="/"), s) + self.assert_frame_data_equivalent(result, s.str.split(pat="/")) + + def test_split_pandas_examples_expand_not_categorical(self): + # When expand=True, there is exception because series is not categorical + s = pd.Series([ + "this is a regular sentence", + "https://docs.python.org/3/tutorial/index.html", + np.nan + ]) + with self.assertRaisesRegex( + frame_base.WontImplementError, + r"split\(\) of non-categorical type is not supported"): + self._evaluate(lambda s: s.str.split(expand=True), s) + + with self.assertRaisesRegex( + frame_base.WontImplementError, + r"rsplit\(\) of non-categorical type is not supported"): + self._evaluate(lambda s: s.str.rsplit(expand=True), s) + + def test_split_pandas_examples_expand_pat_is_string_literal1(self): + # When expand=True and pattern is treated as a string literal + s = pd.Series([ + "this is a regular sentence", + "https://docs.python.org/3/tutorial/index.html", + np.nan + ]) + s = s.astype( + pd.CategoricalDtype( + categories=[ + 'this is a regular sentence', + 'https://docs.python.org/3/tutorial/index.html' + ])) + result = self._evaluate(lambda s: s.str.split(expand=True), s) + self.assert_frame_data_equivalent(result, s.str.split(expand=True)) + + result = self._evaluate(lambda s: s.str.rsplit("/", n=1, expand=True), s) + self.assert_frame_data_equivalent( + result, s.str.rsplit("/", n=1, expand=True)) + + @unittest.skipIf(PD_VERSION < (1, 4), "regex arg is new in pandas 1.4") + def test_split_pandas_examples_expand_pat_is_string_literal2(self): + # when regex is None (default) regex pat is string literal if len(pat) == 1 + s = pd.Series(['foojpgbar.jpg']).astype('category') + s = s.astype(pd.CategoricalDtype(categories=["foojpgbar.jpg"])) + result = self._evaluate(lambda s: s.str.split(r".", expand=True), s) + self.assert_frame_data_equivalent(result, s.str.split(r".", expand=True)) + + # When regex=False, pat is interpreted as the string itself + result = self._evaluate( + lambda s: s.str.split(r"\.jpg", regex=False, expand=True), s) + self.assert_frame_data_equivalent( + result, s.str.split(r"\.jpg", regex=False, expand=True)) + + @unittest.skipIf(PD_VERSION < (1, 4), "regex arg is new in pandas 1.4") + def test_split_pandas_examples_expand_pat_is_regex(self): + # when regex is None (default) regex pat is compiled if len(pat) != 1 + s = pd.Series(["foo and bar plus baz"]) + s = s.astype(pd.CategoricalDtype(categories=["foo and bar plus baz"])) + result = self._evaluate(lambda s: s.str.split(r"and|plus", expand=True), s) + self.assert_frame_data_equivalent( + result, s.str.split(r"and|plus", expand=True)) + + s = pd.Series(['foojpgbar.jpg']).astype('category') + s = s.astype(pd.CategoricalDtype(categories=["foojpgbar.jpg"])) + result = self._evaluate(lambda s: s.str.split(r"\.jpg", expand=True), s) + self.assert_frame_data_equivalent( + result, s.str.split(r"\.jpg", expand=True)) + + # When regex=True, pat is interpreted as a regex + result = self._evaluate( + lambda s: s.str.split(r"\.jpg", regex=True, expand=True), s) + self.assert_frame_data_equivalent( + result, s.str.split(r"\.jpg", regex=True, expand=True)) + + # A compiled regex can be passed as pat + result = self._evaluate( + lambda s: s.str.split(re.compile(r"\.jpg"), expand=True), s) + self.assert_frame_data_equivalent( + result, s.str.split(re.compile(r"\.jpg"), expand=True)) + + @unittest.skipIf(PD_VERSION < (1, 4), "regex arg is new in pandas 1.4") + def test_split_pat_is_regex(self): + # regex=True, but expand=False + s = pd.Series(['foojpgbar.jpg']).astype('category') + s = s.astype(pd.CategoricalDtype(categories=["foojpgbar.jpg"])) + result = self._evaluate( + lambda s: s.str.split(r"\.jpg", regex=True, expand=False), s) + self.assert_frame_data_equivalent( + result, s.str.split(r"\.jpg", regex=True, expand=False)) + class AllowNonParallelTest(unittest.TestCase): def _use_non_parallel_operation(self): diff --git a/sdks/python/apache_beam/dataframe/pandas_doctests_test.py b/sdks/python/apache_beam/dataframe/pandas_doctests_test.py index 19113af53e4f..05f06df854d9 100644 --- a/sdks/python/apache_beam/dataframe/pandas_doctests_test.py +++ b/sdks/python/apache_beam/dataframe/pandas_doctests_test.py @@ -584,8 +584,6 @@ def test_string_tests(self): f'{module_name}.StringMethods.get_dummies': ['*'], f'{module_name}.str_get_dummies': ['*'], f'{module_name}.StringMethods': ['s.str.split("_")'], - f'{module_name}.StringMethods.rsplit': ['*'], - f'{module_name}.StringMethods.split': ['*'], }, skip={ # count() on Series with a NaN produces mismatched type if we @@ -602,7 +600,32 @@ def test_string_tests(self): ], # output has incorrect formatting in 1.2.x - f'{module_name}.StringMethods.extractall': ['*'] + f'{module_name}.StringMethods.extractall': ['*'], + + # For split and rsplit, if expand=True, then the series + # must be of CategoricalDtype, which pandas doesn't convert to + f'{module_name}.StringMethods.rsplit': [ + 's.str.split(r"\\+|=", expand=True)', # for pandas<1.4 + 's.str.split(expand=True)', + 's.str.rsplit("/", n=1, expand=True)', + 's.str.split(r"and|plus", expand=True)', + 's.str.split(r".", expand=True)', + 's.str.split(r"\\.jpg", expand=True)', + 's.str.split(r"\\.jpg", regex=True, expand=True)', + 's.str.split(re.compile(r"\\.jpg"), expand=True)', + 's.str.split(r"\\.jpg", regex=False, expand=True)' + ], + f'{module_name}.StringMethods.split': [ + 's.str.split(r"\\+|=", expand=True)', # for pandas<1.4 + 's.str.split(expand=True)', + 's.str.rsplit("/", n=1, expand=True)', + 's.str.split(r"and|plus", expand=True)', + 's.str.split(r".", expand=True)', + 's.str.split(r"\\.jpg", expand=True)', + 's.str.split(r"\\.jpg", regex=True, expand=True)', + 's.str.split(re.compile(r"\\.jpg"), expand=True)', + 's.str.split(r"\\.jpg", regex=False, expand=True)' + ] }) self.assertEqual(result.failed, 0) From d2aac3b67103d1d7b0f09bbd6aea46d450c9f830 Mon Sep 17 00:00:00 2001 From: Rahul Iyer Date: Tue, 15 Feb 2022 23:11:12 -0800 Subject: [PATCH 42/68] py: Import beam plugins before starting SdkHarness --- .../apache_beam/options/pipeline_options.py | 2 +- .../runners/worker/sdk_worker_main.py | 24 +++++++++++++++++++ .../runners/worker/sdk_worker_main_test.py | 8 +++++++ 3 files changed, 33 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index eb5235b796b7..e158c98ac800 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -1081,7 +1081,7 @@ def _add_argparse_args(cls, parser): 'custom code.')) parser.add_argument( '--beam_plugin', - '--beam_plugin', + '--beam_plugins', dest='beam_plugins', action='append', default=None, diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker_main.py b/sdks/python/apache_beam/runners/worker/sdk_worker_main.py index 30f5c39fc6a7..c96f286ecd09 100644 --- a/sdks/python/apache_beam/runners/worker/sdk_worker_main.py +++ b/sdks/python/apache_beam/runners/worker/sdk_worker_main.py @@ -19,6 +19,7 @@ # pytype: skip-file +import importlib import json import logging import os @@ -48,6 +49,25 @@ _ENABLE_GOOGLE_CLOUD_PROFILER = 'enable_google_cloud_profiler' +def _import_beam_plugins(plugins): + for plugin in plugins: + try: + importlib.import_module(plugin) + _LOGGER.info('Imported beam-plugin %s', plugin) + except ImportError: + try: + _LOGGER.debug(( + "Looks like %s is not a module. " + "Trying to import it assuming it's a class"), + plugin) + module, _ = plugin.rsplit('.', 1) + importlib.import_module(module) + _LOGGER.info('Imported %s for beam-plugin %s', module, plugin) + except ImportError as exc: + raise RuntimeError( + 'Failed to import beam-plugin: {}'.format(plugin)) from exc + + def create_harness(environment, dry_run=False): """Creates SDK Fn Harness.""" if 'LOGGING_API_SERVICE_DESCRIPTOR' in environment: @@ -116,6 +136,10 @@ def create_harness(environment, dry_run=False): experiments = sdk_pipeline_options.view_as(DebugOptions).experiments or [] enable_heap_dump = 'enable_heap_dump' in experiments + + beam_plugins = sdk_pipeline_options.view_as(SetupOptions).beam_plugins or [] + _import_beam_plugins(beam_plugins) + if dry_run: return sdk_harness = SdkHarness( diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker_main_test.py b/sdks/python/apache_beam/runners/worker/sdk_worker_main_test.py index ea6a0728a555..c365f7b8477a 100644 --- a/sdks/python/apache_beam/runners/worker/sdk_worker_main_test.py +++ b/sdks/python/apache_beam/runners/worker/sdk_worker_main_test.py @@ -30,6 +30,7 @@ from apache_beam.options.value_provider import RuntimeValueProvider from apache_beam.runners.worker import sdk_worker_main from apache_beam.runners.worker import worker_status +from apache_beam.utils.plugin import BeamPlugin class SdkWorkerMainTest(unittest.TestCase): @@ -99,6 +100,13 @@ def test_runtime_values(self): self.assertTrue(test_runtime_provider.is_accessible()) self.assertEqual(test_runtime_provider.get(), 37) + def test_import_beam_plugins(self): + sdk_worker_main._import_beam_plugins(BeamPlugin.get_all_plugin_paths()) + + def test_import_beam_plugins_failure(self): + with self.assertRaises(RuntimeError): + sdk_worker_main._import_beam_plugins(['this.plugin.does.not.exist']) + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) From 0e53a7e768fc35d6c380d00981dce06a18d9cbfd Mon Sep 17 00:00:00 2001 From: abhijeet-lele <56114083+abhijeet-lele@users.noreply.github.com> Date: Sat, 5 Mar 2022 04:33:15 +0530 Subject: [PATCH 43/68] BEAM-14026 - Fixes bug related to Unnesting nested rows in an array (#16988) * Suggested changes to handle nested row in an array * Beam-14026 Suggested changes to handle nested row in an array * Beam-14026 Enhanced by segregating the code from getBaseValues enhanced test case and example. * Beam-14026 The code is moved from Row to avoid impact to the public interface. The code is moved to BeamUnnestRel.java since its the caller class. The Example code was duplicate, hence dropped. build.gradle updated with the removal of example code. --- .../sql/impl/rel/BeamUnnestRel.java | 35 ++++- .../sql/BeamSqlDslUnnestRowsTest.java | 142 ++++++++++++++++++ 2 files changed, 176 insertions(+), 1 deletion(-) create mode 100644 sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUnnestRowsTest.java diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnnestRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnnestRel.java index 655d75a01e1b..d454dfe1aa00 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnnestRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnnestRel.java @@ -17,8 +17,13 @@ */ package org.apache.beam.sdk.extensions.sql.impl.rel; +import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel; import org.apache.beam.sdk.extensions.sql.impl.planner.BeamRelMetadataQuery; import org.apache.beam.sdk.extensions.sql.impl.planner.NodeStats; @@ -129,6 +134,34 @@ private UnnestFn(Schema outputSchema, List unnestIndices) { this.outputSchema = outputSchema; this.unnestIndices = unnestIndices; } + /** + * This is recursive call to get all the values of the nested rows. The recusion is bounded by + * the amount of nesting with in the data. This mirrors the unnest behavior of calcite towards + * schema. * + */ + private List getNestedRowBaseValues(Row nestedRow) { + return IntStream.range(0, nestedRow.getFieldCount()) + .mapToObj( + (i) -> { + List values = new ArrayList<>(); + Schema.FieldType fieldType = nestedRow.getSchema().getField(i).getType(); + if (fieldType.getTypeName().equals(Schema.TypeName.ROW)) { + @Nullable Row row = nestedRow.getBaseValue(i, Row.class); + if (row == null) { + return Stream.builder().build(); + } + List rowValues = getNestedRowBaseValues(row); + if (null != rowValues) { + values.addAll(rowValues); + } + } else { + values.add(nestedRow.getBaseValue(i)); + } + return values.stream(); + }) + .flatMap(Function.identity()) + .collect(Collectors.toList()); + } @ProcessElement public void process(@Element Row row, OutputReceiver out) { @@ -157,7 +190,7 @@ public void process(@Element Row row, OutputReceiver out) { out.output( Row.withSchema(outputSchema) .addValues(row.getBaseValues()) - .addValues(nestedRow.getBaseValues()) + .addValues(getNestedRowBaseValues(nestedRow)) .build()); } else { out.output( diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUnnestRowsTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUnnestRowsTest.java new file mode 100644 index 000000000000..d72bc249154a --- /dev/null +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUnnestRowsTest.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.extensions.sql; + +import java.util.Arrays; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TupleTag; +import org.junit.Rule; +import org.junit.Test; + +/** Tests for nested rows handling. */ +public class BeamSqlDslUnnestRowsTest { + + @Rule public final TestPipeline pipeline = TestPipeline.create(); + + /** + * TODO([BEAM-14026]): This is a test of the incorrect behavior unnest because calcite flattens + * the row. + */ + @Test + public void testUnnestArrayWithNestedRows() { + + Schema level3Type = + Schema.builder().addInt32Field("c1").addStringField("c2").addDoubleField("c3").build(); + + Row level3Row1 = Row.withSchema(level3Type).addValues(1, "row", 1.0).build(); + Row level3Row2 = Row.withSchema(level3Type).addValues(2, "row", 2.0).build(); + Row level3Row3 = Row.withSchema(level3Type).addValues(3, "row", 3.0).build(); + + // define the input row format level3 + Schema level2Type = + Schema.builder() + .addInt32Field("b1") + .addStringField("b2") + .addRowField("b3", level3Type) + .addDoubleField("b4") + .build(); + + Row level2Row1 = Row.withSchema(level2Type).addValues(1, "row", level3Row1, 1.0).build(); + Row level2Row2 = Row.withSchema(level2Type).addValues(2, "row", level3Row2, 2.0).build(); + Row level2Row3 = Row.withSchema(level2Type).addValues(3, "row", level3Row3, 3.0).build(); + + // define the input row format level3 + Schema level1Type = + Schema.builder() + .addInt32Field("a1") + .addStringField("a2") + .addDoubleField("a3") + .addArrayField("a4", Schema.FieldType.row(level2Type)) + .build(); + Row level1Row1 = + Row.withSchema(level1Type) + .addValues(1, "row", 1.0, Arrays.asList(level2Row1, level2Row2, level2Row3)) + .build(); + Row level1Row2 = + Row.withSchema(level1Type) + .addValues(2, "row", 2.0, Arrays.asList(level2Row1, level2Row2, level2Row3)) + .build(); + Row level1Row3 = + Row.withSchema(level1Type) + .addValues(3, "row", 3.0, Arrays.asList(level2Row1, level2Row2, level2Row3)) + .build(); + + // create a source PCollection with Create.of(); + PCollection inputTable = + PBegin.in(pipeline) + .apply(Create.of(level1Row1, level1Row2, level1Row3).withRowSchema(level1Type)); + + String sql = + "select t.a1, t.a2, t.a3, d.b1, d.b2, d.b4, " + + "d.b3.c1, d.b3.c2, d.b3.c3 from test t cross join unnest(t.a4) d"; + // Case 1. run a simple SQL query over input PCollection with BeamSql.simpleQuery; + PCollection result = + PCollectionTuple.of(new TupleTag<>("test"), inputTable).apply(SqlTransform.query(sql)); + + Schema resultSchema = + Schema.builder() + .addInt32Field("a1") + .addStringField("a2") + .addDoubleField("a3") + .addInt32Field("b1") + .addStringField("b2") + .addDoubleField("b4") + .addInt32Field("c1") + .addStringField("c2") + .addDoubleField("c3") + .build(); + + PAssert.that(result) + .containsInAnyOrder( + Row.withSchema(resultSchema) + .addValues(1, "row", 1.0, 1, "row", 1.0, 1, "row", 1.0) + .build(), + Row.withSchema(resultSchema) + .addValues(1, "row", 1.0, 2, "row", 2.0, 2, "row", 2.0) + .build(), + Row.withSchema(resultSchema) + .addValues(1, "row", 1.0, 3, "row", 3.0, 3, "row", 3.0) + .build(), + Row.withSchema(resultSchema) + .addValues(3, "row", 3.0, 1, "row", 1.0, 1, "row", 1.0) + .build(), + Row.withSchema(resultSchema) + .addValues(3, "row", 3.0, 2, "row", 2.0, 2, "row", 2.0) + .build(), + Row.withSchema(resultSchema) + .addValues(3, "row", 3.0, 3, "row", 3.0, 3, "row", 3.0) + .build(), + Row.withSchema(resultSchema) + .addValues(2, "row", 2.0, 1, "row", 1.0, 1, "row", 1.0) + .build(), + Row.withSchema(resultSchema) + .addValues(2, "row", 2.0, 2, "row", 2.0, 2, "row", 2.0) + .build(), + Row.withSchema(resultSchema) + .addValues(2, "row", 2.0, 3, "row", 3.0, 3, "row", 3.0) + .build()); + pipeline.run(); + } +} From d3a4e72da68cbcf89b0ca1e8ca3167117e2a1595 Mon Sep 17 00:00:00 2001 From: David Cavazos Date: Fri, 4 Mar 2022 13:58:42 -0800 Subject: [PATCH 44/68] Remove resolved issue in notebook --- .../documentation/transforms/python/elementwise/pardo-py.ipynb | 2 -- 1 file changed, 2 deletions(-) diff --git a/examples/notebooks/documentation/transforms/python/elementwise/pardo-py.ipynb b/examples/notebooks/documentation/transforms/python/elementwise/pardo-py.ipynb index c0125c97841a..b74887aea3c4 100644 --- a/examples/notebooks/documentation/transforms/python/elementwise/pardo-py.ipynb +++ b/examples/notebooks/documentation/transforms/python/elementwise/pardo-py.ipynb @@ -365,8 +365,6 @@ "\n", "> *Known issues:*\n", ">\n", - "> * [[BEAM-7885]](https://issues.apache.org/jira/browse/BEAM-7885)\n", - "> `DoFn.setup()` doesn't run for streaming jobs running in the `DirectRunner`.\n", "> * [[BEAM-7340]](https://issues.apache.org/jira/browse/BEAM-7340)\n", "> `DoFn.teardown()` metrics are lost." ] From 2799d01a244a8111a644561a668ea29df9fbddd5 Mon Sep 17 00:00:00 2001 From: Valentyn Tymofieiev Date: Thu, 3 Mar 2022 19:47:48 -0800 Subject: [PATCH 45/68] Bump numpy bound to include 1.22 and regenerate container deps. --- .../py36/base_image_requirements.txt | 37 +++++++-------- .../py37/base_image_requirements.txt | 42 ++++++++--------- .../py38/base_image_requirements.txt | 46 +++++++++---------- .../py39/base_image_requirements.txt | 46 +++++++++---------- sdks/python/setup.py | 2 +- 5 files changed, 87 insertions(+), 86 deletions(-) diff --git a/sdks/python/container/py36/base_image_requirements.txt b/sdks/python/container/py36/base_image_requirements.txt index ecfa3f30d198..592bdb01113e 100644 --- a/sdks/python/container/py36/base_image_requirements.txt +++ b/sdks/python/container/py36/base_image_requirements.txt @@ -31,13 +31,13 @@ cached-property==1.5.2 cachetools==4.2.4 certifi==2021.10.8 cffi==1.15.0 -charset-normalizer==2.0.11 +charset-normalizer==2.0.12 clang==5.0 -click==8.0.3 +click==8.0.4 cloudpickle==2.0.0 crcmod==1.7 cryptography==36.0.1 -Cython==0.29.27 +Cython==0.29.28 dataclasses==0.8 deprecation==2.1.0 dill==0.3.1.1 @@ -47,25 +47,25 @@ execnet==1.9.0 fastavro==1.4.7 fasteners==0.17.3 flatbuffers==1.12 -freezegun==1.1.0 +freezegun==1.2.0 future==0.18.2 gast==0.4.0 google-api-core==1.31.5 -google-api-python-client==2.37.0 +google-api-python-client==2.39.0 google-apitools==0.5.31 google-auth==1.35.0 google-auth-httplib2==0.1.0 google-auth-oauthlib==0.4.6 -google-cloud-bigquery==2.32.0 -google-cloud-bigquery-storage==2.11.0 +google-cloud-bigquery==2.34.1 +google-cloud-bigquery-storage==2.12.0 google-cloud-bigtable==1.7.0 google-cloud-core==1.7.2 google-cloud-datastore==1.15.3 -google-cloud-dlp==3.6.0 +google-cloud-dlp==3.6.1 google-cloud-language==1.3.0 google-cloud-profiler==3.0.7 google-cloud-pubsub==2.9.0 -google-cloud-pubsublite==1.3.0 +google-cloud-pubsublite==1.4.0 google-cloud-recommendations-ai==0.2.0 google-cloud-spanner==1.19.1 google-cloud-videointelligence==1.16.1 @@ -73,19 +73,20 @@ google-cloud-vision==1.0.0 google-crc32c==1.3.0 google-pasta==0.2.0 google-python-cloud-debugger==2.18 -google-resumable-media==2.2.1 -googleapis-common-protos==1.54.0 +google-resumable-media==2.3.1 +googleapis-common-protos==1.55.0 greenlet==1.1.2 grpc-google-iam-v1==0.12.3 -grpcio==1.43.0 +grpcio==1.44.0 grpcio-gcp==0.2.2 -grpcio-status==1.43.0 +grpcio-status==1.44.0 guppy3==3.1.2 h5py==3.1.0 hdfs==2.6.0 httplib2==0.19.1 idna==3.3 importlib-metadata==4.8.3 +importlib-resources==5.4.0 joblib==1.1.0 keras==2.6.0 Keras-Preprocessing==1.1.2 @@ -108,7 +109,7 @@ pandas==1.1.5 parameterized==0.7.5 pbr==5.8.1 pluggy==0.13.1 -proto-plus==1.20.0 +proto-plus==1.20.3 protobuf==3.19.4 psycopg2-binary==2.9.3 py==1.11.0 @@ -126,10 +127,10 @@ pytest-forked==1.4.0 pytest-timeout==1.4.2 pytest-xdist==1.34.0 python-dateutil==2.8.2 -python-snappy==0.6.0 +python-snappy==0.6.1 pytz==2021.3 PyYAML==6.0 -regex==2022.1.18 +regex==2022.3.2 requests==2.27.1 requests-mock==1.9.3 requests-oauthlib==1.3.1 @@ -148,14 +149,14 @@ tensorflow-estimator==2.6.0 termcolor==1.1.0 testcontainers==3.4.2 threadpoolctl==3.1.0 -tqdm==4.62.3 +tqdm==4.63.0 typing-extensions==3.7.4.3 typing-inspect==0.7.1 typing-utils==0.1.0 uritemplate==4.1.1 urllib3==1.26.8 wcwidth==0.2.5 -websocket-client==1.2.3 +websocket-client==1.3.1 Werkzeug==2.0.3 wrapt==1.12.1 zipp==3.6.0 diff --git a/sdks/python/container/py37/base_image_requirements.txt b/sdks/python/container/py37/base_image_requirements.txt index c62ec03c2ed6..41ab489ea60d 100644 --- a/sdks/python/container/py37/base_image_requirements.txt +++ b/sdks/python/container/py37/base_image_requirements.txt @@ -31,12 +31,12 @@ cached-property==1.5.2 cachetools==4.2.4 certifi==2021.10.8 cffi==1.15.0 -charset-normalizer==2.0.11 -click==8.0.3 +charset-normalizer==2.0.12 +click==8.0.4 cloudpickle==2.0.0 crcmod==1.7 cryptography==36.0.1 -Cython==0.29.27 +Cython==0.29.28 deprecation==2.1.0 dill==0.3.1.1 docker==5.0.3 @@ -45,25 +45,25 @@ execnet==1.9.0 fastavro==1.4.9 fasteners==0.17.3 flatbuffers==2.0 -freezegun==1.1.0 +freezegun==1.2.0 future==0.18.2 gast==0.5.3 google-api-core==1.31.5 -google-api-python-client==2.37.0 +google-api-python-client==2.39.0 google-apitools==0.5.31 google-auth==1.35.0 google-auth-httplib2==0.1.0 google-auth-oauthlib==0.4.6 -google-cloud-bigquery==2.32.0 -google-cloud-bigquery-storage==2.11.0 +google-cloud-bigquery==2.34.1 +google-cloud-bigquery-storage==2.12.0 google-cloud-bigtable==1.7.0 google-cloud-core==1.7.2 google-cloud-datastore==1.15.3 -google-cloud-dlp==3.6.0 +google-cloud-dlp==3.6.1 google-cloud-language==1.3.0 google-cloud-profiler==3.0.7 google-cloud-pubsub==2.9.0 -google-cloud-pubsublite==1.3.0 +google-cloud-pubsublite==1.4.0 google-cloud-recommendations-ai==0.2.0 google-cloud-spanner==1.19.1 google-cloud-videointelligence==1.16.1 @@ -71,19 +71,19 @@ google-cloud-vision==1.0.0 google-crc32c==1.3.0 google-pasta==0.2.0 google-python-cloud-debugger==2.18 -google-resumable-media==2.2.1 -googleapis-common-protos==1.54.0 +google-resumable-media==2.3.1 +googleapis-common-protos==1.55.0 greenlet==1.1.2 grpc-google-iam-v1==0.12.3 -grpcio==1.43.0 +grpcio==1.44.0 grpcio-gcp==0.2.2 -grpcio-status==1.43.0 +grpcio-status==1.44.0 guppy3==3.1.2 h5py==3.6.0 hdfs==2.6.0 httplib2==0.19.1 idna==3.3 -importlib-metadata==4.11.0 +importlib-metadata==4.11.2 joblib==1.1.0 keras==2.8.0 Keras-Preprocessing==1.1.2 @@ -100,14 +100,14 @@ numpy==1.21.5 oauth2client==4.1.3 oauthlib==3.2.0 opt-einsum==3.3.0 -orjson==3.6.6 +orjson==3.6.7 overrides==6.1.0 packaging==21.3 pandas==1.3.5 parameterized==0.7.5 pbr==5.8.1 pluggy==0.13.1 -proto-plus==1.20.0 +proto-plus==1.20.3 protobuf==3.19.4 psycopg2-binary==2.9.3 py==1.11.0 @@ -125,10 +125,10 @@ pytest-forked==1.4.0 pytest-timeout==1.4.2 pytest-xdist==1.34.0 python-dateutil==2.8.2 -python-snappy==0.6.0 +python-snappy==0.6.1 pytz==2021.3 PyYAML==6.0 -regex==2022.1.18 +regex==2022.3.2 requests==2.27.1 requests-mock==1.9.3 requests-oauthlib==1.3.1 @@ -148,14 +148,14 @@ termcolor==1.1.0 testcontainers==3.4.2 tf-estimator-nightly==2.8.0.dev2021122109 threadpoolctl==3.1.0 -tqdm==4.62.3 +tqdm==4.63.0 typing-inspect==0.7.1 typing-utils==0.1.0 -typing_extensions==4.0.1 +typing_extensions==4.1.1 uritemplate==4.1.1 urllib3==1.26.8 wcwidth==0.2.5 -websocket-client==1.2.3 +websocket-client==1.3.1 Werkzeug==2.0.3 wrapt==1.13.3 zipp==3.7.0 diff --git a/sdks/python/container/py38/base_image_requirements.txt b/sdks/python/container/py38/base_image_requirements.txt index 42044fdd69f7..ea2430810845 100644 --- a/sdks/python/container/py38/base_image_requirements.txt +++ b/sdks/python/container/py38/base_image_requirements.txt @@ -30,12 +30,12 @@ bs4==0.0.1 cachetools==4.2.4 certifi==2021.10.8 cffi==1.15.0 -charset-normalizer==2.0.11 -click==8.0.3 +charset-normalizer==2.0.12 +click==8.0.4 cloudpickle==2.0.0 crcmod==1.7 cryptography==36.0.1 -Cython==0.29.27 +Cython==0.29.28 deprecation==2.1.0 dill==0.3.1.1 docker==5.0.3 @@ -44,25 +44,25 @@ execnet==1.9.0 fastavro==1.4.9 fasteners==0.17.3 flatbuffers==2.0 -freezegun==1.1.0 +freezegun==1.2.0 future==0.18.2 gast==0.5.3 google-api-core==1.31.5 -google-api-python-client==2.37.0 +google-api-python-client==2.39.0 google-apitools==0.5.31 google-auth==1.35.0 google-auth-httplib2==0.1.0 google-auth-oauthlib==0.4.6 -google-cloud-bigquery==2.32.0 -google-cloud-bigquery-storage==2.11.0 +google-cloud-bigquery==2.34.1 +google-cloud-bigquery-storage==2.12.0 google-cloud-bigtable==1.7.0 google-cloud-core==1.7.2 google-cloud-datastore==1.15.3 -google-cloud-dlp==3.6.0 +google-cloud-dlp==3.6.1 google-cloud-language==1.3.0 google-cloud-profiler==3.0.7 google-cloud-pubsub==2.9.0 -google-cloud-pubsublite==1.3.0 +google-cloud-pubsublite==1.4.0 google-cloud-recommendations-ai==0.2.0 google-cloud-spanner==1.19.1 google-cloud-videointelligence==1.16.1 @@ -70,19 +70,19 @@ google-cloud-vision==1.0.0 google-crc32c==1.3.0 google-pasta==0.2.0 google-python-cloud-debugger==2.18 -google-resumable-media==2.2.1 -googleapis-common-protos==1.54.0 +google-resumable-media==2.3.1 +googleapis-common-protos==1.55.0 greenlet==1.1.2 grpc-google-iam-v1==0.12.3 -grpcio==1.43.0 +grpcio==1.44.0 grpcio-gcp==0.2.2 -grpcio-status==1.43.0 +grpcio-status==1.44.0 guppy3==3.1.2 h5py==3.6.0 hdfs==2.6.0 httplib2==0.19.1 idna==3.3 -importlib-metadata==4.11.0 +importlib-metadata==4.11.2 joblib==1.1.0 keras==2.8.0 Keras-Preprocessing==1.1.2 @@ -95,18 +95,18 @@ more-itertools==8.12.0 mypy-extensions==0.4.3 nltk==3.7 nose==1.3.7 -numpy==1.21.5 +numpy==1.22.2 oauth2client==4.1.3 oauthlib==3.2.0 opt-einsum==3.3.0 -orjson==3.6.6 +orjson==3.6.7 overrides==6.1.0 packaging==21.3 -pandas==1.4.0 +pandas==1.4.1 parameterized==0.7.5 pbr==5.8.1 pluggy==0.13.1 -proto-plus==1.20.0 +proto-plus==1.20.3 protobuf==3.19.4 psycopg2-binary==2.9.3 py==1.11.0 @@ -124,10 +124,10 @@ pytest-forked==1.4.0 pytest-timeout==1.4.2 pytest-xdist==1.34.0 python-dateutil==2.8.2 -python-snappy==0.6.0 +python-snappy==0.6.1 pytz==2021.3 PyYAML==6.0 -regex==2022.1.18 +regex==2022.3.2 requests==2.27.1 requests-mock==1.9.3 requests-oauthlib==1.3.1 @@ -147,14 +147,14 @@ termcolor==1.1.0 testcontainers==3.4.2 tf-estimator-nightly==2.8.0.dev2021122109 threadpoolctl==3.1.0 -tqdm==4.62.3 +tqdm==4.63.0 typing-inspect==0.7.1 typing-utils==0.1.0 -typing_extensions==4.0.1 +typing_extensions==4.1.1 uritemplate==4.1.1 urllib3==1.26.8 wcwidth==0.2.5 -websocket-client==1.2.3 +websocket-client==1.3.1 Werkzeug==2.0.3 wrapt==1.13.3 zipp==3.7.0 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index e5226aa4f9ab..157045b51c08 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -30,12 +30,12 @@ bs4==0.0.1 cachetools==4.2.4 certifi==2021.10.8 cffi==1.15.0 -charset-normalizer==2.0.11 -click==8.0.3 +charset-normalizer==2.0.12 +click==8.0.4 cloudpickle==2.0.0 crcmod==1.7 cryptography==36.0.1 -Cython==0.29.27 +Cython==0.29.28 deprecation==2.1.0 dill==0.3.1.1 docker==5.0.3 @@ -44,25 +44,25 @@ execnet==1.9.0 fastavro==1.4.9 fasteners==0.17.3 flatbuffers==2.0 -freezegun==1.1.0 +freezegun==1.2.0 future==0.18.2 gast==0.5.3 google-api-core==1.31.5 -google-api-python-client==2.37.0 +google-api-python-client==2.39.0 google-apitools==0.5.31 google-auth==1.35.0 google-auth-httplib2==0.1.0 google-auth-oauthlib==0.4.6 -google-cloud-bigquery==2.32.0 -google-cloud-bigquery-storage==2.11.0 +google-cloud-bigquery==2.34.1 +google-cloud-bigquery-storage==2.12.0 google-cloud-bigtable==1.7.0 google-cloud-core==1.7.2 google-cloud-datastore==1.15.3 -google-cloud-dlp==3.6.0 +google-cloud-dlp==3.6.1 google-cloud-language==1.3.0 google-cloud-profiler==3.0.7 google-cloud-pubsub==2.9.0 -google-cloud-pubsublite==1.3.0 +google-cloud-pubsublite==1.4.0 google-cloud-recommendations-ai==0.2.0 google-cloud-spanner==1.19.1 google-cloud-videointelligence==1.16.1 @@ -70,19 +70,19 @@ google-cloud-vision==1.0.0 google-crc32c==1.3.0 google-pasta==0.2.0 google-python-cloud-debugger==2.18 -google-resumable-media==2.2.1 -googleapis-common-protos==1.54.0 +google-resumable-media==2.3.1 +googleapis-common-protos==1.55.0 greenlet==1.1.2 grpc-google-iam-v1==0.12.3 -grpcio==1.43.0 +grpcio==1.44.0 grpcio-gcp==0.2.2 -grpcio-status==1.43.0 +grpcio-status==1.44.0 guppy3==3.1.2 h5py==3.6.0 hdfs==2.6.0 httplib2==0.19.1 idna==3.3 -importlib-metadata==4.11.0 +importlib-metadata==4.11.2 joblib==1.1.0 keras==2.8.0 Keras-Preprocessing==1.1.2 @@ -95,18 +95,18 @@ more-itertools==8.12.0 mypy-extensions==0.4.3 nltk==3.7 nose==1.3.7 -numpy==1.21.5 +numpy==1.22.2 oauth2client==4.1.3 oauthlib==3.2.0 opt-einsum==3.3.0 -orjson==3.6.6 +orjson==3.6.7 overrides==6.1.0 packaging==21.3 -pandas==1.4.0 +pandas==1.4.1 parameterized==0.7.5 pbr==5.8.1 pluggy==0.13.1 -proto-plus==1.20.0 +proto-plus==1.20.3 protobuf==3.19.4 psycopg2-binary==2.9.3 py==1.11.0 @@ -124,10 +124,10 @@ pytest-forked==1.4.0 pytest-timeout==1.4.2 pytest-xdist==1.34.0 python-dateutil==2.8.2 -python-snappy==0.6.0 +python-snappy==0.6.1 pytz==2021.3 PyYAML==6.0 -regex==2022.1.18 +regex==2022.3.2 requests==2.27.1 requests-mock==1.9.3 requests-oauthlib==1.3.1 @@ -147,14 +147,14 @@ termcolor==1.1.0 testcontainers==3.4.2 tf-estimator-nightly==2.8.0.dev2021122109 threadpoolctl==3.1.0 -tqdm==4.62.3 +tqdm==4.63.0 typing-inspect==0.7.1 typing-utils==0.1.0 -typing_extensions==4.0.1 +typing_extensions==4.1.1 uritemplate==4.1.1 urllib3==1.26.8 wcwidth==0.2.5 -websocket-client==1.2.3 +websocket-client==1.3.1 Werkzeug==2.0.3 wrapt==1.13.3 zipp==3.7.0 diff --git a/sdks/python/setup.py b/sdks/python/setup.py index ad99e1d590cd..a0239fa22441 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -142,7 +142,7 @@ def get_version(): 'grpcio>=1.29.0,<2', 'hdfs>=2.1.0,<3.0.0', 'httplib2>=0.8,<0.20.0', - 'numpy>=1.14.3,<1.22.0', + 'numpy>=1.14.3,<1.23.0', 'pymongo>=3.8.0,<4.0.0', 'oauth2client>=2.0.1,<5', 'protobuf>=3.12.2,<4', From 3e8cef62f8d46311569adcacb61e23ff6b2f0bad Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Sun, 6 Mar 2022 23:55:28 -0500 Subject: [PATCH 46/68] [BEAM-13925] Add ability to get metrics on pr-bot performance (#16985) * Add script to get metrics on pr-bot performance * Respond to feedback * fix bad condition --- scripts/ci/pr-bot/gatherMetrics.ts | 528 +++++++++++++++++++++++++++++ 1 file changed, 528 insertions(+) create mode 100644 scripts/ci/pr-bot/gatherMetrics.ts diff --git a/scripts/ci/pr-bot/gatherMetrics.ts b/scripts/ci/pr-bot/gatherMetrics.ts new file mode 100644 index 000000000000..66d0854c89d3 --- /dev/null +++ b/scripts/ci/pr-bot/gatherMetrics.ts @@ -0,0 +1,528 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +const fs = require("fs"); +const github = require("./shared/githubUtils"); +const { + REPO_OWNER, + REPO, + PATH_TO_METRICS_CSV, + BOT_NAME, +} = require("./shared/constants"); + +interface PrStats { + firstTimeContribution: boolean; + timeToFirstReview: number; + timeFromCreationToCompletion: number; + timeFromReviewersMentionedToCompletion: { [key: string]: number }; + mergedDate: Date; +} + +interface AggregatedMetrics { + prsCompleted: number; + prsCompletedByNewContributors: number; + averageTimeToFirstReview: number; + averageTimeToNewContributorFirstReview: number; + averageTimeCreationToCompletion: number; + numUsersPerformingReviews: number; + numCommittersPerformingReviews: number; + numNonCommittersPerformingReviews: number; + giniIndexCommittersPerformingReviews: number; + averageTimeFromCommitterAssignmentToPrMerge: number; +} + +async function getCompletedPullsFromLastYear(): Promise { + const cutoffDate = new Date(); + cutoffDate.setFullYear(new Date().getFullYear() - 1); + console.log(`Getting PRs newer than ${cutoffDate}`); + const githubClient = github.getGitHubClient(); + let result = await githubClient.rest.pulls.list({ + owner: REPO_OWNER, + repo: REPO, + state: "closed", + per_page: 100, // max allowed + }); + let page = 2; + let retries = 0; + let pulls = result.data; + while ( + result.data.length > 0 && + new Date(result.data[result.data.length - 1].created_at) > cutoffDate + ) { + if (retries === 0) { + console.log(`Getting PRs, page: ${page}`); + console.log( + `Current oldest PR = ${new Date( + result.data[result.data.length - 1].created_at + )}` + ); + } + try { + result = await githubClient.rest.pulls.list({ + owner: REPO_OWNER, + repo: REPO, + state: "closed", + per_page: 100, // max allowed + page: page, + }); + pulls = pulls.concat(result.data); + page++; + retries = 0; + } catch (err) { + if (retries >= 3) { + throw err; + } + retries++; + } + } + console.log("Got all PRs, moving to the processing stage"); + return pulls; +} + +// Rather than check the whole repo history (expensive), +// we'll just look at the last year's worth of contributions to check if they're a first time contributor. +// If they contributed > 1 year ago, their experience is probably similar to a first time contributor anyways. +function checkIfFirstTimeContributor( + pull: any, + pullsFromLastYear: any[] +): boolean { + return !pullsFromLastYear.some( + (pullFromLastYear) => + pullFromLastYear.created_at < pull.created_at && + pullFromLastYear.user.login === pull.user.login + ); +} + +// Get time between pr creation and the first comment, approval, or merge that isn't done by: +// (a) the author +// (b) automated tooling +function getTimeToFirstReview( + pull: any, + comments: any[], + reviews: any[], + creationDate: Date, + mergedDate: Date +): number { + let timeToFirstReview = mergedDate.getTime() - creationDate.getTime(); + + const firstReviewed = reviews.find( + (review) => review.user.login != pull.user.login + ); + if (firstReviewed) { + const firstReviewDate = new Date(firstReviewed.submitted_at); + timeToFirstReview = Math.min( + timeToFirstReview, + firstReviewDate.getTime() - creationDate.getTime() + ); + } + for (const comment of comments) { + if ( + comment.user.login != pull.user.login && + comment.user.login != BOT_NAME + ) { + let commentTime = new Date(comment.created_at); + timeToFirstReview = Math.min( + timeToFirstReview, + commentTime.getTime() - creationDate.getTime() + ); + } + } + return timeToFirstReview; +} + +// Takes a R: @reviewer comment and extracts all reviewers tagged +// Returns an empty list if no reviewer can be extracted +function extractReviewersTaggedFromCommentBody(body: string): string[] { + if (!body) { + return []; + } + body = body.toLowerCase(); + if (body.indexOf("r: @") < 0) { + return []; + } + let usernames: string[] = []; + const reviewerStrings = body.split(" @"); + // Start at index 1 since we don't care about anything before the first @ + for (let i = 1; i < reviewerStrings.length; i++) { + const curBlock = reviewerStrings[i]; + let usernameIndex = 0; + let curUsername = ""; + while ( + usernameIndex < curBlock.length && + curBlock[usernameIndex].match(/^[0-9a-z]+$/) + ) { + curUsername += curBlock[usernameIndex]; + usernameIndex += 1; + } + // Filter out username from PR template + if (curUsername && curUsername != "username") { + usernames.push(curUsername); + } + } + + return usernames; +} + +// Returns a dictionary mapping reviewers to the amount of time from their first comment to pr completion. +function getTimeFromReviewerMentionedToCompletion( + pull: any, + comments: any[], + reviewComments: any[], + mergedDate: Date +): { [key: string]: number } { + comments = comments.concat(reviewComments); + comments.push(pull); + let timeToCompletionPerReviewer = {}; + for (const comment of comments) { + const reviewersTagged = extractReviewersTaggedFromCommentBody(comment.body); + const commentCreationDate = new Date(comment.created_at); + const timeToCompletion = + mergedDate.getTime() - commentCreationDate.getTime(); + for (const reviewer of reviewersTagged) { + if (reviewer in timeToCompletionPerReviewer) { + timeToCompletionPerReviewer[reviewer] = Math.max( + timeToCompletion, + timeToCompletionPerReviewer[reviewer] + ); + } else { + timeToCompletionPerReviewer[reviewer] = timeToCompletion; + } + } + } + + return timeToCompletionPerReviewer; +} + +async function extractPrStats( + pull: any, + pullsFromLastYear: any[] +): Promise { + const githubClient = github.getGitHubClient(); + const creationDate = new Date(pull.created_at); + const mergedDate = new Date(pull.merged_at); + const reviews = ( + await githubClient.rest.pulls.listReviews({ + owner: REPO_OWNER, + repo: REPO, + pull_number: pull.number, + }) + ).data; + // GitHub has a concept of review comments (must be part of a review) and issue comments on a repo, so we need to look at both + const comments = ( + await githubClient.rest.issues.listComments({ + owner: REPO_OWNER, + repo: REPO, + issue_number: pull.number, + }) + ).data; + const reviewComments = ( + await githubClient.rest.pulls.listReviewComments({ + owner: REPO_OWNER, + repo: REPO, + pull_number: pull.number, + }) + ).data; + const prStats: PrStats = { + firstTimeContribution: checkIfFirstTimeContributor(pull, pullsFromLastYear), + timeToFirstReview: getTimeToFirstReview( + pull, + comments, + reviews, + creationDate, + mergedDate + ), + timeFromCreationToCompletion: mergedDate.getTime() - creationDate.getTime(), + timeFromReviewersMentionedToCompletion: + getTimeFromReviewerMentionedToCompletion( + pull, + comments, + reviewComments, + mergedDate + ), + mergedDate: mergedDate, + }; + + return prStats; +} + +function getMetricBucketStartDate(pullStat: PrStats, bucketEnd: Date): number { + const bucketStart = bucketEnd; + while (bucketStart.getTime() > pullStat.mergedDate.getTime()) { + bucketStart.setDate(bucketStart.getDate() - 7); + } + + return bucketStart.getTime(); +} + +function distinctReviewers(pullStats: PrStats[]): string[] { + let users: Set = new Set(); + for (const pullStat of pullStats) { + for (const user of Object.keys( + pullStat.timeFromReviewersMentionedToCompletion + )) { + users.add(user); + } + } + return Array.from(users); +} + +async function committersFromReviewers(users: string[]): Promise { + let committers: string[] = []; + for (const user of users) { + if (await github.checkIfCommitter(user)) { + committers.push(user); + } + } + return committers; +} + +function averageTimeFromCommitterAssignmentToPrMerge( + pullStats: PrStats[], + committers: string[] +): number { + if (committers.length === 0) { + return 0; + } + let numCommitterReviews = 0; + let totalTimeFromAssignToMerge = 0; + for (const pullStat of pullStats) { + for (const reviewer of Object.keys( + pullStat.timeFromReviewersMentionedToCompletion + )) { + if (committers.indexOf(reviewer) > -1) { + numCommitterReviews++; + totalTimeFromAssignToMerge += + pullStat.timeFromReviewersMentionedToCompletion[reviewer]; + } + } + } + if (numCommitterReviews === 0) { + return 0; + } + + return totalTimeFromAssignToMerge / numCommitterReviews; +} + +// Calculates a gini index of inequality for reviews. +// 0 is perfectly equally distributed, 1 is inequally distributed (with 1 person having all reviews) +function getGiniIndexForCommitterReviews( + pullStats: PrStats[], + committers: string[] +) { + let reviewsPerCommitter: { [key: string]: number } = {}; + for (const pullStat of pullStats) { + for (const reviewer of Object.keys( + pullStat.timeFromReviewersMentionedToCompletion + )) { + if (committers.indexOf(reviewer) > -1) { + if (reviewer in reviewsPerCommitter) { + reviewsPerCommitter[reviewer]++; + } else { + reviewsPerCommitter[reviewer] = 1; + } + } + } + } + let reviewCounts = Object.values(reviewsPerCommitter); + reviewCounts.sort(); + let giniNumerator = 0; + let giniDenominator = 0; + const n = reviewCounts.length; + for (let i = 1; i <= reviewCounts.length; i++) { + let yi = reviewCounts[i - 1]; + giniNumerator += (n + 1 - i) * yi; + giniDenominator += yi; + } + return (1 / n) * (n + 1 - (2 * giniNumerator) / giniDenominator); +} + +async function aggregateStatsForBucket( + pullStats: PrStats[] +): Promise { + const reviewers = distinctReviewers(pullStats); + const committers = await committersFromReviewers(reviewers); + const firstTimePrs = pullStats.filter( + (pullStat) => pullStat.firstTimeContribution + ); + let averageTimeToNewContributorFirstReview = 0; + if (firstTimePrs.length > 0) { + averageTimeToNewContributorFirstReview = + firstTimePrs.reduce( + (sumTime, prStat) => sumTime + prStat.timeToFirstReview, + 0 + ) / firstTimePrs.length; + } + return { + prsCompleted: pullStats.length, + prsCompletedByNewContributors: firstTimePrs.length, + averageTimeToFirstReview: + pullStats.reduce( + (sumTime, prStat) => sumTime + prStat.timeToFirstReview, + 0 + ) / pullStats.length, + averageTimeToNewContributorFirstReview: + averageTimeToNewContributorFirstReview, + averageTimeCreationToCompletion: + pullStats.reduce( + (sumTime, prStat) => sumTime + prStat.timeFromCreationToCompletion, + 0 + ) / pullStats.length, + numUsersPerformingReviews: reviewers.length, + numCommittersPerformingReviews: committers.length, + numNonCommittersPerformingReviews: reviewers.length - committers.length, + giniIndexCommittersPerformingReviews: getGiniIndexForCommitterReviews( + pullStats, + committers + ), + averageTimeFromCommitterAssignmentToPrMerge: + averageTimeFromCommitterAssignmentToPrMerge(pullStats, committers), + }; +} + +function convertMsToRoundedMinutes(milliseconds: number): number { + return Math.floor(milliseconds / 60_000); +} + +async function reportMetrics(statBuckets: { [key: number]: PrStats[] }) { + console.log("---------------------------------"); + console.log("PR Stats"); + console.log("---------------------------------"); + let csvOutput = ""; + csvOutput += + "Bucket start (bucketed by merge time),PRs Completed,PRs completed by first time contributors,Average time in minutes to first review,Average time in minutes to first review for new contributors,Average time in minutes from PR creation to completion,Total number of reviewers,Total number of committers performing reviews,Total number of non-committers performing reviews,Gini index (fairness) of committers performing reviews,Average time in minutes from committer assignment to PR merge"; + const startDates = Object.keys(statBuckets); + for (let i = 0; i < startDates.length; i++) { + let startDate = startDates[i]; + let aggregatedStats = await aggregateStatsForBucket(statBuckets[startDate]); + console.log(); + + const bucketStart = new Date(parseInt(startDate)); + console.log("Bucket start:", bucketStart); + csvOutput += `\n${bucketStart.toDateString()}`; + + console.log("PRs completed:", aggregatedStats.prsCompleted); + csvOutput += `,${aggregatedStats.prsCompleted}`; + + console.log( + "PRs completed by first time contributors:", + aggregatedStats.prsCompletedByNewContributors + ); + csvOutput += `,${aggregatedStats.prsCompletedByNewContributors}`; + + console.log( + "Average time in minutes to first review:", + convertMsToRoundedMinutes(aggregatedStats.averageTimeToFirstReview) + ); + csvOutput += `,${convertMsToRoundedMinutes( + aggregatedStats.averageTimeToFirstReview + )}`; + + console.log( + "Average time in minutes to first review for new contributors:", + convertMsToRoundedMinutes( + aggregatedStats.averageTimeToNewContributorFirstReview + ) + ); + csvOutput += `,${convertMsToRoundedMinutes( + aggregatedStats.averageTimeToNewContributorFirstReview + )}`; + + console.log( + "Average time in minutes from PR creation to completion:", + convertMsToRoundedMinutes(aggregatedStats.averageTimeCreationToCompletion) + ); + csvOutput += `,${convertMsToRoundedMinutes( + aggregatedStats.averageTimeCreationToCompletion + )}`; + + console.log( + "Total number of reviewers:", + aggregatedStats.numUsersPerformingReviews + ); + csvOutput += `,${aggregatedStats.numUsersPerformingReviews}`; + + console.log( + "Total number of committers performing reviews:", + aggregatedStats.numCommittersPerformingReviews + ); + csvOutput += `,${aggregatedStats.numCommittersPerformingReviews}`; + + console.log( + "Total number of non-committers performing reviews:", + aggregatedStats.numNonCommittersPerformingReviews + ); + csvOutput += `,${aggregatedStats.numNonCommittersPerformingReviews}`; + + console.log( + "Gini index (fairness) of committers performing reviews:", + aggregatedStats.giniIndexCommittersPerformingReviews + ); + csvOutput += `,${aggregatedStats.giniIndexCommittersPerformingReviews}`; + + console.log( + "Average time in minutes from committer assignment to PR merge:", + convertMsToRoundedMinutes( + aggregatedStats.averageTimeFromCommitterAssignmentToPrMerge + ) + ); + csvOutput += `,${convertMsToRoundedMinutes( + aggregatedStats.averageTimeFromCommitterAssignmentToPrMerge + )}`; + } + fs.writeFileSync(PATH_TO_METRICS_CSV, csvOutput); + console.log(`Output written to ${PATH_TO_METRICS_CSV}`); +} + +async function gatherMetrics() { + // We will only aggregate metrics from the last 90 days, + // but will look further back to determine if this is a user's first contribution + const pullsFromLastYear = await getCompletedPullsFromLastYear(); + let cutoffDate = new Date(); + cutoffDate.setDate(cutoffDate.getDate() - 90); + + let pullStats: PrStats[] = []; + console.log("Extracting stats from pulls - this may take a while"); + for (let i = 0; i < pullsFromLastYear.length; i++) { + let pull = pullsFromLastYear[i]; + if (new Date(pull.created_at) > cutoffDate && pull.merged_at) { + pullStats.push(await extractPrStats(pull, pullsFromLastYear)); + } + if (i % 10 === 0) { + process.stdout.write("."); + } + } + + console.log("\nDone extracting stats, formatting results"); + + let statBuckets: { [key: number]: PrStats[] } = {}; + let bucketEnd = new Date(); + bucketEnd.setUTCHours(23, 59, 59, 999); + pullStats.forEach((pullStat) => { + let bucketStart = getMetricBucketStartDate(pullStat, bucketEnd); + if (bucketStart in statBuckets) { + statBuckets[bucketStart].push(pullStat); + } else { + statBuckets[bucketStart] = [pullStat]; + } + }); + await reportMetrics(statBuckets); +} + +gatherMetrics(); + +export {}; From f4952658ae473c3ef346b6db43a7502a33638bce Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Thu, 3 Mar 2022 16:57:26 -0500 Subject: [PATCH 47/68] [BEAM-11085] Test that windows are correctly observed in DoFns --- .../beam/core/runtime/exec/fullvalue_test.go | 20 ++++++ .../pkg/beam/core/runtime/exec/pardo_test.go | 69 +++++++++++++++++++ 2 files changed, 89 insertions(+) diff --git a/sdks/go/pkg/beam/core/runtime/exec/fullvalue_test.go b/sdks/go/pkg/beam/core/runtime/exec/fullvalue_test.go index f24348530b54..a3216fc194e1 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/fullvalue_test.go +++ b/sdks/go/pkg/beam/core/runtime/exec/fullvalue_test.go @@ -46,6 +46,26 @@ func makeValues(vs ...interface{}) []FullValue { return ret } +func makeWindowedInput(ws []typex.Window, vs ...interface{}) []MainInput { + var ret []MainInput + for _, v := range makeWindowedValues(ws, vs...) { + ret = append(ret, MainInput{Key: v}) + } + return ret +} + +func makeWindowedValues(ws []typex.Window, vs ...interface{}) []FullValue { + var ret []FullValue + for _, v := range vs { + ret = append(ret, FullValue{ + Windows: ws, + Timestamp: mtime.ZeroTimestamp, + Elm: v, + }) + } + return ret +} + func makeValuesNoWindowOrTime(vs ...interface{}) []FullValue { var ret []FullValue for _, v := range vs { diff --git a/sdks/go/pkg/beam/core/runtime/exec/pardo_test.go b/sdks/go/pkg/beam/core/runtime/exec/pardo_test.go index f131f6f8ee70..f277aefd5113 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/pardo_test.go +++ b/sdks/go/pkg/beam/core/runtime/exec/pardo_test.go @@ -98,6 +98,75 @@ func TestParDo(t *testing.T) { } } +func windowObserverFn(w typex.Window, word string) string { + if _, ok := w.(window.GlobalWindow); ok { + return fmt.Sprintf("%v-%v", word, "global") + } + if iw, ok := w.(window.IntervalWindow); ok { + return fmt.Sprintf("%v-%v-%v", word, iw.Start, iw.End) + } + return fmt.Sprintf("Couldnt cast %v to a window type", w) +} + +// TestParDo verifies that the ParDo node works correctly for side inputs and emitters. It uses a special +// dofn that uses all forms of sideinput. +func TestParDo_WindowObservation(t *testing.T) { + tests := []struct { + ws []typex.Window + out1 string + out2 string + }{ + { + ws: []typex.Window{window.GlobalWindow{}}, + out1: "testInput1-global", + out2: "testInput2-global", + }, + { + ws: []typex.Window{window.IntervalWindow{ + Start: 100, + End: 200, + }}, + out1: "testInput1-100-200", + out2: "testInput2-100-200", + }, + } + for _, test := range tests { + fn, err := graph.NewDoFn(windowObserverFn) + if err != nil { + t.Fatalf("invalid function: %v", err) + } + + g := graph.New() + nN := g.NewNode(typex.New(reflectx.String), window.DefaultWindowingStrategy(), true) + + edge, err := graph.NewParDo(g, g.Root(), fn, []*graph.Node{nN}, nil, nil) + if err != nil { + t.Fatalf("invalid pardo: %v", err) + } + + out := &CaptureNode{UID: 1} + pardo := &ParDo{UID: 3, Fn: edge.DoFn, Inbound: edge.Input, Out: []Node{out}} + n := &FixedRoot{UID: 4, Elements: makeWindowedInput(test.ws, "testInput1", "testInput2"), Out: pardo} + + p, err := NewPlan("a", []Unit{n, pardo, out}) + if err != nil { + t.Fatalf("failed to construct plan: %v", err) + } + + if err := p.Execute(context.Background(), "1", DataContext{}); err != nil { + t.Fatalf("execute failed: %v", err) + } + if err := p.Down(context.Background()); err != nil { + t.Fatalf("down failed: %v", err) + } + + expected := makeWindowedValues(test.ws, test.out1, test.out2) + if !equalList(out.Elements, expected) { + t.Errorf("pardo(windowObserverFn) = %v, want %v", extractValues(out.Elements...), extractValues(expected...)) + } + } +} + func emitSumFn(n int, emit func(int)) { emit(n + 1) } From f593b52eb99c54a2338122f439e1e326ce1a5ad7 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Mon, 7 Mar 2022 13:30:05 -0500 Subject: [PATCH 48/68] Give pr bot write permissions on pr update --- .github/workflows/pr-bot-pr-updates.yml | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/.github/workflows/pr-bot-pr-updates.yml b/.github/workflows/pr-bot-pr-updates.yml index e6f8610ef0c6..dd6d68ce6fc3 100644 --- a/.github/workflows/pr-bot-pr-updates.yml +++ b/.github/workflows/pr-bot-pr-updates.yml @@ -24,12 +24,23 @@ on: jobs: process-pr-update: + # Give GITHUB_TOKEN permissions to write pull request comments and to the state branch, and read PR related info + permissions: + contents: write + pull-requests: write + checks: read + issues: read + statuses: read + # Don't run on forks if: github.repository == 'apache/beam' runs-on: ubuntu-latest steps: + # Pin to master so users can't do anything malicious on their own branch and run it here. - uses: actions/checkout@v2 + with: + ref: 'master' - run: npm install working-directory: 'scripts/ci/pr-bot' From 7d49f8a69d58174814853b86538f3f5b302fcd96 Mon Sep 17 00:00:00 2001 From: laraschmidt Date: Mon, 7 Mar 2022 15:02:52 -0800 Subject: [PATCH 49/68] Adding a logical type for Schemas using proto serialization. (#16940) --- .../src/main/proto/beam_runner_api.proto | 4 ++ .../beam/sdk/schemas/SchemaTranslation.java | 10 +++- .../logicaltypes/SchemaLogicalType.java | 60 +++++++++++++++++++ .../sdk/schemas/SchemaTranslationTest.java | 2 + .../logicaltypes/LogicalTypesTest.java | 17 ++++++ 5 files changed, 90 insertions(+), 3 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/SchemaLogicalType.java diff --git a/model/pipeline/src/main/proto/beam_runner_api.proto b/model/pipeline/src/main/proto/beam_runner_api.proto index 105bdbda521a..7fdb5aaf5e86 100644 --- a/model/pipeline/src/main/proto/beam_runner_api.proto +++ b/model/pipeline/src/main/proto/beam_runner_api.proto @@ -1055,6 +1055,10 @@ message StandardCoders { // - A timestamp without a timezone where seconds + micros represents the // amount of time since the epoch. // + // beam:logical_type:schema:v1 + // - Representation type: BYTES + // - A Beam Schema stored as a serialized proto. + // // The payload for RowCoder is an instance of Schema. // Components: None // Experimental. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java index 25983caf2b15..e1112b2472de 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java @@ -43,6 +43,7 @@ import org.apache.beam.sdk.schemas.Schema.LogicalType; import org.apache.beam.sdk.schemas.Schema.TypeName; import org.apache.beam.sdk.schemas.logicaltypes.MicrosInstant; +import org.apache.beam.sdk.schemas.logicaltypes.SchemaLogicalType; import org.apache.beam.sdk.schemas.logicaltypes.UnknownLogicalType; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.Row; @@ -67,10 +68,13 @@ public class SchemaTranslation { private static final String URN_BEAM_LOGICAL_JAVASDK = "beam:logical_type:javasdk:v1"; // TODO(BEAM-7855): Populate this with a LogicalTypeRegistrar, which includes a way to construct - // the LogicalType - // given an argument. + // the LogicalType given an argument. private static final ImmutableMap>> - STANDARD_LOGICAL_TYPES = ImmutableMap.of(MicrosInstant.IDENTIFIER, MicrosInstant.class); + STANDARD_LOGICAL_TYPES = + ImmutableMap.>>builder() + .put(MicrosInstant.IDENTIFIER, MicrosInstant.class) + .put(SchemaLogicalType.IDENTIFIER, SchemaLogicalType.class) + .build(); public static SchemaApi.Schema schemaToProto(Schema schema, boolean serializeLogicalType) { String uuid = schema.getUUID() != null ? schema.getUUID().toString() : ""; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/SchemaLogicalType.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/SchemaLogicalType.java new file mode 100644 index 000000000000..a7fcbdc50f93 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/SchemaLogicalType.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.schemas.logicaltypes; + +import org.apache.beam.model.pipeline.v1.SchemaApi; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.Schema.FieldType; +import org.apache.beam.sdk.schemas.SchemaTranslation; +import org.apache.beam.vendor.grpc.v1p43p2.com.google.protobuf.InvalidProtocolBufferException; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** A schema represented as a serialized proto bytes. */ +public class SchemaLogicalType implements Schema.LogicalType { + public static final String IDENTIFIER = "beam:logical_type:schema:v1"; + + @Override + public String getIdentifier() { + return IDENTIFIER; + } + + @Override + public @Nullable FieldType getArgumentType() { + return null; + } + + @Override + public FieldType getBaseType() { + return FieldType.BYTES; + } + + @Override + public byte @NonNull [] toBaseType(Schema input) { + return SchemaTranslation.schemaToProto(input, true).toByteArray(); + } + + @Override + public org.apache.beam.sdk.schemas.Schema toInputType(byte @NonNull [] base) { + try { + return SchemaTranslation.schemaFromProto(SchemaApi.Schema.parseFrom(base)); + } catch (InvalidProtocolBufferException e) { + throw new RuntimeException(e); + } + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java index 2c0cadb45eaa..f4274de02ea5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java @@ -41,6 +41,7 @@ import org.apache.beam.sdk.schemas.logicaltypes.DateTime; import org.apache.beam.sdk.schemas.logicaltypes.FixedBytes; import org.apache.beam.sdk.schemas.logicaltypes.MicrosInstant; +import org.apache.beam.sdk.schemas.logicaltypes.SchemaLogicalType; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.grpc.v1p43p2.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Charsets; @@ -180,6 +181,7 @@ public static Iterable data() { .add(Schema.of(Field.of("logical_argument", FieldType.logicalType(new DateTime())))) .add( Schema.of(Field.of("single_arg_argument", FieldType.logicalType(FixedBytes.of(100))))) + .add(Schema.of(Field.of("schema", FieldType.logicalType(new SchemaLogicalType())))) .build(); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/logicaltypes/LogicalTypesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/logicaltypes/LogicalTypesTest.java index dd07f8eef8fa..8ee20f1ad167 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/logicaltypes/LogicalTypesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/logicaltypes/LogicalTypesTest.java @@ -113,4 +113,21 @@ public void testUuid() { assertEquals(uuid, row.getLogicalTypeValue(0, UUID.class)); assertEquals(uuidAsRow, row.getBaseValue(0, Row.class)); } + + @Test + public void testSchema() { + Schema schemaValue = + Schema.of( + Field.of("fieldOne", FieldType.BOOLEAN), + Field.of("nested", FieldType.logicalType(new SchemaLogicalType()))); + + Schema schema = Schema.builder().addLogicalTypeField("schema", new SchemaLogicalType()).build(); + Row row = Row.withSchema(schema).addValues(schemaValue).build(); + assertEquals(schemaValue, row.getLogicalTypeValue(0, Schema.class)); + + // Check base type conversion. + assertEquals( + schemaValue, + new SchemaLogicalType().toInputType(new SchemaLogicalType().toBaseType(schemaValue))); + } } From d8cea6391ef0e33b9ba67a735e63c3f5eb2d69b4 Mon Sep 17 00:00:00 2001 From: Marcin Kuthan Date: Tue, 8 Mar 2022 01:04:27 +0100 Subject: [PATCH 50/68] BEAM-13765 missing PAssert methods (#16668) --- .../org/apache/beam/sdk/testing/PAssert.java | 47 +++++++++++++++++++ 1 file changed, 47 insertions(+) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java index 4a73361dfc4f..99d41ad84242 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java @@ -233,6 +233,19 @@ public interface IterableAssert { */ IterableAssert inWindow(BoundedWindow window); + /** + * Creates a new {@link IterableAssert} like this one, but with the assertion restricted to only + * run on the provided window. + * + *

The assertion will expect outputs to be produced to the provided window exactly once. If + * the upstream {@link Trigger} may produce output multiple times, consider instead using {@link + * #inFinalPane(BoundedWindow)} or {@link #inOnTimePane(BoundedWindow)}. + * + * @return a new {@link IterableAssert} like this one but with the assertion only applied to the + * specified window. + */ + IterableAssert inOnlyPane(BoundedWindow window); + /** * Creates a new {@link IterableAssert} like this one, but with the assertion restricted to only * run on the provided window, running the checker only on the final pane for each key. @@ -340,6 +353,20 @@ public interface IterableAssert { /** Builder interface for assertions applicable to a single value. */ public interface SingletonAssert { + /** + * Creates a new {@link SingletonAssert} like this one, but with the assertion restricted to + * only run on the provided window. + * + *

The assertion will concatenate all panes present in the provided window if the {@link + * Trigger} produces multiple panes. If the windowing strategy accumulates fired panes and + * triggers fire multple times, consider using instead {@link #inFinalPane(BoundedWindow)} or + * {@link #inOnTimePane(BoundedWindow)}. + * + * @return a new {@link SingletonAssert} like this one but with the assertion only applied to + * the specified window. + */ + SingletonAssert inWindow(BoundedWindow window); + /** * Creates a new {@link SingletonAssert} like this one, but with the assertion restricted to * only run on the provided window. @@ -631,6 +658,11 @@ public PCollectionContentsAssert inWindow(BoundedWindow window) { return withPane(window, PaneExtractors.allPanes()); } + @Override + public PCollectionContentsAssert inOnlyPane(BoundedWindow window) { + return withPane(window, PaneExtractors.onlyPane(site)); + } + @Override public PCollectionContentsAssert inFinalPane(BoundedWindow window) { return withPane(window, PaneExtractors.finalPane()); @@ -822,6 +854,11 @@ public PCollectionSingletonIterableAssert inWindow(BoundedWindow window) { return withPanes(window, PaneExtractors.allPanes()); } + @Override + public PCollectionSingletonIterableAssert inOnlyPane(BoundedWindow window) { + return withPanes(window, PaneExtractors.onlyPane(site)); + } + @Override public PCollectionSingletonIterableAssert inFinalPane(BoundedWindow window) { return withPanes(window, PaneExtractors.finalPane()); @@ -943,6 +980,11 @@ private static class PCollectionSingletonAssert implements SingletonAssert this.site = site; } + @Override + public PCollectionSingletonAssert inWindow(BoundedWindow window) { + return withPanes(window, PaneExtractors.allPanes()); + } + @Override public PCollectionSingletonAssert inFinalPane(BoundedWindow window) { return withPanes(window, PaneExtractors.finalPane()); @@ -1071,6 +1113,11 @@ private PCollectionViewAssert( this.site = site; } + @Override + public PCollectionViewAssert inWindow(BoundedWindow window) { + return inPane(window, PaneExtractors.allPanes()); + } + @Override public PCollectionViewAssert inOnlyPane(BoundedWindow window) { return inPane(window, PaneExtractors.onlyPane(site)); From 83f05a9fbea32c3327c8fb342e11a1b17592d08f Mon Sep 17 00:00:00 2001 From: Ritesh Ghorse Date: Mon, 7 Mar 2022 22:23:35 -0500 Subject: [PATCH 51/68] [BEAM-13909] improve coverage of Provision package (#17014) * improve coverage of provision package * updated comments --- sdks/go/pkg/beam/provision/provision_test.go | 56 ++++++++++++++++++++ 1 file changed, 56 insertions(+) diff --git a/sdks/go/pkg/beam/provision/provision_test.go b/sdks/go/pkg/beam/provision/provision_test.go index f29bc9b3be51..92dc315fc062 100644 --- a/sdks/go/pkg/beam/provision/provision_test.go +++ b/sdks/go/pkg/beam/provision/provision_test.go @@ -16,8 +16,16 @@ package provision import ( + "context" + "fmt" + "log" + "net" "reflect" + "sync" "testing" + + fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1" + "google.golang.org/grpc" ) type s struct { @@ -52,3 +60,51 @@ func TestConversions(t *testing.T) { } } } + +type ProvisionServiceServicer struct { + fnpb.UnimplementedProvisionServiceServer +} + +func (p ProvisionServiceServicer) GetProvisionInfo(ctx context.Context, req *fnpb.GetProvisionInfoRequest) (*fnpb.GetProvisionInfoResponse, error) { + return &fnpb.GetProvisionInfoResponse{Info: &fnpb.ProvisionInfo{RetrievalToken: "token"}}, nil +} + +func setup(addr *string, wg *sync.WaitGroup) { + l, err := net.Listen("tcp", ":0") + defer l.Close() + if err != nil { + log.Fatalf("failed to find an open port: %v", err) + } + port := l.Addr().(*net.TCPAddr).Port + *addr = fmt.Sprintf(":%d", port) + + server := grpc.NewServer() + defer server.Stop() + + prs := &ProvisionServiceServicer{} + fnpb.RegisterProvisionServiceServer(server, prs) + + wg.Done() + + if err := server.Serve(l); err != nil { + log.Fatalf("cannot serve the server: %v", err) + } +} + +func TestProvisionInfo(t *testing.T) { + + endpoint := "" + var wg sync.WaitGroup + wg.Add(1) + go setup(&endpoint, &wg) + wg.Wait() + + got, err := Info(context.Background(), endpoint) + if err != nil { + t.Errorf("error in response: %v", err) + } + want := &fnpb.ProvisionInfo{RetrievalToken: "token"} + if got.GetRetrievalToken() != want.GetRetrievalToken() { + t.Errorf("provision.Info() = %v, want %v", got, want) + } +} From a995627ffbe3d770b501a5cca79178b5ff076b03 Mon Sep 17 00:00:00 2001 From: Jack McCluskey Date: Fri, 4 Mar 2022 18:23:18 +0000 Subject: [PATCH 52/68] [BEAM-14050] Update taxi.go example instructions --- sdks/go/examples/kafka/taxi.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/go/examples/kafka/taxi.go b/sdks/go/examples/kafka/taxi.go index aeb539be5c0b..99b6e46637c0 100644 --- a/sdks/go/examples/kafka/taxi.go +++ b/sdks/go/examples/kafka/taxi.go @@ -68,7 +68,8 @@ // export JOB_NAME="kafka-taxi-`date +%Y%m%d-%H%M%S`" // export BOOTSTRAP_SERVERS="123.45.67.89:1234" // export EXPANSION_ADDR="localhost:1234" -// go run ./sdks/go/examples/kafka/taxi.go \ +// cd ./sdks/go +// go run ./examples/kafka/taxi.go \ // --runner=DataflowRunner \ // --temp_location=$TEMP_LOCATION \ // --staging_location=$STAGING_LOCATION \ From 493a0df145a2ad2ce317914e4ba7f26f5940be02 Mon Sep 17 00:00:00 2001 From: ansh0l Date: Tue, 8 Mar 2022 19:30:16 +0530 Subject: [PATCH 53/68] Merge pull request #17027: [BEAM-11205] Upgrade GCP Libraries BOM dependencies to 24.4.0 --- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- sdks/java/container/license_scripts/dep_urls_java.yaml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 08b3dca52a3d..edfcba7e0acf 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -577,7 +577,7 @@ class BeamModulePlugin implements Plugin { // The GCP Libraries BOM dashboard shows the versions set by the BOM: // https://storage.googleapis.com/cloud-opensource-java-dashboard/com.google.cloud/libraries-bom/24.3.0/artifact_details.html // Update libraries-bom version on sdks/java/container/license_scripts/dep_urls_java.yaml - google_cloud_platform_libraries_bom : "com.google.cloud:libraries-bom:24.3.0", + google_cloud_platform_libraries_bom : "com.google.cloud:libraries-bom:24.4.0", google_cloud_spanner : "com.google.cloud:google-cloud-spanner", // google_cloud_platform_libraries_bom sets version google_cloud_spanner_test : "com.google.cloud:google-cloud-spanner:$google_cloud_spanner_version:tests", google_code_gson : "com.google.code.gson:gson:$google_code_gson_version", diff --git a/sdks/java/container/license_scripts/dep_urls_java.yaml b/sdks/java/container/license_scripts/dep_urls_java.yaml index 57745777c26f..256e78d8f67c 100644 --- a/sdks/java/container/license_scripts/dep_urls_java.yaml +++ b/sdks/java/container/license_scripts/dep_urls_java.yaml @@ -42,7 +42,7 @@ jaxen: '1.1.6': type: "3-Clause BSD" libraries-bom: - '24.3.0': + '24.4.0': license: "https://raw.githubusercontent.com/GoogleCloudPlatform/cloud-opensource-java/master/LICENSE" type: "Apache License 2.0" paranamer: From 22f3603c26f0729d938443ab78dcb043a33bfc39 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Tue, 8 Mar 2022 12:36:25 -0500 Subject: [PATCH 54/68] [BEAM-13709] Inconsistent behavior when parsing boolean flags across different APIs in Python SDK (#16929) --- .../apache_beam/options/pipeline_options.py | 29 ++++++++++- .../options/pipeline_options_test.py | 50 +++++++++++++++++++ 2 files changed, 78 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index e158c98ac800..14aa54ec735c 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -55,6 +55,11 @@ _LOGGER = logging.getLogger(__name__) +# Map defined with option names to flag names for boolean options +# that have a destination(dest) in parser.add_argument() different +# from the flag name and whose default value is `None`. +_FLAG_THAT_SETS_FALSE_VALUE = {'use_public_ips': 'no_use_public_ips'} + def _static_value_provider_of(value_type): """"Helper function to plug a ValueProvider into argparse. @@ -180,7 +185,15 @@ def __init__(self, flags=None, **kwargs): flags: An iterable of command line arguments to be used. If not specified then sys.argv will be used as input for parsing arguments. - **kwargs: Add overrides for arguments passed in flags. + **kwargs: Add overrides for arguments passed in flags. For overrides + of arguments, please pass the `option names` instead of + flag names. + Option names: These are defined as dest in the + parser.add_argument() for each flag. Passing flags + like {no_use_public_ips: True}, for which the dest is + defined to a different flag name in the parser, + would be discarded. Instead, pass the dest of + the flag (dest of no_use_public_ips is use_public_ips). """ # Initializing logging configuration in case the user did not set it up. logging.basicConfig() @@ -237,9 +250,22 @@ def from_dictionary(cls, options): """ flags = [] for k, v in options.items(): + # Note: If a boolean flag is True in the dictionary, + # implicitly the method assumes the boolean flag is + # specified as a command line argument. If the + # boolean flag is False, this method simply discards them. + # Eg: {no_auth: True} is similar to python your_file.py --no_auth + # {no_auth: False} is similar to python your_file.py. if isinstance(v, bool): if v: flags.append('--%s' % k) + elif k in _FLAG_THAT_SETS_FALSE_VALUE: + # Capture overriding flags, which have a different dest + # from the flag name defined in the parser.add_argument + # Eg: no_use_public_ips, which has the dest=use_public_ips + # different from flag name + flag_that_disables_the_option = (_FLAG_THAT_SETS_FALSE_VALUE[k]) + flags.append('--%s' % flag_that_disables_the_option) elif isinstance(v, list): for i in v: flags.append('--%s=%s' % (k, i)) @@ -353,6 +379,7 @@ def view_as(self, cls): """ view = cls(self._flags) + for option_name in view._visible_option_list(): # Initialize values of keys defined by a cls. # diff --git a/sdks/python/apache_beam/options/pipeline_options_test.py b/sdks/python/apache_beam/options/pipeline_options_test.py index 9392055799c5..3195f3594fe0 100644 --- a/sdks/python/apache_beam/options/pipeline_options_test.py +++ b/sdks/python/apache_beam/options/pipeline_options_test.py @@ -31,11 +31,14 @@ from apache_beam.options.pipeline_options import ProfilingOptions from apache_beam.options.pipeline_options import TypeOptions from apache_beam.options.pipeline_options import WorkerOptions +from apache_beam.options.pipeline_options import _BeamArgumentParser from apache_beam.options.value_provider import RuntimeValueProvider from apache_beam.options.value_provider import StaticValueProvider from apache_beam.transforms.display import DisplayData from apache_beam.transforms.display_test import DisplayDataItemMatcher +_LOGGER = logging.getLogger(__name__) + class PipelineOptionsTest(unittest.TestCase): def setUp(self): @@ -647,6 +650,53 @@ def test_dataflow_service_options(self): self.assertEqual( options.get_all_options()['dataflow_service_options'], None) + def test_options_store_false_with_different_dest(self): + parser = _BeamArgumentParser() + for cls in PipelineOptions.__subclasses__(): + cls._add_argparse_args(parser) + + actions = parser._actions.copy() + options_to_flags = {} + options_diff_dest_store_true = {} + + for i in range(len(actions)): + flag_names = actions[i].option_strings + option_name = actions[i].dest + + if isinstance(actions[i].const, bool): + for flag_name in flag_names: + flag_name = flag_name.strip('-') + if flag_name != option_name: + # Capture flags which has store_action=True and has a + # different dest. This behavior would be confusing. + if actions[i].const: + options_diff_dest_store_true[flag_name] = option_name + continue + # check the flags like no_use_public_ips + # default is None, action is {True, False} + if actions[i].default is None: + options_to_flags[option_name] = flag_name + + self.assertEqual( + len(options_diff_dest_store_true), + 0, + _LOGGER.error( + "There should be no flags that have a dest " + "different from flag name and action as " + "store_true. It would be confusing " + "to the user. Please specify the dest as the " + "flag_name instead.")) + from apache_beam.options.pipeline_options import ( + _FLAG_THAT_SETS_FALSE_VALUE) + + self.assertDictEqual( + _FLAG_THAT_SETS_FALSE_VALUE, + options_to_flags, + "If you are adding a new boolean flag with default=None," + " with different dest/option_name from the flag name, please add " + "the dest and the flag name to the map " + "_FLAG_THAT_SETS_FALSE_VALUE in PipelineOptions.py") + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) From a6bb7ae98dcc95d0ab29598765479505212c7f25 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 7 Mar 2022 16:54:57 -0800 Subject: [PATCH 55/68] Update dataflow API client. --- .../clients/dataflow/dataflow_v1b3_client.py | 282 +------- .../dataflow/dataflow_v1b3_messages.py | 667 ++++++------------ 2 files changed, 234 insertions(+), 715 deletions(-) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_client.py b/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_client.py index 985934dd3653..07415805ce3c 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_client.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_client.py @@ -1,26 +1,10 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - """Generated client library for dataflow version v1b3.""" # NOTE: This file is autogenerated and should not be edited by hand. -from apitools.base.py import base_api +from __future__ import absolute_import -from apache_beam.runners.dataflow.internal.clients.dataflow import dataflow_v1b3_messages as messages +from apitools.base.py import base_api +from . import dataflow_v1b3_messages as messages class DataflowV1b3(base_api.BaseApiClient): @@ -47,7 +31,7 @@ def __init__(self, url='', credentials=None, additional_http_headers=None, response_encoding=None): """Create a new dataflow handle.""" url = url or self.BASE_URL - super().__init__( + super(DataflowV1b3, self).__init__( url, credentials=credentials, get_credentials=get_credentials, http=http, model=model, log_request=log_request, log_response=log_response, @@ -55,8 +39,6 @@ def __init__(self, url='', credentials=None, default_global_params=default_global_params, additional_http_headers=additional_http_headers, response_encoding=response_encoding) - self.projects_catalogTemplates_templateVersions = self.ProjectsCatalogTemplatesTemplateVersionsService(self) - self.projects_catalogTemplates = self.ProjectsCatalogTemplatesService(self) self.projects_jobs_debug = self.ProjectsJobsDebugService(self) self.projects_jobs_messages = self.ProjectsJobsMessagesService(self) self.projects_jobs_workItems = self.ProjectsJobsWorkItemsService(self) @@ -73,199 +55,16 @@ def __init__(self, url='', credentials=None, self.projects_locations_templates = self.ProjectsLocationsTemplatesService(self) self.projects_locations = self.ProjectsLocationsService(self) self.projects_snapshots = self.ProjectsSnapshotsService(self) - self.projects_templateVersions = self.ProjectsTemplateVersionsService(self) self.projects_templates = self.ProjectsTemplatesService(self) self.projects = self.ProjectsService(self) - class ProjectsCatalogTemplatesTemplateVersionsService(base_api.BaseApiService): - """Service class for the projects_catalogTemplates_templateVersions resource.""" - - _NAME = 'projects_catalogTemplates_templateVersions' - - def __init__(self, client): - super().__init__(client) - self._upload_configs = { - } - - def Create(self, request, global_params=None): - r"""Creates a new Template with TemplateVersion. Requires project_id(projects) and template display_name(catalogTemplates). The template display_name is set by the user. - - Args: - request: (DataflowProjectsCatalogTemplatesTemplateVersionsCreateRequest) input message - global_params: (StandardQueryParameters, default: None) global arguments - Returns: - (TemplateVersion) The response message. - """ - config = self.GetMethodConfig('Create') - return self._RunMethod( - config, request, global_params=global_params) - - Create.method_config = lambda: base_api.ApiMethodInfo( - flat_path='v1b3/projects/{projectsId}/catalogTemplates/{catalogTemplatesId}/templateVersions', - http_method='POST', - method_id='dataflow.projects.catalogTemplates.templateVersions.create', - ordered_params=['parent'], - path_params=['parent'], - query_params=[], - relative_path='v1b3/{+parent}/templateVersions', - request_field='createTemplateVersionRequest', - request_type_name='DataflowProjectsCatalogTemplatesTemplateVersionsCreateRequest', - response_type_name='TemplateVersion', - supports_download=False, - ) - - class ProjectsCatalogTemplatesService(base_api.BaseApiService): - """Service class for the projects_catalogTemplates resource.""" - - _NAME = 'projects_catalogTemplates' - - def __init__(self, client): - super().__init__(client) - self._upload_configs = { - } - - def Commit(self, request, global_params=None): - r"""Creates a new TemplateVersion (Important: not new Template) entry in the spanner table. Requires project_id and display_name (template). - - Args: - request: (DataflowProjectsCatalogTemplatesCommitRequest) input message - global_params: (StandardQueryParameters, default: None) global arguments - Returns: - (TemplateVersion) The response message. - """ - config = self.GetMethodConfig('Commit') - return self._RunMethod( - config, request, global_params=global_params) - - Commit.method_config = lambda: base_api.ApiMethodInfo( - flat_path='v1b3/projects/{projectsId}/catalogTemplates/{catalogTemplatesId}:commit', - http_method='POST', - method_id='dataflow.projects.catalogTemplates.commit', - ordered_params=['name'], - path_params=['name'], - query_params=[], - relative_path='v1b3/{+name}:commit', - request_field='commitTemplateVersionRequest', - request_type_name='DataflowProjectsCatalogTemplatesCommitRequest', - response_type_name='TemplateVersion', - supports_download=False, - ) - - def Delete(self, request, global_params=None): - r"""Deletes an existing Template. Do nothing if Template does not exist. - - Args: - request: (DataflowProjectsCatalogTemplatesDeleteRequest) input message - global_params: (StandardQueryParameters, default: None) global arguments - Returns: - (Empty) The response message. - """ - config = self.GetMethodConfig('Delete') - return self._RunMethod( - config, request, global_params=global_params) - - Delete.method_config = lambda: base_api.ApiMethodInfo( - flat_path='v1b3/projects/{projectsId}/catalogTemplates/{catalogTemplatesId}', - http_method='DELETE', - method_id='dataflow.projects.catalogTemplates.delete', - ordered_params=['name'], - path_params=['name'], - query_params=[], - relative_path='v1b3/{+name}', - request_field='', - request_type_name='DataflowProjectsCatalogTemplatesDeleteRequest', - response_type_name='Empty', - supports_download=False, - ) - - def Get(self, request, global_params=None): - r"""Get TemplateVersion using project_id and display_name with an optional version_id field. Get latest (has tag "latest") TemplateVersion if version_id not set. - - Args: - request: (DataflowProjectsCatalogTemplatesGetRequest) input message - global_params: (StandardQueryParameters, default: None) global arguments - Returns: - (TemplateVersion) The response message. - """ - config = self.GetMethodConfig('Get') - return self._RunMethod( - config, request, global_params=global_params) - - Get.method_config = lambda: base_api.ApiMethodInfo( - flat_path='v1b3/projects/{projectsId}/catalogTemplates/{catalogTemplatesId}', - http_method='GET', - method_id='dataflow.projects.catalogTemplates.get', - ordered_params=['name'], - path_params=['name'], - query_params=[], - relative_path='v1b3/{+name}', - request_field='', - request_type_name='DataflowProjectsCatalogTemplatesGetRequest', - response_type_name='TemplateVersion', - supports_download=False, - ) - - def Label(self, request, global_params=None): - r"""Updates the label of the TemplateVersion. Label can be duplicated in Template, so either add or remove the label in the TemplateVersion. - - Args: - request: (DataflowProjectsCatalogTemplatesLabelRequest) input message - global_params: (StandardQueryParameters, default: None) global arguments - Returns: - (ModifyTemplateVersionLabelResponse) The response message. - """ - config = self.GetMethodConfig('Label') - return self._RunMethod( - config, request, global_params=global_params) - - Label.method_config = lambda: base_api.ApiMethodInfo( - flat_path='v1b3/projects/{projectsId}/catalogTemplates/{catalogTemplatesId}:label', - http_method='POST', - method_id='dataflow.projects.catalogTemplates.label', - ordered_params=['name'], - path_params=['name'], - query_params=[], - relative_path='v1b3/{+name}:label', - request_field='modifyTemplateVersionLabelRequest', - request_type_name='DataflowProjectsCatalogTemplatesLabelRequest', - response_type_name='ModifyTemplateVersionLabelResponse', - supports_download=False, - ) - - def Tag(self, request, global_params=None): - r"""Updates the tag of the TemplateVersion, and tag is unique in Template. If tag exists in another TemplateVersion in the Template, updates the tag to this TemplateVersion will remove it from the old TemplateVersion and add it to this TemplateVersion. If request is remove_only (remove_only = true), remove the tag from this TemplateVersion. - - Args: - request: (DataflowProjectsCatalogTemplatesTagRequest) input message - global_params: (StandardQueryParameters, default: None) global arguments - Returns: - (ModifyTemplateVersionTagResponse) The response message. - """ - config = self.GetMethodConfig('Tag') - return self._RunMethod( - config, request, global_params=global_params) - - Tag.method_config = lambda: base_api.ApiMethodInfo( - flat_path='v1b3/projects/{projectsId}/catalogTemplates/{catalogTemplatesId}:tag', - http_method='POST', - method_id='dataflow.projects.catalogTemplates.tag', - ordered_params=['name'], - path_params=['name'], - query_params=[], - relative_path='v1b3/{+name}:tag', - request_field='modifyTemplateVersionTagRequest', - request_type_name='DataflowProjectsCatalogTemplatesTagRequest', - response_type_name='ModifyTemplateVersionTagResponse', - supports_download=False, - ) - class ProjectsJobsDebugService(base_api.BaseApiService): """Service class for the projects_jobs_debug resource.""" _NAME = 'projects_jobs_debug' def __init__(self, client): - super().__init__(client) + super(DataflowV1b3.ProjectsJobsDebugService, self).__init__(client) self._upload_configs = { } @@ -327,7 +126,7 @@ class ProjectsJobsMessagesService(base_api.BaseApiService): _NAME = 'projects_jobs_messages' def __init__(self, client): - super().__init__(client) + super(DataflowV1b3.ProjectsJobsMessagesService, self).__init__(client) self._upload_configs = { } @@ -363,7 +162,7 @@ class ProjectsJobsWorkItemsService(base_api.BaseApiService): _NAME = 'projects_jobs_workItems' def __init__(self, client): - super().__init__(client) + super(DataflowV1b3.ProjectsJobsWorkItemsService, self).__init__(client) self._upload_configs = { } @@ -425,7 +224,7 @@ class ProjectsJobsService(base_api.BaseApiService): _NAME = 'projects_jobs' def __init__(self, client): - super().__init__(client) + super(DataflowV1b3.ProjectsJobsService, self).__init__(client) self._upload_configs = { } @@ -617,7 +416,7 @@ class ProjectsLocationsFlexTemplatesService(base_api.BaseApiService): _NAME = 'projects_locations_flexTemplates' def __init__(self, client): - super().__init__(client) + super(DataflowV1b3.ProjectsLocationsFlexTemplatesService, self).__init__(client) self._upload_configs = { } @@ -653,7 +452,7 @@ class ProjectsLocationsJobsDebugService(base_api.BaseApiService): _NAME = 'projects_locations_jobs_debug' def __init__(self, client): - super().__init__(client) + super(DataflowV1b3.ProjectsLocationsJobsDebugService, self).__init__(client) self._upload_configs = { } @@ -715,7 +514,7 @@ class ProjectsLocationsJobsMessagesService(base_api.BaseApiService): _NAME = 'projects_locations_jobs_messages' def __init__(self, client): - super().__init__(client) + super(DataflowV1b3.ProjectsLocationsJobsMessagesService, self).__init__(client) self._upload_configs = { } @@ -751,7 +550,7 @@ class ProjectsLocationsJobsSnapshotsService(base_api.BaseApiService): _NAME = 'projects_locations_jobs_snapshots' def __init__(self, client): - super().__init__(client) + super(DataflowV1b3.ProjectsLocationsJobsSnapshotsService, self).__init__(client) self._upload_configs = { } @@ -787,7 +586,7 @@ class ProjectsLocationsJobsStagesService(base_api.BaseApiService): _NAME = 'projects_locations_jobs_stages' def __init__(self, client): - super().__init__(client) + super(DataflowV1b3.ProjectsLocationsJobsStagesService, self).__init__(client) self._upload_configs = { } @@ -823,7 +622,7 @@ class ProjectsLocationsJobsWorkItemsService(base_api.BaseApiService): _NAME = 'projects_locations_jobs_workItems' def __init__(self, client): - super().__init__(client) + super(DataflowV1b3.ProjectsLocationsJobsWorkItemsService, self).__init__(client) self._upload_configs = { } @@ -885,7 +684,7 @@ class ProjectsLocationsJobsService(base_api.BaseApiService): _NAME = 'projects_locations_jobs' def __init__(self, client): - super().__init__(client) + super(DataflowV1b3.ProjectsLocationsJobsService, self).__init__(client) self._upload_configs = { } @@ -1077,7 +876,7 @@ class ProjectsLocationsSnapshotsService(base_api.BaseApiService): _NAME = 'projects_locations_snapshots' def __init__(self, client): - super().__init__(client) + super(DataflowV1b3.ProjectsLocationsSnapshotsService, self).__init__(client) self._upload_configs = { } @@ -1165,7 +964,7 @@ class ProjectsLocationsSqlService(base_api.BaseApiService): _NAME = 'projects_locations_sql' def __init__(self, client): - super().__init__(client) + super(DataflowV1b3.ProjectsLocationsSqlService, self).__init__(client) self._upload_configs = { } @@ -1201,7 +1000,7 @@ class ProjectsLocationsTemplatesService(base_api.BaseApiService): _NAME = 'projects_locations_templates' def __init__(self, client): - super().__init__(client) + super(DataflowV1b3.ProjectsLocationsTemplatesService, self).__init__(client) self._upload_configs = { } @@ -1289,7 +1088,7 @@ class ProjectsLocationsService(base_api.BaseApiService): _NAME = 'projects_locations' def __init__(self, client): - super().__init__(client) + super(DataflowV1b3.ProjectsLocationsService, self).__init__(client) self._upload_configs = { } @@ -1325,7 +1124,7 @@ class ProjectsSnapshotsService(base_api.BaseApiService): _NAME = 'projects_snapshots' def __init__(self, client): - super().__init__(client) + super(DataflowV1b3.ProjectsSnapshotsService, self).__init__(client) self._upload_configs = { } @@ -1381,50 +1180,13 @@ def List(self, request, global_params=None): supports_download=False, ) - class ProjectsTemplateVersionsService(base_api.BaseApiService): - """Service class for the projects_templateVersions resource.""" - - _NAME = 'projects_templateVersions' - - def __init__(self, client): - super().__init__(client) - self._upload_configs = { - } - - def List(self, request, global_params=None): - r"""List TemplateVersions using project_id and an optional display_name field. List all the TemplateVersions in the Template if display set. List all the TemplateVersions in the Project if display_name not set. - - Args: - request: (DataflowProjectsTemplateVersionsListRequest) input message - global_params: (StandardQueryParameters, default: None) global arguments - Returns: - (ListTemplateVersionsResponse) The response message. - """ - config = self.GetMethodConfig('List') - return self._RunMethod( - config, request, global_params=global_params) - - List.method_config = lambda: base_api.ApiMethodInfo( - flat_path='v1b3/projects/{projectsId}/templateVersions', - http_method='GET', - method_id='dataflow.projects.templateVersions.list', - ordered_params=['parent'], - path_params=['parent'], - query_params=['pageSize', 'pageToken'], - relative_path='v1b3/{+parent}/templateVersions', - request_field='', - request_type_name='DataflowProjectsTemplateVersionsListRequest', - response_type_name='ListTemplateVersionsResponse', - supports_download=False, - ) - class ProjectsTemplatesService(base_api.BaseApiService): """Service class for the projects_templates resource.""" _NAME = 'projects_templates' def __init__(self, client): - super().__init__(client) + super(DataflowV1b3.ProjectsTemplatesService, self).__init__(client) self._upload_configs = { } @@ -1512,7 +1274,7 @@ class ProjectsService(base_api.BaseApiService): _NAME = 'projects' def __init__(self, client): - super().__init__(client) + super(DataflowV1b3.ProjectsService, self).__init__(client) self._upload_configs = { } diff --git a/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_messages.py b/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_messages.py index b8b9f94d1aeb..e2e8ae879a36 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_messages.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_messages.py @@ -1,26 +1,11 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - """Generated message classes for dataflow version v1b3. Manages Google Cloud Dataflow projects on Google Cloud Platform. """ # NOTE: This file is autogenerated and should not be edited by hand. +from __future__ import absolute_import + from apitools.base.protorpclite import messages as _messages from apitools.base.py import encoding from apitools.base.py import extra_types @@ -103,20 +88,6 @@ class ApproximateSplitRequest(_messages.Message): position = _messages.MessageField('Position', 3) -class Artifact(_messages.Message): - r"""Job information for templates. - - Fields: - containerSpec: Container image path set for flex Template. - jobGraphGcsPath: job_graph_gcs_path set for legacy Template. - metadata: Metadata set for legacy Template. - """ - - containerSpec = _messages.MessageField('ContainerSpec', 1) - jobGraphGcsPath = _messages.StringField(2) - metadata = _messages.MessageField('TemplateMetadata', 3) - - class AutoscalingEvent(_messages.Message): r"""A structured message reporting an autoscaling decision made by the Dataflow service. @@ -135,8 +106,7 @@ class AutoscalingEvent(_messages.Message): time: The time this event was emitted to indicate a new target or current num_workers value. workerPool: A short and friendly name for the worker pool this event - refers to, populated from the value of - PoolStageRelation::user_pool_name. + refers to. """ class EventTypeValueValuesEnum(_messages.Enum): @@ -219,7 +189,7 @@ class BigQueryIODetails(_messages.Message): class BigTableIODetails(_messages.Message): - r"""Metadata for a BigTable connector used by the job. + r"""Metadata for a Cloud Bigtable connector used by the job. Fields: instanceId: InstanceId accessed in the connection. @@ -248,16 +218,6 @@ class CPUTime(_messages.Message): totalMs = _messages.IntegerField(3, variant=_messages.Variant.UINT64) -class CommitTemplateVersionRequest(_messages.Message): - r"""Commit will add a new TemplateVersion to an existing template. - - Fields: - templateVersion: TemplateVersion obejct to create. - """ - - templateVersion = _messages.MessageField('TemplateVersion', 1) - - class ComponentSource(_messages.Message): r"""Description of an interstitial value between transforms in an execution stage. @@ -585,16 +545,6 @@ class AdditionalProperty(_messages.Message): parameters = _messages.MessageField('ParametersValue', 5) -class CreateTemplateVersionRequest(_messages.Message): - r"""Creates a new Template with TemplateVersions. - - Fields: - templateVersion: The TemplateVersion object to create. - """ - - templateVersion = _messages.MessageField('TemplateVersion', 1) - - class CustomSourceLocation(_messages.Message): r"""Identifies the location of a custom souce. @@ -622,93 +572,6 @@ class DataDiskAssignment(_messages.Message): vmInstance = _messages.StringField(2) -class DataflowProjectsCatalogTemplatesCommitRequest(_messages.Message): - r"""A DataflowProjectsCatalogTemplatesCommitRequest object. - - Fields: - commitTemplateVersionRequest: A CommitTemplateVersionRequest resource to - be passed as the request body. - name: The location of the template, name includes project_id and - display_name. Commit using project_id(pid1) and display_name(tid1). - Format: projects/{pid1}/catalogTemplates/{tid1} - """ - - commitTemplateVersionRequest = _messages.MessageField('CommitTemplateVersionRequest', 1) - name = _messages.StringField(2, required=True) - - -class DataflowProjectsCatalogTemplatesDeleteRequest(_messages.Message): - r"""A DataflowProjectsCatalogTemplatesDeleteRequest object. - - Fields: - name: name includes project_id and display_name. Delete by - project_id(pid1) and display_name(tid1). Format: - projects/{pid1}/catalogTemplates/{tid1} - """ - - name = _messages.StringField(1, required=True) - - -class DataflowProjectsCatalogTemplatesGetRequest(_messages.Message): - r"""A DataflowProjectsCatalogTemplatesGetRequest object. - - Fields: - name: Resource name includes project_id and display_name. version_id is - optional. Get the latest TemplateVersion if version_id not set. Get by - project_id(pid1) and display_name(tid1): Format: - projects/{pid1}/catalogTemplates/{tid1} Get by project_id(pid1), - display_name(tid1), and version_id(vid1): Format: - projects/{pid1}/catalogTemplates/{tid1@vid} - """ - - name = _messages.StringField(1, required=True) - - -class DataflowProjectsCatalogTemplatesLabelRequest(_messages.Message): - r"""A DataflowProjectsCatalogTemplatesLabelRequest object. - - Fields: - modifyTemplateVersionLabelRequest: A ModifyTemplateVersionLabelRequest - resource to be passed as the request body. - name: Resource name includes project_id, display_name, and version_id. - Updates by project_id(pid1), display_name(tid1), and version_id(vid1): - Format: projects/{pid1}/catalogTemplates/{tid1@vid} - """ - - modifyTemplateVersionLabelRequest = _messages.MessageField('ModifyTemplateVersionLabelRequest', 1) - name = _messages.StringField(2, required=True) - - -class DataflowProjectsCatalogTemplatesTagRequest(_messages.Message): - r"""A DataflowProjectsCatalogTemplatesTagRequest object. - - Fields: - modifyTemplateVersionTagRequest: A ModifyTemplateVersionTagRequest - resource to be passed as the request body. - name: Resource name includes project_id, display_name, and version_id. - Updates by project_id(pid1), display_name(tid1), and version_id(vid1): - Format: projects/{pid1}/catalogTemplates/{tid1@vid} - """ - - modifyTemplateVersionTagRequest = _messages.MessageField('ModifyTemplateVersionTagRequest', 1) - name = _messages.StringField(2, required=True) - - -class DataflowProjectsCatalogTemplatesTemplateVersionsCreateRequest(_messages.Message): - r"""A DataflowProjectsCatalogTemplatesTemplateVersionsCreateRequest object. - - Fields: - createTemplateVersionRequest: A CreateTemplateVersionRequest resource to - be passed as the request body. - parent: The parent project and template that the TemplateVersion will be - created under. Create using project_id(pid1) and display_name(tid1). - Format: projects/{pid1}/catalogTemplates/{tid1} - """ - - createTemplateVersionRequest = _messages.MessageField('CreateTemplateVersionRequest', 1) - parent = _messages.StringField(2, required=True) - - class DataflowProjectsDeleteSnapshotsRequest(_messages.Message): r"""A DataflowProjectsDeleteSnapshotsRequest object. @@ -1687,8 +1550,9 @@ class DataflowProjectsLocationsTemplatesLaunchRequest(_messages.Message): r"""A DataflowProjectsLocationsTemplatesLaunchRequest object. Fields: - dynamicTemplate_gcsPath: Path to dynamic template spec file on GCS. The - file must be a Json serialized DynamicTemplateFieSpec object. + dynamicTemplate_gcsPath: Path to dynamic template spec file on Cloud + Storage. The file must be a Json serialized DynamicTemplateFieSpec + object. dynamicTemplate_stagingLocation: Cloud Storage path for staging dependencies. Must be a valid Cloud Storage URL, beginning with `gs://`. gcsPath: A Cloud Storage path to the template from which to create the @@ -1759,24 +1623,6 @@ class DataflowProjectsSnapshotsListRequest(_messages.Message): projectId = _messages.StringField(3, required=True) -class DataflowProjectsTemplateVersionsListRequest(_messages.Message): - r"""A DataflowProjectsTemplateVersionsListRequest object. - - Fields: - pageSize: The maximum number of TemplateVersions to return per page. - pageToken: The page token, received from a previous ListTemplateVersions - call. Provide this to retrieve the subsequent page. - parent: parent includes project_id, and display_name is optional. List by - project_id(pid1) and display_name(tid1). Format: - projects/{pid1}/catalogTemplates/{tid1} List by project_id(pid1). - Format: projects/{pid1} - """ - - pageSize = _messages.IntegerField(1, variant=_messages.Variant.INT32) - pageToken = _messages.StringField(2) - parent = _messages.StringField(3, required=True) - - class DataflowProjectsTemplatesCreateRequest(_messages.Message): r"""A DataflowProjectsTemplatesCreateRequest object. @@ -1827,8 +1673,9 @@ class DataflowProjectsTemplatesLaunchRequest(_messages.Message): r"""A DataflowProjectsTemplatesLaunchRequest object. Fields: - dynamicTemplate_gcsPath: Path to dynamic template spec file on GCS. The - file must be a Json serialized DynamicTemplateFieSpec object. + dynamicTemplate_gcsPath: Path to dynamic template spec file on Cloud + Storage. The file must be a Json serialized DynamicTemplateFieSpec + object. dynamicTemplate_stagingLocation: Cloud Storage path for staging dependencies. Must be a valid Cloud Storage URL, beginning with `gs://`. gcsPath: A Cloud Storage path to the template from which to create the @@ -2036,16 +1883,6 @@ class DynamicSourceSplit(_messages.Message): residual = _messages.MessageField('DerivedSource', 2) -class Empty(_messages.Message): - r"""A generic empty message that you can re-use to avoid defining duplicated - empty messages in your APIs. A typical example is to use it as the request - or the response type of an API method. For instance: service Foo { rpc - Bar(google.protobuf.Empty) returns (google.protobuf.Empty); } The JSON - representation for `Empty` is empty JSON object `{}`. - """ - - - class Environment(_messages.Message): r"""Describes the environment in which a Dataflow Job runs. @@ -2076,8 +1913,7 @@ class Environment(_messages.Message): debugOptions: Any debugging options to be supplied to the job. experiments: The list of experiments to enable. This field should be used for SDK related experiments and not for service related experiments. The - proper field for service related experiments is service_options. For - more details see the rationale at go/user-specified-service-options. + proper field for service related experiments is service_options. flexResourceSchedulingGoal: Which Flexible Resource Scheduling mode to run in. internalExperiments: Experimental settings. @@ -2094,8 +1930,7 @@ class Environment(_messages.Message): serviceOptions: The list of service options to enable. This field should be used for service related experiments only. These experiments, when graduating to GA, should be replaced by dedicated fields or become - default (i.e. always on). For more details see the rationale at go/user- - specified-service-options. + default (i.e. always on). shuffleMode: Output only. The shuffle mode used for the job. tempStoragePrefix: The prefix of the resources the system should use for temporary storage. The system will append the suffix "/temp-{JOBNAME} to @@ -2336,7 +2171,7 @@ class ExecutionStageStateValueValuesEnum(_messages.Enum): JOB_STATE_RESOURCE_CLEANING_UP: `JOB_STATE_RESOURCE_CLEANING_UP` indicates that the batch job's associated resources are currently being cleaned up after a successful run. Currently, this is an opt-in - feature, please reach out to Cloud support team if you are intersted. + feature, please reach out to Cloud support team if you are interested. """ JOB_STATE_UNKNOWN = 0 JOB_STATE_STOPPED = 1 @@ -2363,7 +2198,7 @@ class ExecutionStageSummary(_messages.Message): generated by the Dataflow service during execution planning. Enums: - KindValueValuesEnum: Type of tranform this stage is executing. + KindValueValuesEnum: Type of transform this stage is executing. Fields: componentSource: Collections produced and consumed by component transforms @@ -2371,7 +2206,7 @@ class ExecutionStageSummary(_messages.Message): componentTransform: Transforms that comprise this execution stage. id: Dataflow service generated id for this stage. inputSource: Input sources for this stage. - kind: Type of tranform this stage is executing. + kind: Type of transform this stage is executing. name: Dataflow service generated name for this stage. outputSource: Output sources for this stage. prerequisiteStage: Other stages that must complete before this stage can @@ -2379,7 +2214,7 @@ class ExecutionStageSummary(_messages.Message): """ class KindValueValuesEnum(_messages.Enum): - r"""Type of tranform this stage is executing. + r"""Type of transform this stage is executing. Values: UNKNOWN_KIND: Unrecognized transform type. @@ -2452,6 +2287,7 @@ class FlexTemplateRuntimeEnvironment(_messages.Message): r"""The environment values to be set at runtime for flex template. Enums: + AutoscalingAlgorithmValueValuesEnum: The algorithm to use for autoscaling FlexrsGoalValueValuesEnum: Set FlexRS goal for the job. https://cloud.google.com/dataflow/docs/guides/flexrs IpConfigurationValueValuesEnum: Configuration for VM IPs. @@ -2470,12 +2306,20 @@ class FlexTemplateRuntimeEnvironment(_messages.Message): restrictions](https://cloud.google.com/compute/docs/labeling- resources#restrictions) page. An object containing a list of "key": value pairs. Example: { "name": "wrench", "mass": "1kg", "count": "3" }. + autoscalingAlgorithm: The algorithm to use for autoscaling + diskSizeGb: Worker disk size, in gigabytes. + dumpHeapOnOom: If true, save a heap dump before killing a thread or + process which is GC thrashing or out of memory. The location of the heap + file will either be echoed back to the user, or the user will be given + the opportunity to download the heap file. enableStreamingEngine: Whether to enable Streaming Engine for the job. flexrsGoal: Set FlexRS goal for the job. https://cloud.google.com/dataflow/docs/guides/flexrs ipConfiguration: Configuration for VM IPs. kmsKeyName: Name for the Cloud KMS key for the job. Key format is: projects//locations//keyRings//cryptoKeys/ + launcherMachineType: The machine type to use for launching the job. The + default is n1-standard-1. machineType: The machine type to use for the job. Defaults to the value from the template if not specified. maxWorkers: The maximum number of Google Compute Engine instances to be @@ -2484,8 +2328,16 @@ class FlexTemplateRuntimeEnvironment(_messages.Message): the service will use the network "default". numWorkers: The initial number of Google Compute Engine instances for the job. + saveHeapDumpsToGcsPath: Cloud Storage bucket (directory) to upload heap + dumps to the given location. Enabling this implies that heap dumps + should be generated on OOM (dump_heap_on_oom is set to true). + sdkContainerImage: Docker registry location of container image to use for + the 'worker harness. Default is the container for the version of the + SDK. Note this field is only valid for portable pipelines. serviceAccountEmail: The email address of the service account to run the job as. + stagingLocation: The Cloud Storage path for staging local files. Must be a + valid Cloud Storage URL, beginning with `gs://`. subnetwork: Subnetwork to which VMs will be assigned, if desired. You can specify a subnetwork using either a complete URL or an abbreviated path. Expected to be of the form "https://www.googleapis.com/compute/v1/projec @@ -2512,6 +2364,19 @@ class FlexTemplateRuntimeEnvironment(_messages.Message): worker_zone will take precedence. """ + class AutoscalingAlgorithmValueValuesEnum(_messages.Enum): + r"""The algorithm to use for autoscaling + + Values: + AUTOSCALING_ALGORITHM_UNKNOWN: The algorithm is unknown, or unspecified. + AUTOSCALING_ALGORITHM_NONE: Disable autoscaling. + AUTOSCALING_ALGORITHM_BASIC: Increase worker count over time to reduce + job execution time. + """ + AUTOSCALING_ALGORITHM_UNKNOWN = 0 + AUTOSCALING_ALGORITHM_NONE = 1 + AUTOSCALING_ALGORITHM_BASIC = 2 + class FlexrsGoalValueValuesEnum(_messages.Enum): r"""Set FlexRS goal for the job. https://cloud.google.com/dataflow/docs/guides/flexrs @@ -2569,20 +2434,27 @@ class AdditionalProperty(_messages.Message): additionalExperiments = _messages.StringField(1, repeated=True) additionalUserLabels = _messages.MessageField('AdditionalUserLabelsValue', 2) - enableStreamingEngine = _messages.BooleanField(3) - flexrsGoal = _messages.EnumField('FlexrsGoalValueValuesEnum', 4) - ipConfiguration = _messages.EnumField('IpConfigurationValueValuesEnum', 5) - kmsKeyName = _messages.StringField(6) - machineType = _messages.StringField(7) - maxWorkers = _messages.IntegerField(8, variant=_messages.Variant.INT32) - network = _messages.StringField(9) - numWorkers = _messages.IntegerField(10, variant=_messages.Variant.INT32) - serviceAccountEmail = _messages.StringField(11) - subnetwork = _messages.StringField(12) - tempLocation = _messages.StringField(13) - workerRegion = _messages.StringField(14) - workerZone = _messages.StringField(15) - zone = _messages.StringField(16) + autoscalingAlgorithm = _messages.EnumField('AutoscalingAlgorithmValueValuesEnum', 3) + diskSizeGb = _messages.IntegerField(4, variant=_messages.Variant.INT32) + dumpHeapOnOom = _messages.BooleanField(5) + enableStreamingEngine = _messages.BooleanField(6) + flexrsGoal = _messages.EnumField('FlexrsGoalValueValuesEnum', 7) + ipConfiguration = _messages.EnumField('IpConfigurationValueValuesEnum', 8) + kmsKeyName = _messages.StringField(9) + launcherMachineType = _messages.StringField(10) + machineType = _messages.StringField(11) + maxWorkers = _messages.IntegerField(12, variant=_messages.Variant.INT32) + network = _messages.StringField(13) + numWorkers = _messages.IntegerField(14, variant=_messages.Variant.INT32) + saveHeapDumpsToGcsPath = _messages.StringField(15) + sdkContainerImage = _messages.StringField(16) + serviceAccountEmail = _messages.StringField(17) + stagingLocation = _messages.StringField(18) + subnetwork = _messages.StringField(19) + tempLocation = _messages.StringField(20) + workerRegion = _messages.StringField(21) + workerZone = _messages.StringField(22) + zone = _messages.StringField(23) class FloatingPointList(_messages.Message): @@ -2809,7 +2681,7 @@ class IntegerMean(_messages.Message): class Job(_messages.Message): - r"""Defines a job to be run by the Cloud Dataflow service. nextID: 26 + r"""Defines a job to be run by the Cloud Dataflow service. Enums: CurrentStateValueValuesEnum: The current state of the job. Jobs are @@ -2907,7 +2779,7 @@ class Job(_messages.Message): steps: Exactly one of step or steps_location should be specified. The top- level steps that constitute the entire job. Only retrieved with JOB_VIEW_ALL. - stepsLocation: The GCS location where the steps are stored. + stepsLocation: The Cloud Storage location where the steps are stored. tempFiles: A set of files the system should be aware of that are used for temporary storage. These temporary files will be removed on job completion. No duplicates are allowed. No file patterns are supported. @@ -2976,7 +2848,7 @@ class CurrentStateValueValuesEnum(_messages.Enum): JOB_STATE_RESOURCE_CLEANING_UP: `JOB_STATE_RESOURCE_CLEANING_UP` indicates that the batch job's associated resources are currently being cleaned up after a successful run. Currently, this is an opt-in - feature, please reach out to Cloud support team if you are intersted. + feature, please reach out to Cloud support team if you are interested. """ JOB_STATE_UNKNOWN = 0 JOB_STATE_STOPPED = 1 @@ -3048,7 +2920,7 @@ class RequestedStateValueValuesEnum(_messages.Enum): JOB_STATE_RESOURCE_CLEANING_UP: `JOB_STATE_RESOURCE_CLEANING_UP` indicates that the batch job's associated resources are currently being cleaned up after a successful run. Currently, this is an opt-in - feature, please reach out to Cloud support team if you are intersted. + feature, please reach out to Cloud support team if you are interested. """ JOB_STATE_UNKNOWN = 0 JOB_STATE_STOPPED = 1 @@ -3287,14 +3159,15 @@ class JobMetadata(_messages.Message): ListJob response and Job SUMMARY view. Fields: - bigTableDetails: Identification of a BigTable source used in the Dataflow - job. + bigTableDetails: Identification of a Cloud Bigtable source used in the + Dataflow job. bigqueryDetails: Identification of a BigQuery source used in the Dataflow job. datastoreDetails: Identification of a Datastore source used in the Dataflow job. fileDetails: Identification of a File source used in the Dataflow job. - pubsubDetails: Identification of a PubSub source used in the Dataflow job. + pubsubDetails: Identification of a Pub/Sub source used in the Dataflow + job. sdkVersion: The SDK version used to run the job. spannerDetails: Identification of a Spanner source used in the Dataflow job. @@ -3383,7 +3256,7 @@ class LaunchFlexTemplateParameter(_messages.Message): Fields: containerSpec: Spec about the container image to launch. - containerSpecGcsPath: Gcs path to a file with json serialized + containerSpecGcsPath: Cloud Storage path to a file with json serialized ContainerSpec as content. environment: The runtime environment for the FlexTemplate job jobName: Required. The job name to use for the created job. For update job @@ -3746,19 +3619,6 @@ class ListSnapshotsResponse(_messages.Message): snapshots = _messages.MessageField('Snapshot', 1, repeated=True) -class ListTemplateVersionsResponse(_messages.Message): - r"""Respond a list of TemplateVersions. - - Fields: - nextPageToken: A token that can be sent as `page_token` to retrieve the - next page. If this field is omitted, there are no subsequent pages. - templateVersions: A list of TemplateVersions. - """ - - nextPageToken = _messages.StringField(1) - templateVersions = _messages.MessageField('TemplateVersion', 2, repeated=True) - - class MapTask(_messages.Message): r"""MapTask consists of an ordered set of instructions, each of which describes one particular low-level operation for the worker to perform in @@ -3787,15 +3647,18 @@ class MemInfo(_messages.Message): Fields: currentLimitBytes: Instantenous memory limit in bytes. + currentOoms: Number of Out of Memory (OOM) events recorded since the + previous measurement. currentRssBytes: Instantenous memory (RSS) size in bytes. timestamp: Timestamp of the measurement. totalGbMs: Total memory (RSS) usage since start up in GB * ms. """ currentLimitBytes = _messages.IntegerField(1, variant=_messages.Variant.UINT64) - currentRssBytes = _messages.IntegerField(2, variant=_messages.Variant.UINT64) - timestamp = _messages.StringField(3) - totalGbMs = _messages.IntegerField(4, variant=_messages.Variant.UINT64) + currentOoms = _messages.IntegerField(2) + currentRssBytes = _messages.IntegerField(3, variant=_messages.Variant.UINT64) + timestamp = _messages.StringField(4) + totalGbMs = _messages.IntegerField(5, variant=_messages.Variant.UINT64) class MetricShortId(_messages.Message): @@ -3921,102 +3784,6 @@ class MetricUpdate(_messages.Message): updateTime = _messages.StringField(11) -class ModifyTemplateVersionLabelRequest(_messages.Message): - r"""Either add the label to TemplateVersion or remove it from the - TemplateVersion. - - Enums: - OpValueValuesEnum: Requests for add label to TemplateVersion or remove - label from TemplateVersion. - - Fields: - key: The label key for update. - op: Requests for add label to TemplateVersion or remove label from - TemplateVersion. - value: The label value for update. - """ - - class OpValueValuesEnum(_messages.Enum): - r"""Requests for add label to TemplateVersion or remove label from - TemplateVersion. - - Values: - OPERATION_UNSPECIFIED: Default value. - ADD: Add the label to the TemplateVersion object. - REMOVE: Remove the label from the TemplateVersion object. - """ - OPERATION_UNSPECIFIED = 0 - ADD = 1 - REMOVE = 2 - - key = _messages.StringField(1) - op = _messages.EnumField('OpValueValuesEnum', 2) - value = _messages.StringField(3) - - -class ModifyTemplateVersionLabelResponse(_messages.Message): - r"""Respond the labels in the TemplateVersion. - - Messages: - LabelsValue: All the label in the TemplateVersion. - - Fields: - labels: All the label in the TemplateVersion. - """ - - @encoding.MapUnrecognizedFields('additionalProperties') - class LabelsValue(_messages.Message): - r"""All the label in the TemplateVersion. - - Messages: - AdditionalProperty: An additional property for a LabelsValue object. - - Fields: - additionalProperties: Additional properties of type LabelsValue - """ - - class AdditionalProperty(_messages.Message): - r"""An additional property for a LabelsValue object. - - Fields: - key: Name of the additional property. - value: A string attribute. - """ - - key = _messages.StringField(1) - value = _messages.StringField(2) - - additionalProperties = _messages.MessageField('AdditionalProperty', 1, repeated=True) - - labels = _messages.MessageField('LabelsValue', 1) - - -class ModifyTemplateVersionTagRequest(_messages.Message): - r"""Add a tag to the current TemplateVersion. If tag exist in another - TemplateVersion in the Template, remove the tag before add it to the current - TemplateVersion. If remove_only set, remove the tag from the current - TemplateVersion. - - Fields: - removeOnly: The flag that indicates if the request is only for remove tag - from TemplateVersion. - tag: The tag for update. - """ - - removeOnly = _messages.BooleanField(1) - tag = _messages.StringField(2) - - -class ModifyTemplateVersionTagResponse(_messages.Message): - r"""Respond the current tags in the TemplateVersion. - - Fields: - tags: All the tags in the TemplateVersion. - """ - - tags = _messages.StringField(1, repeated=True) - - class MountedDataDisk(_messages.Message): r"""Describes mounted data disk. @@ -4198,7 +3965,13 @@ class ParameterMetadata(_messages.Message): ParamTypeValueValuesEnum: Optional. The type of the parameter. Used for selecting input picker. + Messages: + CustomMetadataValue: Optional. Additional metadata for describing this + parameter. + Fields: + customMetadata: Optional. Additional metadata for describing this + parameter. helpText: Required. The help text to display for the parameter. isOptional: Optional. Whether the parameter is optional. Defaults to false. @@ -4215,12 +3988,18 @@ class ParamTypeValueValuesEnum(_messages.Enum): Values: DEFAULT: Default input type. TEXT: The parameter specifies generic text input. - GCS_READ_BUCKET: The parameter specifies a GCS Bucket to read from. - GCS_WRITE_BUCKET: The parameter specifies a GCS Bucket to write to. - GCS_READ_FILE: The parameter specifies a GCS file path to read from. - GCS_WRITE_FILE: The parameter specifies a GCS file path to write to. - GCS_READ_FOLDER: The parameter specifies a GCS folder path to read from. - GCS_WRITE_FOLDER: The parameter specifies a GCS folder to write to. + GCS_READ_BUCKET: The parameter specifies a Cloud Storage Bucket to read + from. + GCS_WRITE_BUCKET: The parameter specifies a Cloud Storage Bucket to + write to. + GCS_READ_FILE: The parameter specifies a Cloud Storage file path to read + from. + GCS_WRITE_FILE: The parameter specifies a Cloud Storage file path to + write to. + GCS_READ_FOLDER: The parameter specifies a Cloud Storage folder path to + read from. + GCS_WRITE_FOLDER: The parameter specifies a Cloud Storage folder to + write to. PUBSUB_TOPIC: The parameter specifies a Pub/Sub Topic. PUBSUB_SUBSCRIPTION: The parameter specifies a Pub/Sub Subscription. """ @@ -4235,12 +4014,38 @@ class ParamTypeValueValuesEnum(_messages.Enum): PUBSUB_TOPIC = 8 PUBSUB_SUBSCRIPTION = 9 - helpText = _messages.StringField(1) - isOptional = _messages.BooleanField(2) - label = _messages.StringField(3) - name = _messages.StringField(4) - paramType = _messages.EnumField('ParamTypeValueValuesEnum', 5) - regexes = _messages.StringField(6, repeated=True) + @encoding.MapUnrecognizedFields('additionalProperties') + class CustomMetadataValue(_messages.Message): + r"""Optional. Additional metadata for describing this parameter. + + Messages: + AdditionalProperty: An additional property for a CustomMetadataValue + object. + + Fields: + additionalProperties: Additional properties of type CustomMetadataValue + """ + + class AdditionalProperty(_messages.Message): + r"""An additional property for a CustomMetadataValue object. + + Fields: + key: Name of the additional property. + value: A string attribute. + """ + + key = _messages.StringField(1) + value = _messages.StringField(2) + + additionalProperties = _messages.MessageField('AdditionalProperty', 1, repeated=True) + + customMetadata = _messages.MessageField('CustomMetadataValue', 1) + helpText = _messages.StringField(2) + isOptional = _messages.BooleanField(3) + label = _messages.StringField(4) + name = _messages.StringField(5) + paramType = _messages.EnumField('ParamTypeValueValuesEnum', 6) + regexes = _messages.StringField(7, repeated=True) class PartialGroupByKeyInstruction(_messages.Message): @@ -4393,7 +4198,7 @@ class ProgressTimeseries(_messages.Message): class PubSubIODetails(_messages.Message): - r"""Metadata for a PubSub connector used by the job. + r"""Metadata for a Pub/Sub connector used by the job. Fields: subscription: Subscription used in the connection. @@ -4666,7 +4471,8 @@ class RuntimeEnvironment(_messages.Message): value pairs. Example: { "name": "wrench", "mass": "1kg", "count": "3" }. Fields: - additionalExperiments: Additional experiment flags for the job. + additionalExperiments: Additional experiment flags for the job, specified + with the `--experiments` option. additionalUserLabels: Additional user labels to be specified for the job. Keys and values should follow the restrictions specified in the [labeling restrictions](https://cloud.google.com/compute/docs/labeling- @@ -4817,6 +4623,9 @@ class SdkHarnessContainerImage(_messages.Message): r"""Defines a SDK harness container for executing Dataflow pipelines. Fields: + capabilities: The set of capabilities enumerated in the above Environment + proto. See also https://github.com/apache/beam/blob/master/model/pipelin + e/src/main/proto/beam_runner_api.proto containerImage: A docker container image that resides in Google Container Registry. environmentId: Environment ID for the Beam runner API proto Environment @@ -4828,9 +4637,10 @@ class SdkHarnessContainerImage(_messages.Message): override this property if needed. """ - containerImage = _messages.StringField(1) - environmentId = _messages.StringField(2) - useSingleCorePerContainer = _messages.BooleanField(3) + capabilities = _messages.StringField(1, repeated=True) + containerImage = _messages.StringField(2) + environmentId = _messages.StringField(3) + useSingleCorePerContainer = _messages.BooleanField(4) class SdkVersion(_messages.Message): @@ -4854,7 +4664,7 @@ class SdkSupportStatusValueValuesEnum(_messages.Enum): STALE: A newer version of the SDK family exists, and an update is recommended. DEPRECATED: This version of the SDK is deprecated and will eventually be - no longer supported. + unsupported. UNSUPPORTED: Support for this SDK version has ended and it should no longer be used. """ @@ -4870,22 +4680,45 @@ class SdkSupportStatusValueValuesEnum(_messages.Enum): class SendDebugCaptureRequest(_messages.Message): - r"""Request to send encoded debug information. + r"""Request to send encoded debug information. Next ID: 8 + + Enums: + DataFormatValueValuesEnum: Format for the data field above (id=5). Fields: componentId: The internal component id for which debug information is sent. data: The encoded debug information. + dataFormat: Format for the data field above (id=5). location: The [regional endpoint] (https://cloud.google.com/dataflow/docs/concepts/regional-endpoints) that contains the job specified by job_id. workerId: The worker id, i.e., VM hostname. """ + class DataFormatValueValuesEnum(_messages.Enum): + r"""Format for the data field above (id=5). + + Values: + DATA_FORMAT_UNSPECIFIED: Format unspecified, parsing is determined based + upon page type and legacy encoding. (go/protodosdonts#do-include-an- + unspecified-value-in-an-enum) + RAW: Raw HTML string. + JSON: JSON-encoded string. + ZLIB: Websafe encoded zlib-compressed string. + BROTLI: Websafe encoded brotli-compressed string. + """ + DATA_FORMAT_UNSPECIFIED = 0 + RAW = 1 + JSON = 2 + ZLIB = 3 + BROTLI = 4 + componentId = _messages.StringField(1) data = _messages.StringField(2) - location = _messages.StringField(3) - workerId = _messages.StringField(4) + dataFormat = _messages.EnumField('DataFormatValueValuesEnum', 3) + location = _messages.StringField(4) + workerId = _messages.StringField(5) class SendDebugCaptureResponse(_messages.Message): @@ -5111,7 +4944,8 @@ class Snapshot(_messages.Message): snapshots in READY state. id: The unique ID of this snapshot. projectId: The project this snapshot belongs to. - pubsubMetadata: PubSub snapshot metadata. + pubsubMetadata: Pub/Sub snapshot metadata. + region: Cloud region where this snapshot lives in, e.g., "us-central1". sourceJobId: The job this snapshot was created from. state: State of the snapshot. ttl: The time after which this snapshot will be automatically deleted. @@ -5142,9 +4976,10 @@ class StateValueValuesEnum(_messages.Enum): id = _messages.StringField(4) projectId = _messages.StringField(5) pubsubMetadata = _messages.MessageField('PubsubSnapshotMetadata', 6, repeated=True) - sourceJobId = _messages.StringField(7) - state = _messages.EnumField('StateValueValuesEnum', 8) - ttl = _messages.StringField(9) + region = _messages.StringField(7) + sourceJobId = _messages.StringField(8) + state = _messages.EnumField('StateValueValuesEnum', 9) + ttl = _messages.StringField(10) class SnapshotJobRequest(_messages.Message): @@ -6118,87 +5953,6 @@ class TemplateMetadata(_messages.Message): parameters = _messages.MessageField('ParameterMetadata', 3, repeated=True) -class TemplateVersion(_messages.Message): - r"""//////////////////////////////////////////////////////////////////////// - ///// //// Template Catalog is used to organize user TemplateVersions. //// - TemplateVersions that have the same project_id and display_name are //// - belong to the same Template. //// Templates with the same project_id belong - to the same Project. //// TemplateVersion may have labels and multiple - labels are allowed. //// Duplicated labels in the same `TemplateVersion` are - not allowed. //// TemplateVersion may have tags and multiple tags are - allowed. Duplicated //// tags in the same `Template` are not allowed! - - Enums: - TypeValueValuesEnum: Either LEGACY or FLEX. This should match with the - type of artifact. - - Messages: - LabelsValue: Labels for the Template Version. Labels can be duplicate - within Template. - - Fields: - artifact: Job graph and metadata if it is a legacy Template. Container - image path and metadata if it is flex Template. - createTime: Creation time of this TemplateVersion. - description: Template description from the user. - displayName: A customized name for Template. Multiple TemplateVersions per - Template. - labels: Labels for the Template Version. Labels can be duplicate within - Template. - projectId: A unique project_id. Multiple Templates per Project. - tags: Alias for version_id, helps locate a TemplateVersion. - type: Either LEGACY or FLEX. This should match with the type of artifact. - versionId: An auto generated version_id for TemplateVersion. - """ - - class TypeValueValuesEnum(_messages.Enum): - r"""Either LEGACY or FLEX. This should match with the type of artifact. - - Values: - TEMPLATE_TYPE_UNSPECIFIED: Default value. Not a useful zero case. - LEGACY: Legacy Template. - FLEX: Flex Template. - """ - TEMPLATE_TYPE_UNSPECIFIED = 0 - LEGACY = 1 - FLEX = 2 - - @encoding.MapUnrecognizedFields('additionalProperties') - class LabelsValue(_messages.Message): - r"""Labels for the Template Version. Labels can be duplicate within - Template. - - Messages: - AdditionalProperty: An additional property for a LabelsValue object. - - Fields: - additionalProperties: Additional properties of type LabelsValue - """ - - class AdditionalProperty(_messages.Message): - r"""An additional property for a LabelsValue object. - - Fields: - key: Name of the additional property. - value: A string attribute. - """ - - key = _messages.StringField(1) - value = _messages.StringField(2) - - additionalProperties = _messages.MessageField('AdditionalProperty', 1, repeated=True) - - artifact = _messages.MessageField('Artifact', 1) - createTime = _messages.StringField(2) - description = _messages.StringField(3) - displayName = _messages.StringField(4) - labels = _messages.MessageField('LabelsValue', 5) - projectId = _messages.StringField(6) - tags = _messages.StringField(7, repeated=True) - type = _messages.EnumField('TypeValueValuesEnum', 8) - versionId = _messages.StringField(9) - - class TopologyConfig(_messages.Message): r"""Global topology of the streaming Dataflow job, including all computations and their sharded locations. @@ -6572,7 +6326,7 @@ class WorkerHealthReport(_messages.Message): PodsValueListEntry: A PodsValueListEntry object. Fields: - msg: A message describing any unusual health reports. + msg: Message describing any unusual health reports. pods: The pods running on the worker. See: http://kubernetes.io/v1.1/docs/api-reference/v1/definitions.html#_v1_pod This field is used by the worker to send the status of the indvidual @@ -6580,6 +6334,8 @@ class WorkerHealthReport(_messages.Message): reportInterval: The interval at which the worker is sending health reports. The default value of 0 should be interpreted as the field is not being explicitly set by the worker. + vmBrokenCode: Code to describe a specific reason, if known, that a VM has + reported broken state. vmIsBroken: Whether the VM is in a permanently broken state. Broken VMs should be abandoned or deleted ASAP to avoid assigning or completing any work. @@ -6615,9 +6371,10 @@ class AdditionalProperty(_messages.Message): msg = _messages.StringField(1) pods = _messages.MessageField('PodsValueListEntry', 2, repeated=True) reportInterval = _messages.StringField(3) - vmIsBroken = _messages.BooleanField(4) - vmIsHealthy = _messages.BooleanField(5) - vmStartupTime = _messages.StringField(6) + vmBrokenCode = _messages.StringField(4) + vmIsBroken = _messages.BooleanField(5) + vmIsHealthy = _messages.BooleanField(6) + vmStartupTime = _messages.StringField(7) class WorkerHealthReportResponse(_messages.Message): @@ -6789,37 +6546,37 @@ class WorkerMessageCode(_messages.Message): This is a struct to allow parameters of different types. Examples: 1. For a "HARNESS_STARTED" message parameters might provide the name of the worker and additional data like timing information. 2. For a - "GCS_DOWNLOAD_ERROR" parameters might contain fields listing the GCS - objects being downloaded and fields containing errors. In general - complex data structures should be avoided. If a worker needs to send a - specific and complicated data structure then please consider defining a - new proto and adding it to the data oneof in WorkerMessageResponse. - Conventions: Parameters should only be used for information that isn't - typically passed as a label. hostname and other worker identifiers - should almost always be passed as labels since they will be included on - most messages. + "GCS_DOWNLOAD_ERROR" parameters might contain fields listing the Cloud + Storage objects being downloaded and fields containing errors. In + general complex data structures should be avoided. If a worker needs to + send a specific and complicated data structure then please consider + defining a new proto and adding it to the data oneof in + WorkerMessageResponse. Conventions: Parameters should only be used for + information that isn't typically passed as a label. hostname and other + worker identifiers should almost always be passed as labels since they + will be included on most messages. Fields: code: The code is a string intended for consumption by a machine that identifies the type of message being sent. Examples: 1. "HARNESS_STARTED" might be used to indicate the worker harness has started. 2. "GCS_DOWNLOAD_ERROR" might be used to indicate an error - downloading a GCS file as part of the boot process of one of the worker - containers. This is a string and not an enum to make it easy to add new - codes without waiting for an API change. + downloading a Cloud Storage file as part of the boot process of one of + the worker containers. This is a string and not an enum to make it easy + to add new codes without waiting for an API change. parameters: Parameters contains specific information about the code. This is a struct to allow parameters of different types. Examples: 1. For a "HARNESS_STARTED" message parameters might provide the name of the worker and additional data like timing information. 2. For a - "GCS_DOWNLOAD_ERROR" parameters might contain fields listing the GCS - objects being downloaded and fields containing errors. In general - complex data structures should be avoided. If a worker needs to send a - specific and complicated data structure then please consider defining a - new proto and adding it to the data oneof in WorkerMessageResponse. - Conventions: Parameters should only be used for information that isn't - typically passed as a label. hostname and other worker identifiers - should almost always be passed as labels since they will be included on - most messages. + "GCS_DOWNLOAD_ERROR" parameters might contain fields listing the Cloud + Storage objects being downloaded and fields containing errors. In + general complex data structures should be avoided. If a worker needs to + send a specific and complicated data structure then please consider + defining a new proto and adding it to the data oneof in + WorkerMessageResponse. Conventions: Parameters should only be used for + information that isn't typically passed as a label. hostname and other + worker identifiers should almost always be passed as labels since they + will be included on most messages. """ @encoding.MapUnrecognizedFields('additionalProperties') @@ -6828,14 +6585,14 @@ class ParametersValue(_messages.Message): struct to allow parameters of different types. Examples: 1. For a "HARNESS_STARTED" message parameters might provide the name of the worker and additional data like timing information. 2. For a "GCS_DOWNLOAD_ERROR" - parameters might contain fields listing the GCS objects being downloaded - and fields containing errors. In general complex data structures should be - avoided. If a worker needs to send a specific and complicated data - structure then please consider defining a new proto and adding it to the - data oneof in WorkerMessageResponse. Conventions: Parameters should only - be used for information that isn't typically passed as a label. hostname - and other worker identifiers should almost always be passed as labels - since they will be included on most messages. + parameters might contain fields listing the Cloud Storage objects being + downloaded and fields containing errors. In general complex data + structures should be avoided. If a worker needs to send a specific and + complicated data structure then please consider defining a new proto and + adding it to the data oneof in WorkerMessageResponse. Conventions: + Parameters should only be used for information that isn't typically passed + as a label. hostname and other worker identifiers should almost always be + passed as labels since they will be included on most messages. Messages: AdditionalProperty: An additional property for a ParametersValue object. @@ -6978,7 +6735,7 @@ class DefaultPackageSetValueValuesEnum(_messages.Enum): the worker unless explicitly specified by the job. DEFAULT_PACKAGE_SET_JAVA: Stage packages typically useful to workers written in Java. - DEFAULT_PACKAGE_SET_PYTHON: Stage pacakges typically useful to workers + DEFAULT_PACKAGE_SET_PYTHON: Stage packages typically useful to workers written in Python. """ DEFAULT_PACKAGE_SET_UNKNOWN = 0 From c9a7707aa48b1bad1a8b8a5477bc01dd959a683a Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 7 Mar 2022 16:58:41 -0800 Subject: [PATCH 56/68] Instructions for updating apitools generated files. --- build.gradle.kts | 3 +++ .../runners/dataflow/internal/clients/README.txt | 11 +++++++++++ 2 files changed, 14 insertions(+) create mode 100644 sdks/python/apache_beam/runners/dataflow/internal/clients/README.txt diff --git a/build.gradle.kts b/build.gradle.kts index a49109d31cf4..c52201eb05e7 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -118,6 +118,9 @@ tasks.rat { // JupyterLab extensions "sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/yarn.lock", + // Autogenerated apitools clients. + "sdks/python/apache_beam/runners/dataflow/internal/clients/*/**/*.py", + // Sample text file for Java quickstart "sdks/java/maven-archetypes/examples/sample.txt", diff --git a/sdks/python/apache_beam/runners/dataflow/internal/clients/README.txt b/sdks/python/apache_beam/runners/dataflow/internal/clients/README.txt new file mode 100644 index 000000000000..1d697caeee6a --- /dev/null +++ b/sdks/python/apache_beam/runners/dataflow/internal/clients/README.txt @@ -0,0 +1,11 @@ +To regenerate these files run + +pip install google-apitools[cli] +gen_client \ + --discovery_url dataflow.v1b3 \ + --overwrite \ + --root_package=. \ + --outdir=apache_beam/runners/dataflow/internal/clients/dataflow \ + client + +Patch up the imports in __init__ to make them conditional. From 52ba2b8fa7e21dcec74c5ebf01ed96ef4f6a16a0 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Tue, 8 Mar 2022 14:35:41 -0500 Subject: [PATCH 57/68] [BEAM-10976] Bundle finalization: Harness and some exec changes (#16980) * Bundle finalization harness side changes * Add testing * Iterate over pardos directly * Track bundlefinalizer in plan.go not pardo * Remove outdated test * Fix pointer issue * Update todos to reference jiras * Cleanup from feedback * Doc nit Co-authored-by: Daniel Oliveira * GetExpirationTime comment Co-authored-by: github-actions Co-authored-by: Daniel Oliveira --- sdks/go/pkg/beam/core/runtime/exec/combine.go | 12 +-- sdks/go/pkg/beam/core/runtime/exec/fn.go | 53 +++++++++-- sdks/go/pkg/beam/core/runtime/exec/fn_test.go | 62 +++++++++++-- sdks/go/pkg/beam/core/runtime/exec/pardo.go | 9 +- sdks/go/pkg/beam/core/runtime/exec/plan.go | 50 ++++++++++ sdks/go/pkg/beam/core/runtime/exec/util.go | 2 +- .../pkg/beam/core/runtime/harness/harness.go | 92 ++++++++++++++++--- sdks/go/pkg/beam/runners/direct/buffer.go | 1 - 8 files changed, 238 insertions(+), 43 deletions(-) diff --git a/sdks/go/pkg/beam/core/runtime/exec/combine.go b/sdks/go/pkg/beam/core/runtime/exec/combine.go index 07dbb2f9a84d..692f3f464710 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/combine.go +++ b/sdks/go/pkg/beam/core/runtime/exec/combine.go @@ -74,7 +74,7 @@ func (n *Combine) Up(ctx context.Context) error { n.states = metrics.NewPTransformState(n.PID) - if _, err := InvokeWithoutEventTime(ctx, n.Fn.SetupFn(), nil); err != nil { + if _, err := InvokeWithoutEventTime(ctx, n.Fn.SetupFn(), nil, nil); err != nil { return n.fail(err) } @@ -107,7 +107,7 @@ func (n *Combine) mergeAccumulators(ctx context.Context, a, b interface{}) (inte } in := &MainInput{Key: FullValue{Elm: a}} - val, err := n.mergeInv.InvokeWithoutEventTime(ctx, in, b) + val, err := n.mergeInv.InvokeWithoutEventTime(ctx, in, nil, b) if err != nil { return nil, n.fail(errors.WithContext(err, "invoking MergeAccumulators")) } @@ -213,7 +213,7 @@ func (n *Combine) Down(ctx context.Context) error { } n.status = Down - if _, err := InvokeWithoutEventTime(ctx, n.Fn.TeardownFn(), nil); err != nil { + if _, err := InvokeWithoutEventTime(ctx, n.Fn.TeardownFn(), nil, nil); err != nil { n.err.TrySetError(err) } return n.err.Error() @@ -230,7 +230,7 @@ func (n *Combine) newAccum(ctx context.Context, key interface{}) (interface{}, e opt = &MainInput{Key: FullValue{Elm: key}} } - val, err := n.createAccumInv.InvokeWithoutEventTime(ctx, opt) + val, err := n.createAccumInv.InvokeWithoutEventTime(ctx, opt, nil) if err != nil { return nil, n.fail(errors.WithContext(err, "invoking CreateAccumulator")) } @@ -273,7 +273,7 @@ func (n *Combine) addInput(ctx context.Context, accum, key, value interface{}, t } v := n.aiValConvert(value) - val, err := n.addInputInv.InvokeWithoutEventTime(ctx, opt, v) + val, err := n.addInputInv.InvokeWithoutEventTime(ctx, opt, nil, v) if err != nil { return nil, n.fail(errors.WithContext(err, "invoking AddInput")) } @@ -287,7 +287,7 @@ func (n *Combine) extract(ctx context.Context, accum interface{}) (interface{}, return accum, nil } - val, err := n.extractOutputInv.InvokeWithoutEventTime(ctx, nil, accum) + val, err := n.extractOutputInv.InvokeWithoutEventTime(ctx, nil, nil, accum) if err != nil { return nil, n.fail(errors.WithContext(err, "invoking ExtractOutput")) } diff --git a/sdks/go/pkg/beam/core/runtime/exec/fn.go b/sdks/go/pkg/beam/core/runtime/exec/fn.go index 456edeba4838..5f90bdf1d774 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/fn.go +++ b/sdks/go/pkg/beam/core/runtime/exec/fn.go @@ -19,6 +19,7 @@ import ( "context" "fmt" "reflect" + "time" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/funcx" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph" @@ -39,23 +40,47 @@ type MainInput struct { RTracker sdf.RTracker } +type bundleFinalizationCallback struct { + callback func() error + validUntil time.Time +} + +// bundleFinalizer holds all the user defined callbacks to be run on bundle finalization. +// Implements typex.BundleFinalization +type bundleFinalizer struct { + callbacks []bundleFinalizationCallback + lastValidCallback time.Time // Used to track when we can safely gc the bundleFinalizer +} + +// RegisterCallback is used to register callbacks during DoFn execution. +func (bf *bundleFinalizer) RegisterCallback(t time.Duration, cb func() error) { + callback := bundleFinalizationCallback{ + callback: cb, + validUntil: time.Now().Add(t), + } + bf.callbacks = append(bf.callbacks, callback) + if bf.lastValidCallback.Before(callback.validUntil) { + bf.lastValidCallback = callback.validUntil + } +} + // Invoke invokes the fn with the given values. The extra values must match the non-main // side input and emitters. It returns the direct output, if any. -func Invoke(ctx context.Context, pn typex.PaneInfo, ws []typex.Window, ts typex.EventTime, fn *funcx.Fn, opt *MainInput, extra ...interface{}) (*FullValue, error) { +func Invoke(ctx context.Context, pn typex.PaneInfo, ws []typex.Window, ts typex.EventTime, fn *funcx.Fn, opt *MainInput, bf *bundleFinalizer, extra ...interface{}) (*FullValue, error) { if fn == nil { return nil, nil // ok: nothing to Invoke } inv := newInvoker(fn) - return inv.Invoke(ctx, pn, ws, ts, opt, extra...) + return inv.Invoke(ctx, pn, ws, ts, opt, bf, extra...) } // InvokeWithoutEventTime runs the given function at time 0 in the global window. -func InvokeWithoutEventTime(ctx context.Context, fn *funcx.Fn, opt *MainInput, extra ...interface{}) (*FullValue, error) { +func InvokeWithoutEventTime(ctx context.Context, fn *funcx.Fn, opt *MainInput, bf *bundleFinalizer, extra ...interface{}) (*FullValue, error) { if fn == nil { return nil, nil // ok: nothing to Invoke } inv := newInvoker(fn) - return inv.InvokeWithoutEventTime(ctx, opt, extra...) + return inv.InvokeWithoutEventTime(ctx, opt, bf, extra...) } // invoker is a container struct for hot path invocations of DoFns, to avoid @@ -64,9 +89,9 @@ type invoker struct { fn *funcx.Fn args []interface{} // TODO(lostluck): 2018/07/06 consider replacing with a slice of functions to run over the args slice, as an improvement. - ctxIdx, pnIdx, wndIdx, etIdx int // specialized input indexes - outEtIdx, outErrIdx int // specialized output indexes - in, out []int // general indexes + ctxIdx, pnIdx, wndIdx, etIdx, bfIdx int // specialized input indexes + outEtIdx, outErrIdx int // specialized output indexes + in, out []int // general indexes ret FullValue // ret is a cached allocation for passing to the next Unit. Units never modify the passed in FullValue. elmConvert, elm2Convert func(interface{}) interface{} // Cached conversion functions, which assums this invoker is always used with the same parameter types. @@ -99,6 +124,11 @@ func newInvoker(fn *funcx.Fn) *invoker { if n.outErrIdx, ok = fn.Error(); !ok { n.outErrIdx = -1 } + // TODO(BEAM-10976) - add this back in once BundleFinalization is implemented + // if n.bfIdx, ok = fn.BundleFinalization(); !ok { + // n.bfIdx = -1 + // } + n.bfIdx = -1 n.initCall() @@ -115,13 +145,13 @@ func (n *invoker) Reset() { } // InvokeWithoutEventTime runs the function at time 0 in the global window. -func (n *invoker) InvokeWithoutEventTime(ctx context.Context, opt *MainInput, extra ...interface{}) (*FullValue, error) { - return n.Invoke(ctx, typex.NoFiringPane(), window.SingleGlobalWindow, mtime.ZeroTimestamp, opt, extra...) +func (n *invoker) InvokeWithoutEventTime(ctx context.Context, opt *MainInput, bf *bundleFinalizer, extra ...interface{}) (*FullValue, error) { + return n.Invoke(ctx, typex.NoFiringPane(), window.SingleGlobalWindow, mtime.ZeroTimestamp, opt, bf, extra...) } // Invoke invokes the fn with the given values. The extra values must match the non-main // side input and emitters. It returns the direct output, if any. -func (n *invoker) Invoke(ctx context.Context, pn typex.PaneInfo, ws []typex.Window, ts typex.EventTime, opt *MainInput, extra ...interface{}) (*FullValue, error) { +func (n *invoker) Invoke(ctx context.Context, pn typex.PaneInfo, ws []typex.Window, ts typex.EventTime, opt *MainInput, bf *bundleFinalizer, extra ...interface{}) (*FullValue, error) { // (1) Populate contexts // extract these to make things easier to read. args := n.args @@ -143,6 +173,9 @@ func (n *invoker) Invoke(ctx context.Context, pn typex.PaneInfo, ws []typex.Wind if n.etIdx >= 0 { args[n.etIdx] = ts } + if n.bfIdx >= 0 { + args[n.bfIdx] = bf + } // (2) Main input from value, if any. i := 0 diff --git a/sdks/go/pkg/beam/core/runtime/exec/fn_test.go b/sdks/go/pkg/beam/core/runtime/exec/fn_test.go index b7ab50108296..b4db872395c6 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/fn_test.go +++ b/sdks/go/pkg/beam/core/runtime/exec/fn_test.go @@ -17,6 +17,7 @@ package exec import ( "context" + "errors" "fmt" "reflect" "testing" @@ -178,7 +179,7 @@ func TestInvoke(t *testing.T) { test.ExpectedTime = ts } - val, err := Invoke(context.Background(), typex.NoFiringPane(), window.SingleGlobalWindow, ts, fn, test.Opt, test.Args...) + val, err := Invoke(context.Background(), typex.NoFiringPane(), window.SingleGlobalWindow, ts, fn, test.Opt, nil, test.Args...) if err != nil { t.Fatalf("Invoke(%v,%v) failed: %v", fn.Fn.Name(), test.Args, err) } @@ -195,6 +196,53 @@ func TestInvoke(t *testing.T) { } } +func TestRegisterCallback(t *testing.T) { + bf := bundleFinalizer{ + callbacks: []bundleFinalizationCallback{}, + lastValidCallback: time.Now(), + } + testVar := 0 + bf.RegisterCallback(500*time.Minute, func() error { + testVar += 5 + return nil + }) + bf.RegisterCallback(2*time.Minute, func() error { + testVar = 25 + return nil + }) + callbackErr := errors.New("Callback error") + bf.RegisterCallback(2*time.Minute, func() error { + return callbackErr + }) + + // We can't do exact equality since this relies on real time, we'll give it a broad range + if bf.lastValidCallback.Before(time.Now().Add(400*time.Minute)) || bf.lastValidCallback.After(time.Now().Add(600*time.Minute)) { + t.Errorf("RegisterCallback() lastValidCallback set to %v, want about 500 minutes", bf.lastValidCallback) + } + if got, want := len(bf.callbacks), 3; got != want { + t.Fatalf("Callbacks in bundleFinalizer does not match number of calls to RegisterCallback(), got %v callbacks, want %v", got, want) + } + + callbackIdx := 0 + if err := bf.callbacks[callbackIdx].callback(); err != nil { + t.Errorf("RegisterCallback() callback at index %v returned unexpected error: %v", callbackIdx, err) + } + if got, want := testVar, 5; got != want { + t.Errorf("RegisterCallback() callback at index %v set testvar to %v, want %v", callbackIdx, got, want) + } + callbackIdx = 1 + if err := bf.callbacks[callbackIdx].callback(); err != nil { + t.Errorf("RegisterCallback() callback at index %v returned error %v, want nil", callbackIdx, err) + } + if got, want := testVar, 25; got != want { + t.Errorf("RegisterCallback() callback at index %v set testvar to %v, want %v", callbackIdx, got, want) + } + callbackIdx = 2 + if err := bf.callbacks[2].callback(); err != callbackErr { + t.Errorf("RegisterCallback() callback at index %v returned error %v, want %v", callbackIdx, err, callbackErr) + } +} + // Benchmarks // Invoke is implemented as a single use of a cached invoker, so a measure of @@ -314,7 +362,7 @@ func BenchmarkInvoke(b *testing.B) { ts := mtime.ZeroTimestamp.Add(2 * time.Millisecond) b.Run(fmt.Sprintf("SingleInvoker_%s", test.Name), func(b *testing.B) { for i := 0; i < b.N; i++ { - _, err := Invoke(context.Background(), typex.NoFiringPane(), window.SingleGlobalWindow, ts, fn, test.Opt, test.Args...) + _, err := Invoke(context.Background(), typex.NoFiringPane(), window.SingleGlobalWindow, ts, fn, test.Opt, nil, test.Args...) if err != nil { b.Fatalf("Invoke(%v,%v) failed: %v", fn.Fn.Name(), test.Args, err) } @@ -323,7 +371,7 @@ func BenchmarkInvoke(b *testing.B) { b.Run(fmt.Sprintf("CachedInvoker_%s", test.Name), func(b *testing.B) { inv := newInvoker(fn) for i := 0; i < b.N; i++ { - _, err := inv.Invoke(context.Background(), typex.NoFiringPane(), window.SingleGlobalWindow, ts, test.Opt, test.Args...) + _, err := inv.Invoke(context.Background(), typex.NoFiringPane(), window.SingleGlobalWindow, ts, test.Opt, nil, test.Args...) if err != nil { b.Fatalf("Invoke(%v,%v) failed: %v", fn.Fn.Name(), test.Args, err) } @@ -416,7 +464,7 @@ func BenchmarkInvokeCall(b *testing.B) { ctx := context.Background() n := 0 for i := 0; i < b.N; i++ { - ret, _ := InvokeWithoutEventTime(ctx, fn, &MainInput{Key: FullValue{Elm: n}}) + ret, _ := InvokeWithoutEventTime(ctx, fn, &MainInput{Key: FullValue{Elm: n}}, nil) n = ret.Elm.(int) } b.Log(n) @@ -427,7 +475,7 @@ func BenchmarkInvokeCallExtra(b *testing.B) { ctx := context.Background() n := 0 for i := 0; i < b.N; i++ { - ret, _ := InvokeWithoutEventTime(ctx, fn, nil, n) + ret, _ := InvokeWithoutEventTime(ctx, fn, nil, nil, n) n = ret.Elm.(int) } b.Log(n) @@ -453,7 +501,7 @@ func BenchmarkInvokeFnCall(b *testing.B) { ctx := context.Background() n := 0 for i := 0; i < b.N; i++ { - ret, _ := InvokeWithoutEventTime(ctx, fn, &MainInput{Key: FullValue{Elm: n}}) + ret, _ := InvokeWithoutEventTime(ctx, fn, &MainInput{Key: FullValue{Elm: n}}, nil) n = ret.Elm.(int) } b.Log(n) @@ -464,7 +512,7 @@ func BenchmarkInvokeFnCallExtra(b *testing.B) { ctx := context.Background() n := 0 for i := 0; i < b.N; i++ { - ret, _ := InvokeWithoutEventTime(ctx, fn, nil, n) + ret, _ := InvokeWithoutEventTime(ctx, fn, nil, nil, n) n = ret.Elm.(int) } b.Log(n) diff --git a/sdks/go/pkg/beam/core/runtime/exec/pardo.go b/sdks/go/pkg/beam/core/runtime/exec/pardo.go index acd086745ce7..aefcd57e0ed3 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/pardo.go +++ b/sdks/go/pkg/beam/core/runtime/exec/pardo.go @@ -42,6 +42,7 @@ type ParDo struct { emitters []ReusableEmitter ctx context.Context inv *invoker + bf *bundleFinalizer reader StateReader cache *cacheElm @@ -83,7 +84,7 @@ func (n *ParDo) Up(ctx context.Context) error { // Subsequent bundles might run this same node, and the context here would be // incorrectly refering to the older bundleId. setupCtx := metrics.SetPTransformID(ctx, n.PID) - if _, err := InvokeWithoutEventTime(setupCtx, n.Fn.SetupFn(), nil); err != nil { + if _, err := InvokeWithoutEventTime(setupCtx, n.Fn.SetupFn(), nil, nil); err != nil { return n.fail(err) } @@ -229,7 +230,7 @@ func (n *ParDo) Down(ctx context.Context) error { n.reader = nil n.cache = nil - if _, err := InvokeWithoutEventTime(ctx, n.Fn.TeardownFn(), nil); err != nil { + if _, err := InvokeWithoutEventTime(ctx, n.Fn.TeardownFn(), nil, nil); err != nil { n.err.TrySetError(err) } return n.err.Error() @@ -295,7 +296,7 @@ func (n *ParDo) invokeDataFn(ctx context.Context, pn typex.PaneInfo, ws []typex. if err := n.preInvoke(ctx, ws, ts); err != nil { return nil, err } - val, err = Invoke(ctx, pn, ws, ts, fn, opt, n.cache.extra...) + val, err = Invoke(ctx, pn, ws, ts, fn, opt, n.bf, n.cache.extra...) if err != nil { return nil, err } @@ -313,7 +314,7 @@ func (n *ParDo) invokeProcessFn(ctx context.Context, pn typex.PaneInfo, ws []typ if err := n.preInvoke(ctx, ws, ts); err != nil { return nil, err } - val, err = n.inv.Invoke(ctx, pn, ws, ts, opt, n.cache.extra...) + val, err = n.inv.Invoke(ctx, pn, ws, ts, opt, n.bf, n.cache.extra...) if err != nil { return nil, err } diff --git a/sdks/go/pkg/beam/core/runtime/exec/plan.go b/sdks/go/pkg/beam/core/runtime/exec/plan.go index 7f89ce37322c..abda426f639c 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/plan.go +++ b/sdks/go/pkg/beam/core/runtime/exec/plan.go @@ -21,6 +21,7 @@ import ( "context" "fmt" "strings" + "time" "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" ) @@ -33,6 +34,7 @@ type Plan struct { roots []Root units []Unit pcols []*PCollection + bf *bundleFinalizer status Status @@ -45,6 +47,10 @@ func NewPlan(id string, units []Unit) (*Plan, error) { var roots []Root var pcols []*PCollection var source *DataSource + bf := bundleFinalizer{ + callbacks: []bundleFinalizationCallback{}, + lastValidCallback: time.Now(), + } for _, u := range units { if u == nil { @@ -59,6 +65,9 @@ func NewPlan(id string, units []Unit) (*Plan, error) { if p, ok := u.(*PCollection); ok { pcols = append(pcols, p) } + if p, ok := u.(*ParDo); ok { + p.bf = &bf + } } if len(roots) == 0 { return nil, errors.Errorf("no root units") @@ -70,6 +79,7 @@ func NewPlan(id string, units []Unit) (*Plan, error) { roots: roots, units: units, pcols: pcols, + bf: &bf, source: source, }, nil } @@ -131,6 +141,46 @@ func (p *Plan) Execute(ctx context.Context, id string, manager DataContext) erro return nil } +// Finalize runs any callbacks registered by the bundleFinalizer. Should be run on bundle finalization. +func (p *Plan) Finalize() error { + if p.status != Up { + return errors.Errorf("invalid status for plan %v: %v", p.id, p.status) + } + failedIndices := []int{} + for idx, bfc := range p.bf.callbacks { + if time.Now().Before(bfc.validUntil) { + if err := bfc.callback(); err != nil { + failedIndices = append(failedIndices, idx) + } + } + } + + newFinalizer := bundleFinalizer{ + callbacks: []bundleFinalizationCallback{}, + lastValidCallback: time.Now(), + } + + for _, idx := range failedIndices { + newFinalizer.callbacks = append(newFinalizer.callbacks, p.bf.callbacks[idx]) + if newFinalizer.lastValidCallback.Before(p.bf.callbacks[idx].validUntil) { + newFinalizer.lastValidCallback = p.bf.callbacks[idx].validUntil + } + } + + p.bf = &newFinalizer + + if len(failedIndices) > 0 { + return errors.Errorf("Plan %v failed %v callbacks", p.ID(), len(failedIndices)) + } + return nil +} + +// GetExpirationTime returns the last expiration time of any of the callbacks registered by the bundleFinalizer. +// Once we have passed this time, it is safe to move this plan to inactive without missing any valid callbacks. +func (p *Plan) GetExpirationTime() time.Time { + return p.bf.lastValidCallback +} + // Down takes the plan and associated units down. Does not panic. func (p *Plan) Down(ctx context.Context) error { if p.status == Down { diff --git a/sdks/go/pkg/beam/core/runtime/exec/util.go b/sdks/go/pkg/beam/core/runtime/exec/util.go index 2996b6dd159b..019b19070570 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/util.go +++ b/sdks/go/pkg/beam/core/runtime/exec/util.go @@ -72,7 +72,7 @@ func MultiStartBundle(ctx context.Context, id string, data DataContext, list ... return nil } -// MultiFinishBundle calls StartBundle on multiple nodes. Convenience function. +// MultiFinishBundle calls FinishBundle on multiple nodes. Convenience function. func MultiFinishBundle(ctx context.Context, list ...Node) error { for _, n := range list { if err := n.FinishBundle(ctx); err != nil { diff --git a/sdks/go/pkg/beam/core/runtime/harness/harness.go b/sdks/go/pkg/beam/core/runtime/harness/harness.go index 5b0e77883f43..5b861f15188a 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/harness.go +++ b/sdks/go/pkg/beam/core/runtime/harness/harness.go @@ -102,16 +102,17 @@ func Main(ctx context.Context, loggingEndpoint, controlEndpoint string) error { sideCache.Init(cacheSize) ctrl := &control{ - lookupDesc: lookupDesc, - descriptors: make(map[bundleDescriptorID]*fnpb.ProcessBundleDescriptor), - plans: make(map[bundleDescriptorID][]*exec.Plan), - active: make(map[instructionID]*exec.Plan), - inactive: newCircleBuffer(), - metStore: make(map[instructionID]*metrics.Store), - failed: make(map[instructionID]error), - data: &DataChannelManager{}, - state: &StateChannelManager{}, - cache: &sideCache, + lookupDesc: lookupDesc, + descriptors: make(map[bundleDescriptorID]*fnpb.ProcessBundleDescriptor), + plans: make(map[bundleDescriptorID][]*exec.Plan), + active: make(map[instructionID]*exec.Plan), + awaitingFinalization: make(map[instructionID]awaitingFinalization), + inactive: newCircleBuffer(), + metStore: make(map[instructionID]*metrics.Store), + failed: make(map[instructionID]error), + data: &DataChannelManager{}, + state: &StateChannelManager{}, + cache: &sideCache, } // gRPC requires all readers of a stream be the same goroutine, so this goroutine @@ -222,11 +223,19 @@ func (c *circleBuffer) Contains(instID instructionID) bool { return ok } +type awaitingFinalization struct { + expiration time.Time + plan *exec.Plan + bdID bundleDescriptorID +} + type control struct { lookupDesc func(bundleDescriptorID) (*fnpb.ProcessBundleDescriptor, error) descriptors map[bundleDescriptorID]*fnpb.ProcessBundleDescriptor // protected by mu // plans that are candidates for execution. plans map[bundleDescriptorID][]*exec.Plan // protected by mu + // plans that are awaiting bundle finalization. + awaitingFinalization map[instructionID]awaitingFinalization //protected by mu // plans that are actively being executed. // a plan can only be in one of these maps at any time. active map[instructionID]*exec.Plan // protected by mu @@ -338,14 +347,36 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe c.cache.CompleteBundle(tokens...) mons, pylds := monitoring(plan, store) + requiresFinalization := false // Move the plan back to the candidate state c.mu.Lock() // Mark the instruction as failed. if err != nil { c.failed[instID] = err } else { - // Non failure plans can be re-used. - c.plans[bdID] = append(c.plans[bdID], plan) + // Non failure plans should either be moved to the finalized state + // or to plans so they can be re-used. + expiration := plan.GetExpirationTime() + if time.Now().Before(expiration) { + // TODO(BEAM-10976) - we can be a little smarter about data structures here by + // by storing plans awaiting finalization in a heap. That way when we expire plans + // here its O(1) instead of O(n) (though adding/finalizing will still be O(logn)) + requiresFinalization = true + c.awaitingFinalization[instID] = awaitingFinalization{ + expiration: expiration, + plan: plan, + bdID: bdID, + } + // Move any plans that have exceeded their expiration back into the re-use pool + for id, af := range c.awaitingFinalization { + if time.Now().After(af.expiration) { + c.plans[af.bdID] = append(c.plans[af.bdID], af.plan) + delete(c.awaitingFinalization, id) + } + } + } else { + c.plans[bdID] = append(c.plans[bdID], plan) + } } delete(c.active, instID) if removed, ok := c.inactive.Insert(instID); ok { @@ -362,12 +393,38 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe InstructionId: string(instID), Response: &fnpb.InstructionResponse_ProcessBundle{ ProcessBundle: &fnpb.ProcessBundleResponse{ - MonitoringData: pylds, - MonitoringInfos: mons, + MonitoringData: pylds, + MonitoringInfos: mons, + RequiresFinalization: requiresFinalization, }, }, } + case req.GetFinalizeBundle() != nil: + msg := req.GetFinalizeBundle() + + ref := instructionID(msg.GetInstructionId()) + + af, ok := c.awaitingFinalization[ref] + if !ok { + return fail(ctx, instID, "finalize bundle failed for instruction %v: couldn't find plan in finalizing map", ref) + } + + if time.Now().Before(af.expiration) { + if err := af.plan.Finalize(); err != nil { + return fail(ctx, instID, "finalize bundle failed for instruction %v using plan %v : %v", ref, af.bdID, err) + } + } + c.plans[af.bdID] = append(c.plans[af.bdID], af.plan) + delete(c.awaitingFinalization, ref) + + return &fnpb.InstructionResponse{ + InstructionId: string(instID), + Response: &fnpb.InstructionResponse_FinalizeBundle{ + FinalizeBundle: &fnpb.FinalizeBundleResponse{}, + }, + } + case req.GetProcessBundleProgress() != nil: msg := req.GetProcessBundleProgress() @@ -506,6 +563,13 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe func (c *control) getPlanOrResponse(ctx context.Context, kind string, instID, ref instructionID) (*exec.Plan, *metrics.Store, *fnpb.InstructionResponse) { c.mu.Lock() plan, ok := c.active[ref] + if !ok { + var af awaitingFinalization + af, ok = c.awaitingFinalization[ref] + if ok { + plan = af.plan + } + } err := c.failed[ref] store := c.metStore[ref] defer c.mu.Unlock() diff --git a/sdks/go/pkg/beam/runners/direct/buffer.go b/sdks/go/pkg/beam/runners/direct/buffer.go index 901cfe929af7..8e92ff72d0b5 100644 --- a/sdks/go/pkg/beam/runners/direct/buffer.go +++ b/sdks/go/pkg/beam/runners/direct/buffer.go @@ -163,7 +163,6 @@ func (w *wait) FinishBundle(ctx context.Context) error { } w.done = true return w.next.FinishBundle(ctx) - } func (w *wait) Down(ctx context.Context) error { From ac568c7d9a2abb108c0f6c54ae1c0d77d024112b Mon Sep 17 00:00:00 2001 From: bullet03 Date: Wed, 9 Mar 2022 06:48:05 +0300 Subject: [PATCH 58/68] Merge pull request #16976 from [BEAM-14010] [Website] Add Playground section to the Home page * [BEAM-14010] [Website] Add Playground section to the Home page * Update button to "Try Playground" Co-authored-by: Aydar Zainutdinov * [BEAM-14010] [Website] change button name * [BEAM-14010] [Website] align header to center * [BEAM-14010] [Website] change link Co-authored-by: Alex Kosolapov Co-authored-by: Aydar Zainutdinov --- website/www/site/assets/scss/_global.sass | 4 +++- website/www/site/i18n/home/en.yaml | 2 ++ website/www/site/layouts/_default/baseof.html | 1 + website/www/site/layouts/index.html | 21 +++++++++++++++++++ 4 files changed, 27 insertions(+), 1 deletion(-) diff --git a/website/www/site/assets/scss/_global.sass b/website/www/site/assets/scss/_global.sass index dcbb49f5d1c9..fae897ef3b04 100644 --- a/website/www/site/assets/scss/_global.sass +++ b/website/www/site/assets/scss/_global.sass @@ -26,6 +26,9 @@ body .body background: #fff margin: 0 auto + .body__contained + h1 + text-align: center .no__padding padding: 0 &:not(.body--index) @@ -34,7 +37,6 @@ body padding: 0 padding: 0 30px max-width: 1280px - figure img width: 100% diff --git a/website/www/site/i18n/home/en.yaml b/website/www/site/i18n/home/en.yaml index 2d9157be2c1a..e44d3374edfd 100644 --- a/website/www/site/i18n/home/en.yaml +++ b/website/www/site/i18n/home/en.yaml @@ -22,6 +22,8 @@ translation: "Python Quickstart" - id: home-go-quickstart translation: "Go Quickstart" +- id: home-playground + translation: "Try Playground" - id: home-hero-blog-title translation: "The latest from the blog" - id: home-cards-title diff --git a/website/www/site/layouts/_default/baseof.html b/website/www/site/layouts/_default/baseof.html index 2a75558f5d85..56f5edc47802 100644 --- a/website/www/site/layouts/_default/baseof.html +++ b/website/www/site/layouts/_default/baseof.html @@ -21,6 +21,7 @@ {{ block "hero-section" . }}{{ end }} {{ block "ctas-section" . }}{{ end }} {{ block "pillars-section" . }}{{ end }} + {{ block "playground-section" . }}{{ end }} {{ block "graphic-section" . }}{{ end }} {{ block "calendar-section" . }}{{ end }} {{ block "quotes-section" . }}{{ end }} diff --git a/website/www/site/layouts/index.html b/website/www/site/layouts/index.html index 10a9f213d916..146a21dd2587 100644 --- a/website/www/site/layouts/index.html +++ b/website/www/site/layouts/index.html @@ -60,6 +60,24 @@

{{ end }} +{{ define "playground-section" }} +
+

Try Beam Playground

+

Beam Playground is an interactive environment to try out Beam transforms and examples without having to install Apache Beam in your environment. + You can try an Apache Beam examples at Beam Playground (Beta). +

+
+
+ +
+{{ end }} + {{ define "graphic-section" }}
{{ end }} From ed4e751c85614765778150bcc1d93796996ab365 Mon Sep 17 00:00:00 2001 From: Yichi Zhang Date: Tue, 8 Mar 2022 20:23:44 -0800 Subject: [PATCH 59/68] [BEAM-12447] Upgrade cloud build client and add/cleanup options (#17032) --- .../apache_beam/options/pipeline_options.py | 9 +- .../runners/dataflow/internal/apiclient.py | 5 + .../internal/clients/cloudbuild/__init__.py | 2 - .../cloudbuild/cloudbuild_v1_client.py | 1367 ++++++++-- .../cloudbuild/cloudbuild_v1_messages.py | 2335 +++++++++++++++-- .../portability/sdk_container_builder.py | 30 +- 6 files changed, 3387 insertions(+), 361 deletions(-) diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index 14aa54ec735c..2c12a9f6f5b6 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -1176,14 +1176,11 @@ def _add_argparse_args(cls, parser): 'can also subclass SdkContainerImageBuilder and use that to build ' 'in other environments.')) parser.add_argument( - '--prebuild_sdk_container_base_image', + '--cloud_build_machine_type', default=None, help=( - 'The base image to use when pre-building the sdk container image ' - 'with dependencies, if not specified, by default the released ' - 'public apache beam python sdk container image corresponding to ' - 'the sdk version will be used, if a dev sdk is used the base ' - 'image will default to the latest released sdk image.')) + 'If specified, use the machine type explicitly when prebuilding' + 'SDK container image on Google Cloud Build.')) parser.add_argument( '--docker_registry_push_url', default=None, diff --git a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py index bd6af8c8958b..9a9a4f9692e7 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py @@ -20,6 +20,11 @@ Dataflow client utility functions.""" # pytype: skip-file +# To regenerate the client: +# pip install google-apitools[cli] +# gen_client --discovery_url=cloudbuild.v1 --overwrite \ +# --outdir=apache_beam/runners/dataflow/internal/clients/cloudbuild \ +# --root_package=. client import codecs from functools import partial diff --git a/sdks/python/apache_beam/runners/dataflow/internal/clients/cloudbuild/__init__.py b/sdks/python/apache_beam/runners/dataflow/internal/clients/cloudbuild/__init__.py index 35017d88b02d..767bd4cec605 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/clients/cloudbuild/__init__.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/clients/cloudbuild/__init__.py @@ -25,11 +25,9 @@ # pylint: disable=wrong-import-order, wrong-import-position try: from apitools.base.py import * - from apache_beam.runners.dataflow.internal.clients.cloudbuild.cloudbuild_v1_client import * from apache_beam.runners.dataflow.internal.clients.cloudbuild.cloudbuild_v1_messages import * except ImportError: pass -# pylint: enable=wrong-import-order, wrong-import-position __path__ = pkgutil.extend_path(__path__, __name__) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/clients/cloudbuild/cloudbuild_v1_client.py b/sdks/python/apache_beam/runners/dataflow/internal/clients/cloudbuild/cloudbuild_v1_client.py index fe60374480e2..9d699aba5892 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/clients/cloudbuild/cloudbuild_v1_client.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/clients/cloudbuild/cloudbuild_v1_client.py @@ -17,13 +17,13 @@ """Generated client library for cloudbuild version v1.""" # NOTE: This file is autogenerated and should not be edited by hand. +# mypy: ignore-errors # To regenerate the client: # pip install google-apitools[cli] # gen_client --discovery_url=cloudbuild.v1 --overwrite \ # --outdir=apache_beam/runners/dataflow/internal/clients/cloudbuild \ # --root_package=. client -# -# mypy: ignore-errors + from apitools.base.py import base_api from . import cloudbuild_v1_messages as messages @@ -73,14 +73,65 @@ def __init__( default_global_params=default_global_params, additional_http_headers=additional_http_headers, response_encoding=response_encoding) + self.locations = self.LocationsService(self) self.operations = self.OperationsService(self) self.projects_builds = self.ProjectsBuildsService(self) + self.projects_githubEnterpriseConfigs = self.ProjectsGithubEnterpriseConfigsService( + self) + self.projects_locations_bitbucketServerConfigs_connectedRepositories = self.ProjectsLocationsBitbucketServerConfigsConnectedRepositoriesService( + self) + self.projects_locations_bitbucketServerConfigs_repos = self.ProjectsLocationsBitbucketServerConfigsReposService( + self) + self.projects_locations_bitbucketServerConfigs = self.ProjectsLocationsBitbucketServerConfigsService( + self) self.projects_locations_builds = self.ProjectsLocationsBuildsService(self) + self.projects_locations_githubEnterpriseConfigs = self.ProjectsLocationsGithubEnterpriseConfigsService( + self) self.projects_locations_operations = self.ProjectsLocationsOperationsService( self) + self.projects_locations_triggers = self.ProjectsLocationsTriggersService( + self) + self.projects_locations_workerPools = self.ProjectsLocationsWorkerPoolsService( + self) self.projects_locations = self.ProjectsLocationsService(self) self.projects_triggers = self.ProjectsTriggersService(self) self.projects = self.ProjectsService(self) + self.v1 = self.V1Service(self) + + class LocationsService(base_api.BaseApiService): + """Service class for the locations resource.""" + + _NAME = 'locations' + + def __init__(self, client): + super(CloudbuildV1.LocationsService, self).__init__(client) + self._upload_configs = {} + + def RegionalWebhook(self, request, global_params=None): + r"""ReceiveRegionalWebhook is called when the API receives a regional GitHub webhook. + + Args: + request: (CloudbuildLocationsRegionalWebhookRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Empty) The response message. + """ + config = self.GetMethodConfig('RegionalWebhook') + return self._RunMethod(config, request, global_params=global_params) + + RegionalWebhook.method_config = lambda: base_api.ApiMethodInfo( + flat_path='v1/locations/{locationsId}/regionalWebhook', + http_method='POST', + method_id='cloudbuild.locations.regionalWebhook', + ordered_params=['location'], + path_params=['location'], + query_params=['webhookKey'], + relative_path='v1/{+location}/regionalWebhook', + request_field='httpBody', + request_type_name='CloudbuildLocationsRegionalWebhookRequest', + response_type_name='Empty', + supports_download=False, + ) class OperationsService(base_api.BaseApiService): """Service class for the operations resource.""" @@ -152,6 +203,32 @@ def __init__(self, client): super(CloudbuildV1.ProjectsBuildsService, self).__init__(client) self._upload_configs = {} + def Approve(self, request, global_params=None): + r"""Approves or rejects a pending build. If approved, the returned LRO will be analogous to the LRO returned from a CreateBuild call. If rejected, the returned LRO will be immediately done. + + Args: + request: (CloudbuildProjectsBuildsApproveRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Operation) The response message. + """ + config = self.GetMethodConfig('Approve') + return self._RunMethod(config, request, global_params=global_params) + + Approve.method_config = lambda: base_api.ApiMethodInfo( + flat_path='v1/projects/{projectsId}/builds/{buildsId}:approve', + http_method='POST', + method_id='cloudbuild.projects.builds.approve', + ordered_params=['name'], + path_params=['name'], + query_params=[], + relative_path='v1/{+name}:approve', + request_field='approveBuildRequest', + request_type_name='CloudbuildProjectsBuildsApproveRequest', + response_type_name='Operation', + supports_download=False, + ) + def Cancel(self, request, global_params=None): r"""Cancels a build in progress. @@ -277,326 +354,1288 @@ def Retry(self, request, global_params=None): supports_download=False, ) - class ProjectsLocationsBuildsService(base_api.BaseApiService): - """Service class for the projects_locations_builds resource.""" + class ProjectsGithubEnterpriseConfigsService(base_api.BaseApiService): + """Service class for the projects_githubEnterpriseConfigs resource.""" - _NAME = 'projects_locations_builds' + _NAME = 'projects_githubEnterpriseConfigs' def __init__(self, client): - super(CloudbuildV1.ProjectsLocationsBuildsService, self).__init__(client) + super(CloudbuildV1.ProjectsGithubEnterpriseConfigsService, + self).__init__(client) self._upload_configs = {} - def Cancel(self, request, global_params=None): - r"""Cancels a build in progress. + def Create(self, request, global_params=None): + r"""Create an association between a GCP project and a GitHub Enterprise server. Args: - request: (CancelBuildRequest) input message + request: (CloudbuildProjectsGithubEnterpriseConfigsCreateRequest) input message global_params: (StandardQueryParameters, default: None) global arguments Returns: - (Build) The response message. + (Operation) The response message. """ - config = self.GetMethodConfig('Cancel') + config = self.GetMethodConfig('Create') return self._RunMethod(config, request, global_params=global_params) - Cancel.method_config = lambda: base_api.ApiMethodInfo( - flat_path= - 'v1/projects/{projectsId}/locations/{locationsId}/builds/{buildsId}:cancel', + Create.method_config = lambda: base_api.ApiMethodInfo( + flat_path='v1/projects/{projectsId}/githubEnterpriseConfigs', http_method='POST', - method_id='cloudbuild.projects.locations.builds.cancel', - ordered_params=['name'], - path_params=['name'], - query_params=[], - relative_path='v1/{+name}:cancel', - request_field='', - request_type_name='CancelBuildRequest', - response_type_name='Build', + method_id='cloudbuild.projects.githubEnterpriseConfigs.create', + ordered_params=['parent'], + path_params=['parent'], + query_params=['gheConfigId', 'projectId'], + relative_path='v1/{+parent}/githubEnterpriseConfigs', + request_field='gitHubEnterpriseConfig', + request_type_name= + 'CloudbuildProjectsGithubEnterpriseConfigsCreateRequest', + response_type_name='Operation', supports_download=False, ) - def Create(self, request, global_params=None): - r"""Starts a build with the specified configuration. This method returns a long-running `Operation`, which includes the build ID. Pass the build ID to `GetBuild` to determine the build status (such as `SUCCESS` or `FAILURE`). + def Delete(self, request, global_params=None): + r"""Delete an association between a GCP project and a GitHub Enterprise server. Args: - request: (CloudbuildProjectsLocationsBuildsCreateRequest) input message + request: (CloudbuildProjectsGithubEnterpriseConfigsDeleteRequest) input message global_params: (StandardQueryParameters, default: None) global arguments Returns: (Operation) The response message. """ - config = self.GetMethodConfig('Create') + config = self.GetMethodConfig('Delete') return self._RunMethod(config, request, global_params=global_params) - Create.method_config = lambda: base_api.ApiMethodInfo( - flat_path='v1/projects/{projectsId}/locations/{locationsId}/builds', - http_method='POST', - method_id='cloudbuild.projects.locations.builds.create', - ordered_params=['parent'], - path_params=['parent'], - query_params=['projectId'], - relative_path='v1/{+parent}/builds', - request_field='build', - request_type_name='CloudbuildProjectsLocationsBuildsCreateRequest', + Delete.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/githubEnterpriseConfigs/{githubEnterpriseConfigsId}', + http_method='DELETE', + method_id='cloudbuild.projects.githubEnterpriseConfigs.delete', + ordered_params=['name'], + path_params=['name'], + query_params=['configId', 'projectId'], + relative_path='v1/{+name}', + request_field='', + request_type_name= + 'CloudbuildProjectsGithubEnterpriseConfigsDeleteRequest', response_type_name='Operation', supports_download=False, ) def Get(self, request, global_params=None): - r"""Returns information about a previously requested build. The `Build` that is returned includes its status (such as `SUCCESS`, `FAILURE`, or `WORKING`), and timing information. + r"""Retrieve a GitHubEnterpriseConfig. Args: - request: (CloudbuildProjectsLocationsBuildsGetRequest) input message + request: (CloudbuildProjectsGithubEnterpriseConfigsGetRequest) input message global_params: (StandardQueryParameters, default: None) global arguments Returns: - (Build) The response message. + (GitHubEnterpriseConfig) The response message. """ config = self.GetMethodConfig('Get') return self._RunMethod(config, request, global_params=global_params) Get.method_config = lambda: base_api.ApiMethodInfo( flat_path= - 'v1/projects/{projectsId}/locations/{locationsId}/builds/{buildsId}', + 'v1/projects/{projectsId}/githubEnterpriseConfigs/{githubEnterpriseConfigsId}', http_method='GET', - method_id='cloudbuild.projects.locations.builds.get', + method_id='cloudbuild.projects.githubEnterpriseConfigs.get', ordered_params=['name'], path_params=['name'], - query_params=['id', 'projectId'], + query_params=['configId', 'projectId'], relative_path='v1/{+name}', request_field='', - request_type_name='CloudbuildProjectsLocationsBuildsGetRequest', - response_type_name='Build', + request_type_name='CloudbuildProjectsGithubEnterpriseConfigsGetRequest', + response_type_name='GitHubEnterpriseConfig', supports_download=False, ) def List(self, request, global_params=None): - r"""Lists previously requested builds. Previously requested builds may still be in-progress, or may have finished successfully or unsuccessfully. + r"""List all GitHubEnterpriseConfigs for a given project. Args: - request: (CloudbuildProjectsLocationsBuildsListRequest) input message + request: (CloudbuildProjectsGithubEnterpriseConfigsListRequest) input message global_params: (StandardQueryParameters, default: None) global arguments Returns: - (ListBuildsResponse) The response message. + (ListGithubEnterpriseConfigsResponse) The response message. """ config = self.GetMethodConfig('List') return self._RunMethod(config, request, global_params=global_params) List.method_config = lambda: base_api.ApiMethodInfo( - flat_path='v1/projects/{projectsId}/locations/{locationsId}/builds', + flat_path='v1/projects/{projectsId}/githubEnterpriseConfigs', http_method='GET', - method_id='cloudbuild.projects.locations.builds.list', + method_id='cloudbuild.projects.githubEnterpriseConfigs.list', ordered_params=['parent'], path_params=['parent'], - query_params=['filter', 'pageSize', 'pageToken', 'projectId'], - relative_path='v1/{+parent}/builds', + query_params=['projectId'], + relative_path='v1/{+parent}/githubEnterpriseConfigs', request_field='', - request_type_name='CloudbuildProjectsLocationsBuildsListRequest', - response_type_name='ListBuildsResponse', + request_type_name= + 'CloudbuildProjectsGithubEnterpriseConfigsListRequest', + response_type_name='ListGithubEnterpriseConfigsResponse', supports_download=False, ) - def Retry(self, request, global_params=None): - r"""Creates a new build based on the specified build. This method creates a new build using the original build request, which may or may not result in an identical build. For triggered builds: * Triggered builds resolve to a precise revision; therefore a retry of a triggered build will result in a build that uses the same revision. For non-triggered builds that specify `RepoSource`: * If the original build built from the tip of a branch, the retried build will build from the tip of that branch, which may not be the same revision as the original build. * If the original build specified a commit sha or revision ID, the retried build will use the identical source. For builds that specify `StorageSource`: * If the original build pulled source from Google Cloud Storage without specifying the generation of the object, the new build will use the current object, which may be different from the original build source. * If the original build pulled source from Cloud Storage and specified the generation of the object, the new build will attempt to use the same object, which may or may not be available depending on the bucket's lifecycle management settings. + def Patch(self, request, global_params=None): + r"""Update an association between a GCP project and a GitHub Enterprise server. Args: - request: (RetryBuildRequest) input message + request: (CloudbuildProjectsGithubEnterpriseConfigsPatchRequest) input message global_params: (StandardQueryParameters, default: None) global arguments Returns: (Operation) The response message. """ - config = self.GetMethodConfig('Retry') + config = self.GetMethodConfig('Patch') return self._RunMethod(config, request, global_params=global_params) - Retry.method_config = lambda: base_api.ApiMethodInfo( + Patch.method_config = lambda: base_api.ApiMethodInfo( flat_path= - 'v1/projects/{projectsId}/locations/{locationsId}/builds/{buildsId}:retry', - http_method='POST', - method_id='cloudbuild.projects.locations.builds.retry', + 'v1/projects/{projectsId}/githubEnterpriseConfigs/{githubEnterpriseConfigsId}', + http_method='PATCH', + method_id='cloudbuild.projects.githubEnterpriseConfigs.patch', ordered_params=['name'], path_params=['name'], - query_params=[], - relative_path='v1/{+name}:retry', - request_field='', - request_type_name='RetryBuildRequest', + query_params=['updateMask'], + relative_path='v1/{+name}', + request_field='gitHubEnterpriseConfig', + request_type_name= + 'CloudbuildProjectsGithubEnterpriseConfigsPatchRequest', response_type_name='Operation', supports_download=False, ) - class ProjectsLocationsOperationsService(base_api.BaseApiService): - """Service class for the projects_locations_operations resource.""" + class ProjectsLocationsBitbucketServerConfigsConnectedRepositoriesService( + base_api.BaseApiService): + """Service class for the projects_locations_bitbucketServerConfigs_connectedRepositories resource.""" - _NAME = 'projects_locations_operations' + _NAME = 'projects_locations_bitbucketServerConfigs_connectedRepositories' def __init__(self, client): - super(CloudbuildV1.ProjectsLocationsOperationsService, - self).__init__(client) + super( + CloudbuildV1. + ProjectsLocationsBitbucketServerConfigsConnectedRepositoriesService, + self).__init__(client) self._upload_configs = {} - def Cancel(self, request, global_params=None): - r"""Starts asynchronous cancellation on a long-running operation. The server makes a best effort to cancel the operation, but success is not guaranteed. If the server doesn't support this method, it returns `google.rpc.Code.UNIMPLEMENTED`. Clients can use Operations.GetOperation or other methods to check whether the cancellation succeeded or whether the operation completed despite cancellation. On successful cancellation, the operation is not deleted; instead, it becomes an operation with an Operation.error value with a google.rpc.Status.code of 1, corresponding to `Code.CANCELLED`. + def BatchCreate(self, request, global_params=None): + r"""Batch connecting Bitbucket Server repositories to Cloud Build. Args: - request: (CloudbuildProjectsLocationsOperationsCancelRequest) input message + request: (CloudbuildProjectsLocationsBitbucketServerConfigsConnectedRepositoriesBatchCreateRequest) input message global_params: (StandardQueryParameters, default: None) global arguments Returns: - (Empty) The response message. + (Operation) The response message. """ - config = self.GetMethodConfig('Cancel') + config = self.GetMethodConfig('BatchCreate') return self._RunMethod(config, request, global_params=global_params) - Cancel.method_config = lambda: base_api.ApiMethodInfo( + BatchCreate.method_config = lambda: base_api.ApiMethodInfo( flat_path= - 'v1/projects/{projectsId}/locations/{locationsId}/operations/{operationsId}:cancel', + 'v1/projects/{projectsId}/locations/{locationsId}/bitbucketServerConfigs/{bitbucketServerConfigsId}/connectedRepositories:batchCreate', http_method='POST', - method_id='cloudbuild.projects.locations.operations.cancel', - ordered_params=['name'], - path_params=['name'], + method_id= + 'cloudbuild.projects.locations.bitbucketServerConfigs.connectedRepositories.batchCreate', + ordered_params=['parent'], + path_params=['parent'], query_params=[], - relative_path='v1/{+name}:cancel', - request_field='cancelOperationRequest', - request_type_name='CloudbuildProjectsLocationsOperationsCancelRequest', - response_type_name='Empty', + relative_path='v1/{+parent}/connectedRepositories:batchCreate', + request_field='batchCreateBitbucketServerConnectedRepositoriesRequest', + request_type_name= + 'CloudbuildProjectsLocationsBitbucketServerConfigsConnectedRepositoriesBatchCreateRequest', + response_type_name='Operation', supports_download=False, ) - def Get(self, request, global_params=None): - r"""Gets the latest state of a long-running operation. Clients can use this method to poll the operation result at intervals as recommended by the API service. + class ProjectsLocationsBitbucketServerConfigsReposService( + base_api.BaseApiService): + """Service class for the projects_locations_bitbucketServerConfigs_repos resource.""" + + _NAME = 'projects_locations_bitbucketServerConfigs_repos' + + def __init__(self, client): + super( + CloudbuildV1.ProjectsLocationsBitbucketServerConfigsReposService, + self).__init__(client) + self._upload_configs = {} + + def List(self, request, global_params=None): + r"""List all repositories for a given `BitbucketServerConfig`. This API is experimental. Args: - request: (CloudbuildProjectsLocationsOperationsGetRequest) input message + request: (CloudbuildProjectsLocationsBitbucketServerConfigsReposListRequest) input message global_params: (StandardQueryParameters, default: None) global arguments Returns: - (Operation) The response message. + (ListBitbucketServerRepositoriesResponse) The response message. """ - config = self.GetMethodConfig('Get') + config = self.GetMethodConfig('List') return self._RunMethod(config, request, global_params=global_params) - Get.method_config = lambda: base_api.ApiMethodInfo( + List.method_config = lambda: base_api.ApiMethodInfo( flat_path= - 'v1/projects/{projectsId}/locations/{locationsId}/operations/{operationsId}', + 'v1/projects/{projectsId}/locations/{locationsId}/bitbucketServerConfigs/{bitbucketServerConfigsId}/repos', http_method='GET', - method_id='cloudbuild.projects.locations.operations.get', - ordered_params=['name'], - path_params=['name'], - query_params=[], - relative_path='v1/{+name}', + method_id= + 'cloudbuild.projects.locations.bitbucketServerConfigs.repos.list', + ordered_params=['parent'], + path_params=['parent'], + query_params=['pageSize', 'pageToken'], + relative_path='v1/{+parent}/repos', request_field='', - request_type_name='CloudbuildProjectsLocationsOperationsGetRequest', - response_type_name='Operation', + request_type_name= + 'CloudbuildProjectsLocationsBitbucketServerConfigsReposListRequest', + response_type_name='ListBitbucketServerRepositoriesResponse', supports_download=False, ) - class ProjectsLocationsService(base_api.BaseApiService): - """Service class for the projects_locations resource.""" + class ProjectsLocationsBitbucketServerConfigsService(base_api.BaseApiService): + """Service class for the projects_locations_bitbucketServerConfigs resource.""" - _NAME = 'projects_locations' + _NAME = 'projects_locations_bitbucketServerConfigs' def __init__(self, client): - super(CloudbuildV1.ProjectsLocationsService, self).__init__(client) + super(CloudbuildV1.ProjectsLocationsBitbucketServerConfigsService, + self).__init__(client) self._upload_configs = {} - class ProjectsTriggersService(base_api.BaseApiService): - """Service class for the projects_triggers resource.""" + def AddBitbucketServerConnectedRepository( + self, request, global_params=None): + r"""Add a Bitbucket Server repository to a given BitbucketServerConfig's connected repositories. This API is experimental. - _NAME = 'projects_triggers' + Args: + request: (CloudbuildProjectsLocationsBitbucketServerConfigsAddBitbucketServerConnectedRepositoryRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (AddBitbucketServerConnectedRepositoryResponse) The response message. + """ + config = self.GetMethodConfig('AddBitbucketServerConnectedRepository') + return self._RunMethod(config, request, global_params=global_params) - def __init__(self, client): - super(CloudbuildV1.ProjectsTriggersService, self).__init__(client) - self._upload_configs = {} + AddBitbucketServerConnectedRepository.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/bitbucketServerConfigs/{bitbucketServerConfigsId}:addBitbucketServerConnectedRepository', + http_method='POST', + method_id= + 'cloudbuild.projects.locations.bitbucketServerConfigs.addBitbucketServerConnectedRepository', + ordered_params=['config'], + path_params=['config'], + query_params=[], + relative_path='v1/{+config}:addBitbucketServerConnectedRepository', + request_field='addBitbucketServerConnectedRepositoryRequest', + request_type_name= + 'CloudbuildProjectsLocationsBitbucketServerConfigsAddBitbucketServerConnectedRepositoryRequest', + response_type_name='AddBitbucketServerConnectedRepositoryResponse', + supports_download=False, + ) def Create(self, request, global_params=None): - r"""Creates a new `BuildTrigger`. This API is experimental. + r"""Creates a new `BitbucketServerConfig`. This API is experimental. Args: - request: (CloudbuildProjectsTriggersCreateRequest) input message + request: (CloudbuildProjectsLocationsBitbucketServerConfigsCreateRequest) input message global_params: (StandardQueryParameters, default: None) global arguments Returns: - (BuildTrigger) The response message. + (Operation) The response message. """ config = self.GetMethodConfig('Create') return self._RunMethod(config, request, global_params=global_params) Create.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/bitbucketServerConfigs', http_method='POST', - method_id='cloudbuild.projects.triggers.create', - ordered_params=['projectId'], - path_params=['projectId'], - query_params=[], - relative_path='v1/projects/{projectId}/triggers', - request_field='buildTrigger', - request_type_name='CloudbuildProjectsTriggersCreateRequest', - response_type_name='BuildTrigger', + method_id='cloudbuild.projects.locations.bitbucketServerConfigs.create', + ordered_params=['parent'], + path_params=['parent'], + query_params=['bitbucketServerConfigId'], + relative_path='v1/{+parent}/bitbucketServerConfigs', + request_field='bitbucketServerConfig', + request_type_name= + 'CloudbuildProjectsLocationsBitbucketServerConfigsCreateRequest', + response_type_name='Operation', supports_download=False, ) def Delete(self, request, global_params=None): - r"""Deletes a `BuildTrigger` by its project ID and trigger ID. This API is experimental. + r"""Delete a `BitbucketServerConfig`. This API is experimental. Args: - request: (CloudbuildProjectsTriggersDeleteRequest) input message + request: (CloudbuildProjectsLocationsBitbucketServerConfigsDeleteRequest) input message global_params: (StandardQueryParameters, default: None) global arguments Returns: - (Empty) The response message. + (Operation) The response message. """ config = self.GetMethodConfig('Delete') return self._RunMethod(config, request, global_params=global_params) Delete.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/bitbucketServerConfigs/{bitbucketServerConfigsId}', http_method='DELETE', - method_id='cloudbuild.projects.triggers.delete', - ordered_params=['projectId', 'triggerId'], - path_params=['projectId', 'triggerId'], + method_id='cloudbuild.projects.locations.bitbucketServerConfigs.delete', + ordered_params=['name'], + path_params=['name'], query_params=[], - relative_path='v1/projects/{projectId}/triggers/{triggerId}', + relative_path='v1/{+name}', request_field='', - request_type_name='CloudbuildProjectsTriggersDeleteRequest', - response_type_name='Empty', + request_type_name= + 'CloudbuildProjectsLocationsBitbucketServerConfigsDeleteRequest', + response_type_name='Operation', supports_download=False, ) def Get(self, request, global_params=None): - r"""Returns information about a `BuildTrigger`. This API is experimental. + r"""Retrieve a `BitbucketServerConfig`. This API is experimental. Args: - request: (CloudbuildProjectsTriggersGetRequest) input message + request: (CloudbuildProjectsLocationsBitbucketServerConfigsGetRequest) input message global_params: (StandardQueryParameters, default: None) global arguments Returns: - (BuildTrigger) The response message. + (BitbucketServerConfig) The response message. """ config = self.GetMethodConfig('Get') return self._RunMethod(config, request, global_params=global_params) Get.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/bitbucketServerConfigs/{bitbucketServerConfigsId}', http_method='GET', - method_id='cloudbuild.projects.triggers.get', - ordered_params=['projectId', 'triggerId'], - path_params=['projectId', 'triggerId'], + method_id='cloudbuild.projects.locations.bitbucketServerConfigs.get', + ordered_params=['name'], + path_params=['name'], query_params=[], - relative_path='v1/projects/{projectId}/triggers/{triggerId}', + relative_path='v1/{+name}', request_field='', - request_type_name='CloudbuildProjectsTriggersGetRequest', - response_type_name='BuildTrigger', + request_type_name= + 'CloudbuildProjectsLocationsBitbucketServerConfigsGetRequest', + response_type_name='BitbucketServerConfig', supports_download=False, ) def List(self, request, global_params=None): - r"""Lists existing `BuildTrigger`s. This API is experimental. + r"""List all `BitbucketServerConfigs` for a given project. This API is experimental. Args: - request: (CloudbuildProjectsTriggersListRequest) input message + request: (CloudbuildProjectsLocationsBitbucketServerConfigsListRequest) input message global_params: (StandardQueryParameters, default: None) global arguments Returns: - (ListBuildTriggersResponse) The response message. + (ListBitbucketServerConfigsResponse) The response message. """ config = self.GetMethodConfig('List') return self._RunMethod(config, request, global_params=global_params) List.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/bitbucketServerConfigs', http_method='GET', - method_id='cloudbuild.projects.triggers.list', - ordered_params=['projectId'], - path_params=['projectId'], + method_id='cloudbuild.projects.locations.bitbucketServerConfigs.list', + ordered_params=['parent'], + path_params=['parent'], query_params=['pageSize', 'pageToken'], - relative_path='v1/projects/{projectId}/triggers', + relative_path='v1/{+parent}/bitbucketServerConfigs', request_field='', - request_type_name='CloudbuildProjectsTriggersListRequest', + request_type_name= + 'CloudbuildProjectsLocationsBitbucketServerConfigsListRequest', + response_type_name='ListBitbucketServerConfigsResponse', + supports_download=False, + ) + + def Patch(self, request, global_params=None): + r"""Updates an existing `BitbucketServerConfig`. This API is experimental. + + Args: + request: (CloudbuildProjectsLocationsBitbucketServerConfigsPatchRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Operation) The response message. + """ + config = self.GetMethodConfig('Patch') + return self._RunMethod(config, request, global_params=global_params) + + Patch.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/bitbucketServerConfigs/{bitbucketServerConfigsId}', + http_method='PATCH', + method_id='cloudbuild.projects.locations.bitbucketServerConfigs.patch', + ordered_params=['name'], + path_params=['name'], + query_params=['updateMask'], + relative_path='v1/{+name}', + request_field='bitbucketServerConfig', + request_type_name= + 'CloudbuildProjectsLocationsBitbucketServerConfigsPatchRequest', + response_type_name='Operation', + supports_download=False, + ) + + def RemoveBitbucketServerConnectedRepository( + self, request, global_params=None): + r"""Remove a Bitbucket Server repository from an given BitbucketServerConfig's connected repositories. This API is experimental. + + Args: + request: (CloudbuildProjectsLocationsBitbucketServerConfigsRemoveBitbucketServerConnectedRepositoryRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Empty) The response message. + """ + config = self.GetMethodConfig('RemoveBitbucketServerConnectedRepository') + return self._RunMethod(config, request, global_params=global_params) + + RemoveBitbucketServerConnectedRepository.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/bitbucketServerConfigs/{bitbucketServerConfigsId}:removeBitbucketServerConnectedRepository', + http_method='POST', + method_id= + 'cloudbuild.projects.locations.bitbucketServerConfigs.removeBitbucketServerConnectedRepository', + ordered_params=['config'], + path_params=['config'], + query_params=[], + relative_path='v1/{+config}:removeBitbucketServerConnectedRepository', + request_field='removeBitbucketServerConnectedRepositoryRequest', + request_type_name= + 'CloudbuildProjectsLocationsBitbucketServerConfigsRemoveBitbucketServerConnectedRepositoryRequest', + response_type_name='Empty', + supports_download=False, + ) + + class ProjectsLocationsBuildsService(base_api.BaseApiService): + """Service class for the projects_locations_builds resource.""" + + _NAME = 'projects_locations_builds' + + def __init__(self, client): + super(CloudbuildV1.ProjectsLocationsBuildsService, self).__init__(client) + self._upload_configs = {} + + def Approve(self, request, global_params=None): + r"""Approves or rejects a pending build. If approved, the returned LRO will be analogous to the LRO returned from a CreateBuild call. If rejected, the returned LRO will be immediately done. + + Args: + request: (CloudbuildProjectsLocationsBuildsApproveRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Operation) The response message. + """ + config = self.GetMethodConfig('Approve') + return self._RunMethod(config, request, global_params=global_params) + + Approve.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/builds/{buildsId}:approve', + http_method='POST', + method_id='cloudbuild.projects.locations.builds.approve', + ordered_params=['name'], + path_params=['name'], + query_params=[], + relative_path='v1/{+name}:approve', + request_field='approveBuildRequest', + request_type_name='CloudbuildProjectsLocationsBuildsApproveRequest', + response_type_name='Operation', + supports_download=False, + ) + + def Cancel(self, request, global_params=None): + r"""Cancels a build in progress. + + Args: + request: (CancelBuildRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Build) The response message. + """ + config = self.GetMethodConfig('Cancel') + return self._RunMethod(config, request, global_params=global_params) + + Cancel.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/builds/{buildsId}:cancel', + http_method='POST', + method_id='cloudbuild.projects.locations.builds.cancel', + ordered_params=['name'], + path_params=['name'], + query_params=[], + relative_path='v1/{+name}:cancel', + request_field='', + request_type_name='CancelBuildRequest', + response_type_name='Build', + supports_download=False, + ) + + def Create(self, request, global_params=None): + r"""Starts a build with the specified configuration. This method returns a long-running `Operation`, which includes the build ID. Pass the build ID to `GetBuild` to determine the build status (such as `SUCCESS` or `FAILURE`). + + Args: + request: (CloudbuildProjectsLocationsBuildsCreateRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Operation) The response message. + """ + config = self.GetMethodConfig('Create') + return self._RunMethod(config, request, global_params=global_params) + + Create.method_config = lambda: base_api.ApiMethodInfo( + flat_path='v1/projects/{projectsId}/locations/{locationsId}/builds', + http_method='POST', + method_id='cloudbuild.projects.locations.builds.create', + ordered_params=['parent'], + path_params=['parent'], + query_params=['projectId'], + relative_path='v1/{+parent}/builds', + request_field='build', + request_type_name='CloudbuildProjectsLocationsBuildsCreateRequest', + response_type_name='Operation', + supports_download=False, + ) + + def Get(self, request, global_params=None): + r"""Returns information about a previously requested build. The `Build` that is returned includes its status (such as `SUCCESS`, `FAILURE`, or `WORKING`), and timing information. + + Args: + request: (CloudbuildProjectsLocationsBuildsGetRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Build) The response message. + """ + config = self.GetMethodConfig('Get') + return self._RunMethod(config, request, global_params=global_params) + + Get.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/builds/{buildsId}', + http_method='GET', + method_id='cloudbuild.projects.locations.builds.get', + ordered_params=['name'], + path_params=['name'], + query_params=['id', 'projectId'], + relative_path='v1/{+name}', + request_field='', + request_type_name='CloudbuildProjectsLocationsBuildsGetRequest', + response_type_name='Build', + supports_download=False, + ) + + def List(self, request, global_params=None): + r"""Lists previously requested builds. Previously requested builds may still be in-progress, or may have finished successfully or unsuccessfully. + + Args: + request: (CloudbuildProjectsLocationsBuildsListRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (ListBuildsResponse) The response message. + """ + config = self.GetMethodConfig('List') + return self._RunMethod(config, request, global_params=global_params) + + List.method_config = lambda: base_api.ApiMethodInfo( + flat_path='v1/projects/{projectsId}/locations/{locationsId}/builds', + http_method='GET', + method_id='cloudbuild.projects.locations.builds.list', + ordered_params=['parent'], + path_params=['parent'], + query_params=['filter', 'pageSize', 'pageToken', 'projectId'], + relative_path='v1/{+parent}/builds', + request_field='', + request_type_name='CloudbuildProjectsLocationsBuildsListRequest', + response_type_name='ListBuildsResponse', + supports_download=False, + ) + + def Retry(self, request, global_params=None): + r"""Creates a new build based on the specified build. This method creates a new build using the original build request, which may or may not result in an identical build. For triggered builds: * Triggered builds resolve to a precise revision; therefore a retry of a triggered build will result in a build that uses the same revision. For non-triggered builds that specify `RepoSource`: * If the original build built from the tip of a branch, the retried build will build from the tip of that branch, which may not be the same revision as the original build. * If the original build specified a commit sha or revision ID, the retried build will use the identical source. For builds that specify `StorageSource`: * If the original build pulled source from Google Cloud Storage without specifying the generation of the object, the new build will use the current object, which may be different from the original build source. * If the original build pulled source from Cloud Storage and specified the generation of the object, the new build will attempt to use the same object, which may or may not be available depending on the bucket's lifecycle management settings. + + Args: + request: (RetryBuildRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Operation) The response message. + """ + config = self.GetMethodConfig('Retry') + return self._RunMethod(config, request, global_params=global_params) + + Retry.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/builds/{buildsId}:retry', + http_method='POST', + method_id='cloudbuild.projects.locations.builds.retry', + ordered_params=['name'], + path_params=['name'], + query_params=[], + relative_path='v1/{+name}:retry', + request_field='', + request_type_name='RetryBuildRequest', + response_type_name='Operation', + supports_download=False, + ) + + class ProjectsLocationsGithubEnterpriseConfigsService(base_api.BaseApiService + ): + """Service class for the projects_locations_githubEnterpriseConfigs resource.""" + + _NAME = 'projects_locations_githubEnterpriseConfigs' + + def __init__(self, client): + super(CloudbuildV1.ProjectsLocationsGithubEnterpriseConfigsService, + self).__init__(client) + self._upload_configs = {} + + def Create(self, request, global_params=None): + r"""Create an association between a GCP project and a GitHub Enterprise server. + + Args: + request: (CloudbuildProjectsLocationsGithubEnterpriseConfigsCreateRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Operation) The response message. + """ + config = self.GetMethodConfig('Create') + return self._RunMethod(config, request, global_params=global_params) + + Create.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/githubEnterpriseConfigs', + http_method='POST', + method_id= + 'cloudbuild.projects.locations.githubEnterpriseConfigs.create', + ordered_params=['parent'], + path_params=['parent'], + query_params=['gheConfigId', 'projectId'], + relative_path='v1/{+parent}/githubEnterpriseConfigs', + request_field='gitHubEnterpriseConfig', + request_type_name= + 'CloudbuildProjectsLocationsGithubEnterpriseConfigsCreateRequest', + response_type_name='Operation', + supports_download=False, + ) + + def Delete(self, request, global_params=None): + r"""Delete an association between a GCP project and a GitHub Enterprise server. + + Args: + request: (CloudbuildProjectsLocationsGithubEnterpriseConfigsDeleteRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Operation) The response message. + """ + config = self.GetMethodConfig('Delete') + return self._RunMethod(config, request, global_params=global_params) + + Delete.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/githubEnterpriseConfigs/{githubEnterpriseConfigsId}', + http_method='DELETE', + method_id= + 'cloudbuild.projects.locations.githubEnterpriseConfigs.delete', + ordered_params=['name'], + path_params=['name'], + query_params=['configId', 'projectId'], + relative_path='v1/{+name}', + request_field='', + request_type_name= + 'CloudbuildProjectsLocationsGithubEnterpriseConfigsDeleteRequest', + response_type_name='Operation', + supports_download=False, + ) + + def Get(self, request, global_params=None): + r"""Retrieve a GitHubEnterpriseConfig. + + Args: + request: (CloudbuildProjectsLocationsGithubEnterpriseConfigsGetRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (GitHubEnterpriseConfig) The response message. + """ + config = self.GetMethodConfig('Get') + return self._RunMethod(config, request, global_params=global_params) + + Get.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/githubEnterpriseConfigs/{githubEnterpriseConfigsId}', + http_method='GET', + method_id='cloudbuild.projects.locations.githubEnterpriseConfigs.get', + ordered_params=['name'], + path_params=['name'], + query_params=['configId', 'projectId'], + relative_path='v1/{+name}', + request_field='', + request_type_name= + 'CloudbuildProjectsLocationsGithubEnterpriseConfigsGetRequest', + response_type_name='GitHubEnterpriseConfig', + supports_download=False, + ) + + def List(self, request, global_params=None): + r"""List all GitHubEnterpriseConfigs for a given project. + + Args: + request: (CloudbuildProjectsLocationsGithubEnterpriseConfigsListRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (ListGithubEnterpriseConfigsResponse) The response message. + """ + config = self.GetMethodConfig('List') + return self._RunMethod(config, request, global_params=global_params) + + List.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/githubEnterpriseConfigs', + http_method='GET', + method_id='cloudbuild.projects.locations.githubEnterpriseConfigs.list', + ordered_params=['parent'], + path_params=['parent'], + query_params=['projectId'], + relative_path='v1/{+parent}/githubEnterpriseConfigs', + request_field='', + request_type_name= + 'CloudbuildProjectsLocationsGithubEnterpriseConfigsListRequest', + response_type_name='ListGithubEnterpriseConfigsResponse', + supports_download=False, + ) + + def Patch(self, request, global_params=None): + r"""Update an association between a GCP project and a GitHub Enterprise server. + + Args: + request: (CloudbuildProjectsLocationsGithubEnterpriseConfigsPatchRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Operation) The response message. + """ + config = self.GetMethodConfig('Patch') + return self._RunMethod(config, request, global_params=global_params) + + Patch.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/githubEnterpriseConfigs/{githubEnterpriseConfigsId}', + http_method='PATCH', + method_id='cloudbuild.projects.locations.githubEnterpriseConfigs.patch', + ordered_params=['name'], + path_params=['name'], + query_params=['updateMask'], + relative_path='v1/{+name}', + request_field='gitHubEnterpriseConfig', + request_type_name= + 'CloudbuildProjectsLocationsGithubEnterpriseConfigsPatchRequest', + response_type_name='Operation', + supports_download=False, + ) + + class ProjectsLocationsOperationsService(base_api.BaseApiService): + """Service class for the projects_locations_operations resource.""" + + _NAME = 'projects_locations_operations' + + def __init__(self, client): + super(CloudbuildV1.ProjectsLocationsOperationsService, + self).__init__(client) + self._upload_configs = {} + + def Cancel(self, request, global_params=None): + r"""Starts asynchronous cancellation on a long-running operation. The server makes a best effort to cancel the operation, but success is not guaranteed. If the server doesn't support this method, it returns `google.rpc.Code.UNIMPLEMENTED`. Clients can use Operations.GetOperation or other methods to check whether the cancellation succeeded or whether the operation completed despite cancellation. On successful cancellation, the operation is not deleted; instead, it becomes an operation with an Operation.error value with a google.rpc.Status.code of 1, corresponding to `Code.CANCELLED`. + + Args: + request: (CloudbuildProjectsLocationsOperationsCancelRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Empty) The response message. + """ + config = self.GetMethodConfig('Cancel') + return self._RunMethod(config, request, global_params=global_params) + + Cancel.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/operations/{operationsId}:cancel', + http_method='POST', + method_id='cloudbuild.projects.locations.operations.cancel', + ordered_params=['name'], + path_params=['name'], + query_params=[], + relative_path='v1/{+name}:cancel', + request_field='cancelOperationRequest', + request_type_name='CloudbuildProjectsLocationsOperationsCancelRequest', + response_type_name='Empty', + supports_download=False, + ) + + def Get(self, request, global_params=None): + r"""Gets the latest state of a long-running operation. Clients can use this method to poll the operation result at intervals as recommended by the API service. + + Args: + request: (CloudbuildProjectsLocationsOperationsGetRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Operation) The response message. + """ + config = self.GetMethodConfig('Get') + return self._RunMethod(config, request, global_params=global_params) + + Get.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/operations/{operationsId}', + http_method='GET', + method_id='cloudbuild.projects.locations.operations.get', + ordered_params=['name'], + path_params=['name'], + query_params=[], + relative_path='v1/{+name}', + request_field='', + request_type_name='CloudbuildProjectsLocationsOperationsGetRequest', + response_type_name='Operation', + supports_download=False, + ) + + class ProjectsLocationsTriggersService(base_api.BaseApiService): + """Service class for the projects_locations_triggers resource.""" + + _NAME = 'projects_locations_triggers' + + def __init__(self, client): + super(CloudbuildV1.ProjectsLocationsTriggersService, + self).__init__(client) + self._upload_configs = {} + + def Create(self, request, global_params=None): + r"""Creates a new `BuildTrigger`. This API is experimental. + + Args: + request: (CloudbuildProjectsLocationsTriggersCreateRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (BuildTrigger) The response message. + """ + config = self.GetMethodConfig('Create') + return self._RunMethod(config, request, global_params=global_params) + + Create.method_config = lambda: base_api.ApiMethodInfo( + flat_path='v1/projects/{projectsId}/locations/{locationsId}/triggers', + http_method='POST', + method_id='cloudbuild.projects.locations.triggers.create', + ordered_params=['parent'], + path_params=['parent'], + query_params=['projectId'], + relative_path='v1/{+parent}/triggers', + request_field='buildTrigger', + request_type_name='CloudbuildProjectsLocationsTriggersCreateRequest', + response_type_name='BuildTrigger', + supports_download=False, + ) + + def Delete(self, request, global_params=None): + r"""Deletes a `BuildTrigger` by its project ID and trigger ID. This API is experimental. + + Args: + request: (CloudbuildProjectsLocationsTriggersDeleteRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Empty) The response message. + """ + config = self.GetMethodConfig('Delete') + return self._RunMethod(config, request, global_params=global_params) + + Delete.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/triggers/{triggersId}', + http_method='DELETE', + method_id='cloudbuild.projects.locations.triggers.delete', + ordered_params=['name'], + path_params=['name'], + query_params=['projectId', 'triggerId'], + relative_path='v1/{+name}', + request_field='', + request_type_name='CloudbuildProjectsLocationsTriggersDeleteRequest', + response_type_name='Empty', + supports_download=False, + ) + + def Get(self, request, global_params=None): + r"""Returns information about a `BuildTrigger`. This API is experimental. + + Args: + request: (CloudbuildProjectsLocationsTriggersGetRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (BuildTrigger) The response message. + """ + config = self.GetMethodConfig('Get') + return self._RunMethod(config, request, global_params=global_params) + + Get.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/triggers/{triggersId}', + http_method='GET', + method_id='cloudbuild.projects.locations.triggers.get', + ordered_params=['name'], + path_params=['name'], + query_params=['projectId', 'triggerId'], + relative_path='v1/{+name}', + request_field='', + request_type_name='CloudbuildProjectsLocationsTriggersGetRequest', + response_type_name='BuildTrigger', + supports_download=False, + ) + + def List(self, request, global_params=None): + r"""Lists existing `BuildTrigger`s. This API is experimental. + + Args: + request: (CloudbuildProjectsLocationsTriggersListRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (ListBuildTriggersResponse) The response message. + """ + config = self.GetMethodConfig('List') + return self._RunMethod(config, request, global_params=global_params) + + List.method_config = lambda: base_api.ApiMethodInfo( + flat_path='v1/projects/{projectsId}/locations/{locationsId}/triggers', + http_method='GET', + method_id='cloudbuild.projects.locations.triggers.list', + ordered_params=['parent'], + path_params=['parent'], + query_params=['pageSize', 'pageToken', 'projectId'], + relative_path='v1/{+parent}/triggers', + request_field='', + request_type_name='CloudbuildProjectsLocationsTriggersListRequest', + response_type_name='ListBuildTriggersResponse', + supports_download=False, + ) + + def Patch(self, request, global_params=None): + r"""Updates a `BuildTrigger` by its project ID and trigger ID. This API is experimental. + + Args: + request: (CloudbuildProjectsLocationsTriggersPatchRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (BuildTrigger) The response message. + """ + config = self.GetMethodConfig('Patch') + return self._RunMethod(config, request, global_params=global_params) + + Patch.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/triggers/{triggersId}', + http_method='PATCH', + method_id='cloudbuild.projects.locations.triggers.patch', + ordered_params=['resourceName'], + path_params=['resourceName'], + query_params=['projectId', 'triggerId'], + relative_path='v1/{+resourceName}', + request_field='buildTrigger', + request_type_name='CloudbuildProjectsLocationsTriggersPatchRequest', + response_type_name='BuildTrigger', + supports_download=False, + ) + + def Run(self, request, global_params=None): + r"""Runs a `BuildTrigger` at a particular source revision. + + Args: + request: (CloudbuildProjectsLocationsTriggersRunRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Operation) The response message. + """ + config = self.GetMethodConfig('Run') + return self._RunMethod(config, request, global_params=global_params) + + Run.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/triggers/{triggersId}:run', + http_method='POST', + method_id='cloudbuild.projects.locations.triggers.run', + ordered_params=['name'], + path_params=['name'], + query_params=[], + relative_path='v1/{+name}:run', + request_field='runBuildTriggerRequest', + request_type_name='CloudbuildProjectsLocationsTriggersRunRequest', + response_type_name='Operation', + supports_download=False, + ) + + def Webhook(self, request, global_params=None): + r"""ReceiveTriggerWebhook [Experimental] is called when the API receives a webhook request targeted at a specific trigger. + + Args: + request: (CloudbuildProjectsLocationsTriggersWebhookRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (ReceiveTriggerWebhookResponse) The response message. + """ + config = self.GetMethodConfig('Webhook') + return self._RunMethod(config, request, global_params=global_params) + + Webhook.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/triggers/{triggersId}:webhook', + http_method='POST', + method_id='cloudbuild.projects.locations.triggers.webhook', + ordered_params=['name'], + path_params=['name'], + query_params=['projectId', 'secret', 'trigger'], + relative_path='v1/{+name}:webhook', + request_field='httpBody', + request_type_name='CloudbuildProjectsLocationsTriggersWebhookRequest', + response_type_name='ReceiveTriggerWebhookResponse', + supports_download=False, + ) + + class ProjectsLocationsWorkerPoolsService(base_api.BaseApiService): + """Service class for the projects_locations_workerPools resource.""" + + _NAME = 'projects_locations_workerPools' + + def __init__(self, client): + super(CloudbuildV1.ProjectsLocationsWorkerPoolsService, + self).__init__(client) + self._upload_configs = {} + + def Create(self, request, global_params=None): + r"""Creates a `WorkerPool`. + + Args: + request: (CloudbuildProjectsLocationsWorkerPoolsCreateRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Operation) The response message. + """ + config = self.GetMethodConfig('Create') + return self._RunMethod(config, request, global_params=global_params) + + Create.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/workerPools', + http_method='POST', + method_id='cloudbuild.projects.locations.workerPools.create', + ordered_params=['parent'], + path_params=['parent'], + query_params=['validateOnly', 'workerPoolId'], + relative_path='v1/{+parent}/workerPools', + request_field='workerPool', + request_type_name='CloudbuildProjectsLocationsWorkerPoolsCreateRequest', + response_type_name='Operation', + supports_download=False, + ) + + def Delete(self, request, global_params=None): + r"""Deletes a `WorkerPool`. + + Args: + request: (CloudbuildProjectsLocationsWorkerPoolsDeleteRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Operation) The response message. + """ + config = self.GetMethodConfig('Delete') + return self._RunMethod(config, request, global_params=global_params) + + Delete.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/workerPools/{workerPoolsId}', + http_method='DELETE', + method_id='cloudbuild.projects.locations.workerPools.delete', + ordered_params=['name'], + path_params=['name'], + query_params=['allowMissing', 'etag', 'validateOnly'], + relative_path='v1/{+name}', + request_field='', + request_type_name='CloudbuildProjectsLocationsWorkerPoolsDeleteRequest', + response_type_name='Operation', + supports_download=False, + ) + + def Get(self, request, global_params=None): + r"""Returns details of a `WorkerPool`. + + Args: + request: (CloudbuildProjectsLocationsWorkerPoolsGetRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (WorkerPool) The response message. + """ + config = self.GetMethodConfig('Get') + return self._RunMethod(config, request, global_params=global_params) + + Get.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/workerPools/{workerPoolsId}', + http_method='GET', + method_id='cloudbuild.projects.locations.workerPools.get', + ordered_params=['name'], + path_params=['name'], + query_params=[], + relative_path='v1/{+name}', + request_field='', + request_type_name='CloudbuildProjectsLocationsWorkerPoolsGetRequest', + response_type_name='WorkerPool', + supports_download=False, + ) + + def List(self, request, global_params=None): + r"""Lists `WorkerPool`s. + + Args: + request: (CloudbuildProjectsLocationsWorkerPoolsListRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (ListWorkerPoolsResponse) The response message. + """ + config = self.GetMethodConfig('List') + return self._RunMethod(config, request, global_params=global_params) + + List.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/workerPools', + http_method='GET', + method_id='cloudbuild.projects.locations.workerPools.list', + ordered_params=['parent'], + path_params=['parent'], + query_params=['pageSize', 'pageToken'], + relative_path='v1/{+parent}/workerPools', + request_field='', + request_type_name='CloudbuildProjectsLocationsWorkerPoolsListRequest', + response_type_name='ListWorkerPoolsResponse', + supports_download=False, + ) + + def Patch(self, request, global_params=None): + r"""Updates a `WorkerPool`. + + Args: + request: (CloudbuildProjectsLocationsWorkerPoolsPatchRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Operation) The response message. + """ + config = self.GetMethodConfig('Patch') + return self._RunMethod(config, request, global_params=global_params) + + Patch.method_config = lambda: base_api.ApiMethodInfo( + flat_path= + 'v1/projects/{projectsId}/locations/{locationsId}/workerPools/{workerPoolsId}', + http_method='PATCH', + method_id='cloudbuild.projects.locations.workerPools.patch', + ordered_params=['name'], + path_params=['name'], + query_params=['updateMask', 'validateOnly'], + relative_path='v1/{+name}', + request_field='workerPool', + request_type_name='CloudbuildProjectsLocationsWorkerPoolsPatchRequest', + response_type_name='Operation', + supports_download=False, + ) + + class ProjectsLocationsService(base_api.BaseApiService): + """Service class for the projects_locations resource.""" + + _NAME = 'projects_locations' + + def __init__(self, client): + super(CloudbuildV1.ProjectsLocationsService, self).__init__(client) + self._upload_configs = {} + + class ProjectsTriggersService(base_api.BaseApiService): + """Service class for the projects_triggers resource.""" + + _NAME = 'projects_triggers' + + def __init__(self, client): + super(CloudbuildV1.ProjectsTriggersService, self).__init__(client) + self._upload_configs = {} + + def Create(self, request, global_params=None): + r"""Creates a new `BuildTrigger`. This API is experimental. + + Args: + request: (CloudbuildProjectsTriggersCreateRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (BuildTrigger) The response message. + """ + config = self.GetMethodConfig('Create') + return self._RunMethod(config, request, global_params=global_params) + + Create.method_config = lambda: base_api.ApiMethodInfo( + http_method='POST', + method_id='cloudbuild.projects.triggers.create', + ordered_params=['projectId'], + path_params=['projectId'], + query_params=['parent'], + relative_path='v1/projects/{projectId}/triggers', + request_field='buildTrigger', + request_type_name='CloudbuildProjectsTriggersCreateRequest', + response_type_name='BuildTrigger', + supports_download=False, + ) + + def Delete(self, request, global_params=None): + r"""Deletes a `BuildTrigger` by its project ID and trigger ID. This API is experimental. + + Args: + request: (CloudbuildProjectsTriggersDeleteRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Empty) The response message. + """ + config = self.GetMethodConfig('Delete') + return self._RunMethod(config, request, global_params=global_params) + + Delete.method_config = lambda: base_api.ApiMethodInfo( + http_method='DELETE', + method_id='cloudbuild.projects.triggers.delete', + ordered_params=['projectId', 'triggerId'], + path_params=['projectId', 'triggerId'], + query_params=['name'], + relative_path='v1/projects/{projectId}/triggers/{triggerId}', + request_field='', + request_type_name='CloudbuildProjectsTriggersDeleteRequest', + response_type_name='Empty', + supports_download=False, + ) + + def Get(self, request, global_params=None): + r"""Returns information about a `BuildTrigger`. This API is experimental. + + Args: + request: (CloudbuildProjectsTriggersGetRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (BuildTrigger) The response message. + """ + config = self.GetMethodConfig('Get') + return self._RunMethod(config, request, global_params=global_params) + + Get.method_config = lambda: base_api.ApiMethodInfo( + http_method='GET', + method_id='cloudbuild.projects.triggers.get', + ordered_params=['projectId', 'triggerId'], + path_params=['projectId', 'triggerId'], + query_params=['name'], + relative_path='v1/projects/{projectId}/triggers/{triggerId}', + request_field='', + request_type_name='CloudbuildProjectsTriggersGetRequest', + response_type_name='BuildTrigger', + supports_download=False, + ) + + def List(self, request, global_params=None): + r"""Lists existing `BuildTrigger`s. This API is experimental. + + Args: + request: (CloudbuildProjectsTriggersListRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (ListBuildTriggersResponse) The response message. + """ + config = self.GetMethodConfig('List') + return self._RunMethod(config, request, global_params=global_params) + + List.method_config = lambda: base_api.ApiMethodInfo( + http_method='GET', + method_id='cloudbuild.projects.triggers.list', + ordered_params=['projectId'], + path_params=['projectId'], + query_params=['pageSize', 'pageToken', 'parent'], + relative_path='v1/projects/{projectId}/triggers', + request_field='', + request_type_name='CloudbuildProjectsTriggersListRequest', response_type_name='ListBuildTriggersResponse', supports_download=False, ) @@ -643,7 +1682,7 @@ def Run(self, request, global_params=None): method_id='cloudbuild.projects.triggers.run', ordered_params=['projectId', 'triggerId'], path_params=['projectId', 'triggerId'], - query_params=[], + query_params=['name'], relative_path='v1/projects/{projectId}/triggers/{triggerId}:run', request_field='repoSource', request_type_name='CloudbuildProjectsTriggersRunRequest', @@ -668,7 +1707,7 @@ def Webhook(self, request, global_params=None): method_id='cloudbuild.projects.triggers.webhook', ordered_params=['projectId', 'trigger'], path_params=['projectId', 'trigger'], - query_params=['secret'], + query_params=['name', 'secret'], relative_path='v1/projects/{projectId}/triggers/{trigger}:webhook', request_field='httpBody', request_type_name='CloudbuildProjectsTriggersWebhookRequest', @@ -684,3 +1723,37 @@ class ProjectsService(base_api.BaseApiService): def __init__(self, client): super(CloudbuildV1.ProjectsService, self).__init__(client) self._upload_configs = {} + + class V1Service(base_api.BaseApiService): + """Service class for the v1 resource.""" + + _NAME = 'v1' + + def __init__(self, client): + super(CloudbuildV1.V1Service, self).__init__(client) + self._upload_configs = {} + + def Webhook(self, request, global_params=None): + r"""ReceiveWebhook is called when the API receives a GitHub webhook. + + Args: + request: (CloudbuildWebhookRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Empty) The response message. + """ + config = self.GetMethodConfig('Webhook') + return self._RunMethod(config, request, global_params=global_params) + + Webhook.method_config = lambda: base_api.ApiMethodInfo( + http_method='POST', + method_id='cloudbuild.webhook', + ordered_params=[], + path_params=[], + query_params=['webhookKey'], + relative_path='v1/webhook', + request_field='httpBody', + request_type_name='CloudbuildWebhookRequest', + response_type_name='Empty', + supports_download=False, + ) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/clients/cloudbuild/cloudbuild_v1_messages.py b/sdks/python/apache_beam/runners/dataflow/internal/clients/cloudbuild/cloudbuild_v1_messages.py index 9dbef65b2137..1e3a75efcc0b 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/clients/cloudbuild/cloudbuild_v1_messages.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/clients/cloudbuild/cloudbuild_v1_messages.py @@ -21,6 +21,11 @@ """ # NOTE: This file is autogenerated and should not be edited by hand. # mypy: ignore-errors +# To regenerate the client: +# pip install google-apitools[cli] +# gen_client --discovery_url=cloudbuild.v1 --overwrite \ +# --outdir=apache_beam/runners/dataflow/internal/clients/cloudbuild \ +# --root_package=. client from apitools.base.protorpclite import messages as _messages from apitools.base.py import encoding @@ -29,6 +34,92 @@ package = 'cloudbuild' +class AddBitbucketServerConnectedRepositoryRequest(_messages.Message): + r"""RPC request object accepted by the AddBitbucketServerConnectedRepository + RPC method. + + Fields: + connectedRepository: The connected repository to add. + """ + + connectedRepository = _messages.MessageField('BitbucketServerRepositoryId', 1) + + +class AddBitbucketServerConnectedRepositoryResponse(_messages.Message): + r"""RPC request object returned by the AddBitbucketServerConnectedRepository + RPC method. + + Fields: + config: The name of the `BitbucketServerConfig` that added connected + repository. Format: `projects/{project}/locations/{location}/bitbucketSe + rverConfigs/{config}` + connectedRepository: The connected repository. + """ + + config = _messages.StringField(1) + connectedRepository = _messages.MessageField('BitbucketServerRepositoryId', 2) + + +class ApprovalConfig(_messages.Message): + r"""ApprovalConfig describes configuration for manual approval of a build. + + Fields: + approvalRequired: Whether or not approval is needed. If this is set on a + build, it will become pending when created, and will need to be + explicitly approved to start. + """ + + approvalRequired = _messages.BooleanField(1) + + +class ApprovalResult(_messages.Message): + r"""ApprovalResult describes the decision and associated metadata of a + manual approval of a build. + + Enums: + DecisionValueValuesEnum: Required. The decision of this manual approval. + + Fields: + approvalTime: Output only. The time when the approval decision was made. + approverAccount: Output only. Email of the user that called the + ApproveBuild API to approve or reject a build at the time that the API + was called. + comment: Optional. An optional comment for this manual approval result. + decision: Required. The decision of this manual approval. + url: Optional. An optional URL tied to this manual approval result. This + field is essentially the same as comment, except that it will be + rendered by the UI differently. An example use case is a link to an + external job that approved this Build. + """ + class DecisionValueValuesEnum(_messages.Enum): + r"""Required. The decision of this manual approval. + + Values: + DECISION_UNSPECIFIED: Default enum type. This should not be used. + APPROVED: Build is approved. + REJECTED: Build is rejected. + """ + DECISION_UNSPECIFIED = 0 + APPROVED = 1 + REJECTED = 2 + + approvalTime = _messages.StringField(1) + approverAccount = _messages.StringField(2) + comment = _messages.StringField(3) + decision = _messages.EnumField('DecisionValueValuesEnum', 4) + url = _messages.StringField(5) + + +class ApproveBuildRequest(_messages.Message): + r"""Request to approve or reject a pending build. + + Fields: + approvalResult: Approval decision and metadata. + """ + + approvalResult = _messages.MessageField('ApprovalResult', 1) + + class ArtifactObjects(_messages.Message): r"""Files in the workspace to upload to Cloud Storage upon successful completion of all build steps. @@ -87,19 +178,212 @@ class Artifacts(_messages.Message): objects = _messages.MessageField('ArtifactObjects', 2) +class BatchCreateBitbucketServerConnectedRepositoriesRequest(_messages.Message): + r"""RPC request object accepted by + BatchCreateBitbucketServerConnectedRepositories RPC method. + + Fields: + requests: Required. Requests to connect Bitbucket Server repositories. + """ + + requests = _messages.MessageField( + 'CreateBitbucketServerConnectedRepositoryRequest', 1, repeated=True) + + +class BatchCreateBitbucketServerConnectedRepositoriesResponse(_messages.Message + ): + r"""Response of BatchCreateBitbucketServerConnectedRepositories RPC method + including all successfully connected Bitbucket Server repositories. + + Fields: + bitbucketServerConnectedRepositories: The connected Bitbucket Server + repositories. + """ + + bitbucketServerConnectedRepositories = _messages.MessageField( + 'BitbucketServerConnectedRepository', 1, repeated=True) + + +class BatchCreateBitbucketServerConnectedRepositoriesResponseMetadata( + _messages.Message): + r"""Metadata for `BatchCreateBitbucketServerConnectedRepositories` + operation. + + Fields: + completeTime: Time the operation was completed. + config: The name of the `BitbucketServerConfig` that added connected + repositories. Format: `projects/{project}/locations/{location}/bitbucket + ServerConfigs/{config}` + createTime: Time the operation was created. + """ + + completeTime = _messages.StringField(1) + config = _messages.StringField(2) + createTime = _messages.StringField(3) + + +class BitbucketServerConfig(_messages.Message): + r"""BitbucketServerConfig represents the configuration for a Bitbucket + Server. + + Fields: + apiKey: Required. Immutable. API Key that will be attached to webhook. + Once this field has been set, it cannot be changed. If you need to + change it, please create another BitbucketServerConfig. + connectedRepositories: Output only. Connected Bitbucket Server + repositories for this config. + createTime: Time when the config was created. + hostUri: Required. Immutable. The URI of the Bitbucket Server host. Once + this field has been set, it cannot be changed. If you need to change it, + please create another BitbucketServerConfig. + name: The resource name for the config. + peeredNetwork: Optional. The network to be used when reaching out to the + Bitbucket Server instance. The VPC network must be enabled for private + service connection. This should be set if the Bitbucket Server instance + is hosted on-premises and not reachable by public internet. If this + field is left empty, no network peering will occur and calls to the + Bitbucket Server instance will be made over the public internet. Must be + in the format `projects/{project}/global/networks/{network}`, where + {project} is a project number or id and {network} is the name of a VPC + network in the project. + secrets: Required. Secret Manager secrets needed by the config. + sslCa: Optional. SSL certificate to use for requests to Bitbucket Server. + The format should be PEM format but the extension can be one of .pem, + .cer, or .crt. + username: Username of the account Cloud Build will use on Bitbucket + Server. + webhookKey: Output only. UUID included in webhook requests. The UUID is + used to look up the corresponding config. + """ + + apiKey = _messages.StringField(1) + connectedRepositories = _messages.MessageField( + 'BitbucketServerRepositoryId', 2, repeated=True) + createTime = _messages.StringField(3) + hostUri = _messages.StringField(4) + name = _messages.StringField(5) + peeredNetwork = _messages.StringField(6) + secrets = _messages.MessageField('BitbucketServerSecrets', 7) + sslCa = _messages.StringField(8) + username = _messages.StringField(9) + webhookKey = _messages.StringField(10) + + +class BitbucketServerConnectedRepository(_messages.Message): + r"""/ BitbucketServerConnectedRepository represents a connected Bitbucket + Server / repository. + + Fields: + parent: The name of the `BitbucketServerConfig` that added connected + repository. Format: `projects/{project}/locations/{location}/bitbucketSe + rverConfigs/{config}` + repo: The Bitbucket Server repositories to connect. + status: Output only. The status of the repo connection request. + """ + + parent = _messages.StringField(1) + repo = _messages.MessageField('BitbucketServerRepositoryId', 2) + status = _messages.MessageField('Status', 3) + + +class BitbucketServerRepository(_messages.Message): + r"""BitbucketServerRepository represents a repository hosted on a Bitbucket + Server. + + Fields: + browseUri: Link to the browse repo page on the Bitbucket Server instance. + description: Description of the repository. + displayName: Display name of the repository. + name: The resource name of the repository. + repoId: Identifier for a repository hosted on a Bitbucket Server. + """ + + browseUri = _messages.StringField(1) + description = _messages.StringField(2) + displayName = _messages.StringField(3) + name = _messages.StringField(4) + repoId = _messages.MessageField('BitbucketServerRepositoryId', 5) + + +class BitbucketServerRepositoryId(_messages.Message): + r"""BitbucketServerRepositoryId identifies a specific repository hosted on a + Bitbucket Server. + + Fields: + projectKey: Required. Identifier for the project storing the repository. + repoSlug: Required. Identifier for the repository. + webhookId: Output only. The ID of the webhook that was created for + receiving events from this repo. We only create and manage a single + webhook for each repo. + """ + + projectKey = _messages.StringField(1) + repoSlug = _messages.StringField(2) + webhookId = _messages.IntegerField(3, variant=_messages.Variant.INT32) + + +class BitbucketServerSecrets(_messages.Message): + r"""BitbucketServerSecrets represents the secrets in Secret Manager for a + Bitbucket Server. + + Fields: + adminAccessTokenVersionName: Required. The resource name for the admin + access token's secret version. + readAccessTokenVersionName: Required. The resource name for the read + access token's secret version. + webhookSecretVersionName: Required. Immutable. The resource name for the + webhook secret's secret version. Once this field has been set, it cannot + be changed. If you need to change it, please create another + BitbucketServerConfig. + """ + + adminAccessTokenVersionName = _messages.StringField(1) + readAccessTokenVersionName = _messages.StringField(2) + webhookSecretVersionName = _messages.StringField(3) + + +class BitbucketServerTriggerConfig(_messages.Message): + r"""BitbucketServerTriggerConfig describes the configuration of a trigger + that creates a build whenever a Bitbucket Server event is received. + + Fields: + bitbucketServerConfig: Output only. The BitbucketServerConfig specified in + the bitbucket_server_config_resource field. + bitbucketServerConfigResource: Required. The Bitbucket server config + resource that this trigger config maps to. + projectKey: Required. Key of the project that the repo is in. For example: + The key for http://mybitbucket.server/projects/TEST/repos/test-repo is + "TEST". + pullRequest: Filter to match changes in pull requests. + push: Filter to match changes in refs like branches, tags. + repoSlug: Required. Slug of the repository. A repository slug is a URL- + friendly version of a repository name, automatically generated by + Bitbucket for use in the URL. For example, if the repository name is + 'test repo', in the URL it would become 'test-repo' as in + http://mybitbucket.server/projects/TEST/repos/test-repo. + """ + + bitbucketServerConfig = _messages.MessageField('BitbucketServerConfig', 1) + bitbucketServerConfigResource = _messages.StringField(2) + projectKey = _messages.StringField(3) + pullRequest = _messages.MessageField('PullRequestFilter', 4) + push = _messages.MessageField('PushFilter', 5) + repoSlug = _messages.StringField(6) + + class Build(_messages.Message): r"""A build resource in the Cloud Build API. At a high level, a `Build` describes where to find source code, how to build it (for example, the builder image to run on the source), and where to store the built artifacts. Fields can include the following variables, which will be expanded when the build is created: - $PROJECT_ID: the project ID of the build. - - $PROJECT_NUMBER: the project number of the build. - $BUILD_ID: the - autogenerated ID of the build. - $REPO_NAME: the source repository name - specified by RepoSource. - $BRANCH_NAME: the branch name specified by - RepoSource. - $TAG_NAME: the tag name specified by RepoSource. - - $REVISION_ID or $COMMIT_SHA: the commit SHA specified by RepoSource or - resolved from the specified branch or tag. - $SHORT_SHA: first 7 characters - of $REVISION_ID or $COMMIT_SHA. + $PROJECT_NUMBER: the project number of the build. - $LOCATION: the + location/region of the build. - $BUILD_ID: the autogenerated ID of the + build. - $REPO_NAME: the source repository name specified by RepoSource. - + $BRANCH_NAME: the branch name specified by RepoSource. - $TAG_NAME: the tag + name specified by RepoSource. - $REVISION_ID or $COMMIT_SHA: the commit SHA + specified by RepoSource or resolved from the specified branch or tag. - + $SHORT_SHA: first 7 characters of $REVISION_ID or $COMMIT_SHA. Enums: StatusValueValuesEnum: Output only. Status of the build. @@ -107,12 +391,14 @@ class Build(_messages.Message): Messages: SubstitutionsValue: Substitutions data for `Build` resource. TimingValue: Output only. Stores timing information for phases of the - build. Valid keys are: * BUILD: time to execute all build steps * PUSH: + build. Valid keys are: * BUILD: time to execute all build steps. * PUSH: time to push all specified images. * FETCHSOURCE: time to fetch source. - If the build does not specify source or images, these keys will not be - included. + * SETUPBUILD: time to set up build. If the build does not specify source + or images, these keys will not be included. Fields: + approval: Output only. Describes this build's approval configuration, + status, and result. artifacts: Artifacts produced by the build that should be uploaded upon successful completion of all build steps. availableSecrets: Secrets and secret environment variables. @@ -120,6 +406,8 @@ class Build(_messages.Message): this build, if it was triggered automatically. createTime: Output only. Time at which the request to create the build was received. + failureInfo: Output only. Contains information about the build when + status=FAILURE. finishTime: Output only. Time at which execution of the build was finished. The difference between finish_time and start_time is the duration of the build's execution. @@ -151,7 +439,7 @@ class Build(_messages.Message): serviceAccount: IAM service account whose credentials will be used at build runtime. Must be of the format `projects/{PROJECT_ID}/serviceAccounts/{ACCOUNT}`. ACCOUNT can be email - address or uniqueId of the service account. This field is in beta. + address or uniqueId of the service account. source: The location of the source files to build. sourceProvenance: Output only. A permanent fixed identifier for source. startTime: Output only. Time at which execution of the build was started. @@ -166,16 +454,20 @@ class Build(_messages.Message): will cease and the build status will be `TIMEOUT`. `timeout` starts ticking from `startTime`. Default time is ten minutes. timing: Output only. Stores timing information for phases of the build. - Valid keys are: * BUILD: time to execute all build steps * PUSH: time to - push all specified images. * FETCHSOURCE: time to fetch source. If the - build does not specify source or images, these keys will not be - included. + Valid keys are: * BUILD: time to execute all build steps. * PUSH: time + to push all specified images. * FETCHSOURCE: time to fetch source. * + SETUPBUILD: time to set up build. If the build does not specify source + or images, these keys will not be included. + warnings: Output only. Non-fatal problems encountered during the execution + of the build. """ class StatusValueValuesEnum(_messages.Enum): r"""Output only. Status of the build. Values: STATUS_UNKNOWN: Status of the build is unknown. + PENDING: Build has been created and is pending execution and queuing. It + has not been queued. QUEUED: Build or step is queued; work has not yet begun. WORKING: Build or step is being executed. SUCCESS: Build or step finished successfully. @@ -186,14 +478,15 @@ class StatusValueValuesEnum(_messages.Enum): EXPIRED: Build was enqueued for longer than the value of `queue_ttl`. """ STATUS_UNKNOWN = 0 - QUEUED = 1 - WORKING = 2 - SUCCESS = 3 - FAILURE = 4 - INTERNAL_ERROR = 5 - TIMEOUT = 6 - CANCELLED = 7 - EXPIRED = 8 + PENDING = 1 + QUEUED = 2 + WORKING = 3 + SUCCESS = 4 + FAILURE = 5 + INTERNAL_ERROR = 6 + TIMEOUT = 7 + CANCELLED = 8 + EXPIRED = 9 @encoding.MapUnrecognizedFields('additionalProperties') class SubstitutionsValue(_messages.Message): @@ -223,9 +516,10 @@ class AdditionalProperty(_messages.Message): @encoding.MapUnrecognizedFields('additionalProperties') class TimingValue(_messages.Message): r"""Output only. Stores timing information for phases of the build. Valid - keys are: * BUILD: time to execute all build steps * PUSH: time to push - all specified images. * FETCHSOURCE: time to fetch source. If the build - does not specify source or images, these keys will not be included. + keys are: * BUILD: time to execute all build steps. * PUSH: time to push + all specified images. * FETCHSOURCE: time to fetch source. * SETUPBUILD: + time to set up build. If the build does not specify source or images, + these keys will not be included. Messages: AdditionalProperty: An additional property for a TimingValue object. @@ -247,32 +541,68 @@ class AdditionalProperty(_messages.Message): additionalProperties = _messages.MessageField( 'AdditionalProperty', 1, repeated=True) - artifacts = _messages.MessageField('Artifacts', 1) - availableSecrets = _messages.MessageField('Secrets', 2) - buildTriggerId = _messages.StringField(3) - createTime = _messages.StringField(4) - finishTime = _messages.StringField(5) - id = _messages.StringField(6) - images = _messages.StringField(7, repeated=True) - logUrl = _messages.StringField(8) - logsBucket = _messages.StringField(9) - name = _messages.StringField(10) - options = _messages.MessageField('BuildOptions', 11) - projectId = _messages.StringField(12) - queueTtl = _messages.StringField(13) - results = _messages.MessageField('Results', 14) - secrets = _messages.MessageField('Secret', 15, repeated=True) - serviceAccount = _messages.StringField(16) - source = _messages.MessageField('Source', 17) - sourceProvenance = _messages.MessageField('SourceProvenance', 18) - startTime = _messages.StringField(19) - status = _messages.EnumField('StatusValueValuesEnum', 20) - statusDetail = _messages.StringField(21) - steps = _messages.MessageField('BuildStep', 22, repeated=True) - substitutions = _messages.MessageField('SubstitutionsValue', 23) - tags = _messages.StringField(24, repeated=True) - timeout = _messages.StringField(25) - timing = _messages.MessageField('TimingValue', 26) + approval = _messages.MessageField('BuildApproval', 1) + artifacts = _messages.MessageField('Artifacts', 2) + availableSecrets = _messages.MessageField('Secrets', 3) + buildTriggerId = _messages.StringField(4) + createTime = _messages.StringField(5) + failureInfo = _messages.MessageField('FailureInfo', 6) + finishTime = _messages.StringField(7) + id = _messages.StringField(8) + images = _messages.StringField(9, repeated=True) + logUrl = _messages.StringField(10) + logsBucket = _messages.StringField(11) + name = _messages.StringField(12) + options = _messages.MessageField('BuildOptions', 13) + projectId = _messages.StringField(14) + queueTtl = _messages.StringField(15) + results = _messages.MessageField('Results', 16) + secrets = _messages.MessageField('Secret', 17, repeated=True) + serviceAccount = _messages.StringField(18) + source = _messages.MessageField('Source', 19) + sourceProvenance = _messages.MessageField('SourceProvenance', 20) + startTime = _messages.StringField(21) + status = _messages.EnumField('StatusValueValuesEnum', 22) + statusDetail = _messages.StringField(23) + steps = _messages.MessageField('BuildStep', 24, repeated=True) + substitutions = _messages.MessageField('SubstitutionsValue', 25) + tags = _messages.StringField(26, repeated=True) + timeout = _messages.StringField(27) + timing = _messages.MessageField('TimingValue', 28) + warnings = _messages.MessageField('Warning', 29, repeated=True) + + +class BuildApproval(_messages.Message): + r"""BuildApproval describes a build's approval configuration, state, and + result. + + Enums: + StateValueValuesEnum: Output only. The state of this build's approval. + + Fields: + config: Output only. Configuration for manual approval of this build. + result: Output only. Result of manual approval for this Build. + state: Output only. The state of this build's approval. + """ + class StateValueValuesEnum(_messages.Enum): + r"""Output only. The state of this build's approval. + + Values: + STATE_UNSPECIFIED: Default enum type. This should not be used. + PENDING: Build approval is pending. + APPROVED: Build approval has been approved. + REJECTED: Build approval has been rejected. + CANCELLED: Build was cancelled while it was still pending approval. + """ + STATE_UNSPECIFIED = 0 + PENDING = 1 + APPROVED = 2 + REJECTED = 3 + CANCELLED = 4 + + config = _messages.MessageField('ApprovalConfig', 1) + result = _messages.MessageField('ApprovalResult', 2) + state = _messages.EnumField('StateValueValuesEnum', 3) class BuildOperationMetadata(_messages.Message): @@ -324,6 +654,10 @@ class BuildOptions(_messages.Message): logging: Option to specify the logging mode, which determines if and where build logs are stored. machineType: Compute Engine machine type on which to run the build. + pool: Optional. Specification for execution on a `WorkerPool`. See + [running builds in a private + pool](https://cloud.google.com/build/docs/private-pools/run-builds-in- + private-pool) for more information. requestedVerifyOption: Requested verifiability options. secretEnv: A list of global environment variables, which are encrypted using a Cloud Key Management Service crypto key. These values must be @@ -341,9 +675,7 @@ class BuildOptions(_messages.Message): build step. Using a global volume in a build with only one step is not valid as it is indicative of a build request with an incorrect configuration. - workerPool: Option to specify a `WorkerPool` for the build. Format: - projects/{project}/locations/{location}/workerPools/{workerPool} This - field is in beta and is available only to restricted users. + workerPool: This field deprecated; please use `pool.name` instead. """ class LogStreamingOptionValueValuesEnum(_messages.Enum): r"""Option to define build log streaming behavior to Google Cloud Storage. @@ -367,12 +699,12 @@ class LoggingValueValuesEnum(_messages.Enum): LOGGING_UNSPECIFIED: The service determines the logging mode. The default is `LEGACY`. Do not rely on the default logging behavior as it may change in the future. - LEGACY: Cloud Logging and Cloud Storage logging are enabled. - GCS_ONLY: Only Cloud Storage logging is enabled. + LEGACY: Build logs are stored in Cloud Logging and Cloud Storage. + GCS_ONLY: Build logs are stored in Cloud Storage. STACKDRIVER_ONLY: This option is the same as CLOUD_LOGGING_ONLY. - CLOUD_LOGGING_ONLY: Only Cloud Logging is enabled. Note that logs for - both the Cloud Console UI and Cloud SDK are based on Cloud Storage - logs, so neither will provide logs if this option is chosen. + CLOUD_LOGGING_ONLY: Build logs are stored in Cloud Logging. Selecting + this option will not allow [logs + streaming](https://cloud.google.com/sdk/gcloud/reference/builds/log). NONE: Turn off all logging. No build logs will be captured. """ LOGGING_UNSPECIFIED = 0 @@ -440,15 +772,16 @@ class SubstitutionOptionValueValuesEnum(_messages.Enum): 'LogStreamingOptionValueValuesEnum', 4) logging = _messages.EnumField('LoggingValueValuesEnum', 5) machineType = _messages.EnumField('MachineTypeValueValuesEnum', 6) + pool = _messages.MessageField('PoolOption', 7) requestedVerifyOption = _messages.EnumField( - 'RequestedVerifyOptionValueValuesEnum', 7) - secretEnv = _messages.StringField(8, repeated=True) + 'RequestedVerifyOptionValueValuesEnum', 8) + secretEnv = _messages.StringField(9, repeated=True) sourceProvenanceHash = _messages.EnumField( - 'SourceProvenanceHashValueListEntryValuesEnum', 9, repeated=True) + 'SourceProvenanceHashValueListEntryValuesEnum', 10, repeated=True) substitutionOption = _messages.EnumField( - 'SubstitutionOptionValueValuesEnum', 10) - volumes = _messages.MessageField('Volume', 11, repeated=True) - workerPool = _messages.StringField(12) + 'SubstitutionOptionValueValuesEnum', 11) + volumes = _messages.MessageField('Volume', 12, repeated=True) + workerPool = _messages.StringField(13) class BuildStep(_messages.Message): @@ -495,6 +828,8 @@ class BuildStep(_messages.Message): to use as the name for a later build step. pullTiming: Output only. Stores timing information for pulling this build step's builder image only. + script: A shell script to be executed in the step. When script is + provided, the user cannot specify the entrypoint or args. secretEnv: A list of environment variables which are encrypted using a Cloud Key Management Service crypto key. These values must be specified in the build's `Secret`. @@ -524,6 +859,8 @@ class StatusValueValuesEnum(_messages.Enum): Values: STATUS_UNKNOWN: Status of the build is unknown. + PENDING: Build has been created and is pending execution and queuing. It + has not been queued. QUEUED: Build or step is queued; work has not yet begun. WORKING: Build or step is being executed. SUCCESS: Build or step finished successfully. @@ -534,14 +871,15 @@ class StatusValueValuesEnum(_messages.Enum): EXPIRED: Build was enqueued for longer than the value of `queue_ttl`. """ STATUS_UNKNOWN = 0 - QUEUED = 1 - WORKING = 2 - SUCCESS = 3 - FAILURE = 4 - INTERNAL_ERROR = 5 - TIMEOUT = 6 - CANCELLED = 7 - EXPIRED = 8 + PENDING = 1 + QUEUED = 2 + WORKING = 3 + SUCCESS = 4 + FAILURE = 5 + INTERNAL_ERROR = 6 + TIMEOUT = 7 + CANCELLED = 8 + EXPIRED = 9 args = _messages.StringField(1, repeated=True) dir = _messages.StringField(2) @@ -550,29 +888,50 @@ class StatusValueValuesEnum(_messages.Enum): id = _messages.StringField(5) name = _messages.StringField(6) pullTiming = _messages.MessageField('TimeSpan', 7) - secretEnv = _messages.StringField(8, repeated=True) - status = _messages.EnumField('StatusValueValuesEnum', 9) - timeout = _messages.StringField(10) - timing = _messages.MessageField('TimeSpan', 11) - volumes = _messages.MessageField('Volume', 12, repeated=True) - waitFor = _messages.StringField(13, repeated=True) + script = _messages.StringField(8) + secretEnv = _messages.StringField(9, repeated=True) + status = _messages.EnumField('StatusValueValuesEnum', 10) + timeout = _messages.StringField(11) + timing = _messages.MessageField('TimeSpan', 12) + volumes = _messages.MessageField('Volume', 13, repeated=True) + waitFor = _messages.StringField(14, repeated=True) class BuildTrigger(_messages.Message): r"""Configuration for an automated build in response to source repository changes. + Enums: + EventTypeValueValuesEnum: EventType allows the user to explicitly set the + type of event to which this BuildTrigger should respond. This field will + be validated against the rest of the configuration if it is set. + Messages: SubstitutionsValue: Substitutions for Build resource. The keys must match the following regular expression: `^_[A-Z0-9_]+$`. Fields: + approvalConfig: Configuration for manual approval to start a build + invocation of this BuildTrigger. + autodetect: Autodetect build configuration. The following precedence is + used (case insensitive): 1. cloudbuild.yaml 2. cloudbuild.yml 3. + cloudbuild.json 4. Dockerfile Currently only available for GitHub App + Triggers. + bitbucketServerTriggerConfig: BitbucketServerTriggerConfig describes the + configuration of a trigger that creates a build whenever a Bitbucket + Server event is received. build: Contents of the build template. createTime: Output only. Time when the trigger was created. description: Human-readable description of this trigger. disabled: If true, the trigger will never automatically execute a build. - filename: Path, from the source root, to a file whose contents is used for - the template. + eventType: EventType allows the user to explicitly set the type of event + to which this BuildTrigger should respond. This field will be validated + against the rest of the configuration if it is set. + filename: Path, from the source root, to the build configuration file + (i.e. cloudbuild.yaml). + filter: A Common Expression Language string. + gitFileSource: The file source describing the local or remote Build + template. github: GitHubEventsConfig describes the configuration of a trigger that creates a build whenever a GitHub event is received. Mutually exclusive with `trigger_template`. @@ -595,6 +954,22 @@ class BuildTrigger(_messages.Message): contain only alphanumeric characters and dashes. + They can be 1-64 characters long. + They must begin and end with an alphanumeric character. + pubsubConfig: PubsubConfig describes the configuration of a trigger that + creates a build whenever a Pub/Sub message is published. + resourceName: The `Trigger` name with format: + `projects/{project}/locations/{location}/triggers/{trigger}`, where + {trigger} is a unique identifier generated by the service. + serviceAccount: The service account used for all user-controlled + operations including UpdateBuildTrigger, RunBuildTrigger, CreateBuild, + and CancelBuild. If no service account is set, then the standard Cloud + Build service account ([PROJECT_NUM]@system.gserviceaccount.com) will be + used instead. Format: + `projects/{PROJECT_ID}/serviceAccounts/{ACCOUNT_ID_OR_EMAIL}` + sourceToBuild: The repo and ref of the repository from which to build. + This field is used only for those triggers that do not respond to SCM + events. Triggers that respond to such events build source at whatever + commit caused the event. This field is currently only used by Webhook, + Pub/Sub, Manual, and Cron triggers. substitutions: Substitutions for Build resource. The keys must match the following regular expression: `^_[A-Z0-9_]+$`. tags: Tags for annotation of a `BuildTrigger` @@ -603,7 +978,27 @@ class BuildTrigger(_messages.Message): interpreted as regular expressions. Any branch or tag change that matches that regular expression will trigger a build. Mutually exclusive with `github`. + webhookConfig: WebhookConfig describes the configuration of a trigger that + creates a build whenever a webhook is sent to a trigger's webhook URL. """ + class EventTypeValueValuesEnum(_messages.Enum): + r"""EventType allows the user to explicitly set the type of event to which + this BuildTrigger should respond. This field will be validated against the + rest of the configuration if it is set. + + Values: + EVENT_TYPE_UNSPECIFIED: EVENT_TYPE_UNSPECIFIED event_types are ignored. + REPO: REPO corresponds to the supported VCS integrations. + WEBHOOK: WEBHOOK corresponds to webhook triggers. + PUBSUB: PUBSUB corresponds to pubsub triggers. + MANUAL: MANUAL corresponds to manual-only invoked triggers. + """ + EVENT_TYPE_UNSPECIFIED = 0 + REPO = 1 + WEBHOOK = 2 + PUBSUB = 3 + MANUAL = 4 + @encoding.MapUnrecognizedFields('additionalProperties') class SubstitutionsValue(_messages.Message): r"""Substitutions for Build resource. The keys must match the following @@ -630,19 +1025,31 @@ class AdditionalProperty(_messages.Message): additionalProperties = _messages.MessageField( 'AdditionalProperty', 1, repeated=True) - build = _messages.MessageField('Build', 1) - createTime = _messages.StringField(2) - description = _messages.StringField(3) - disabled = _messages.BooleanField(4) - filename = _messages.StringField(5) - github = _messages.MessageField('GitHubEventsConfig', 6) - id = _messages.StringField(7) - ignoredFiles = _messages.StringField(8, repeated=True) - includedFiles = _messages.StringField(9, repeated=True) - name = _messages.StringField(10) - substitutions = _messages.MessageField('SubstitutionsValue', 11) - tags = _messages.StringField(12, repeated=True) - triggerTemplate = _messages.MessageField('RepoSource', 13) + approvalConfig = _messages.MessageField('ApprovalConfig', 1) + autodetect = _messages.BooleanField(2) + bitbucketServerTriggerConfig = _messages.MessageField( + 'BitbucketServerTriggerConfig', 3) + build = _messages.MessageField('Build', 4) + createTime = _messages.StringField(5) + description = _messages.StringField(6) + disabled = _messages.BooleanField(7) + eventType = _messages.EnumField('EventTypeValueValuesEnum', 8) + filename = _messages.StringField(9) + filter = _messages.StringField(10) + gitFileSource = _messages.MessageField('GitFileSource', 11) + github = _messages.MessageField('GitHubEventsConfig', 12) + id = _messages.StringField(13) + ignoredFiles = _messages.StringField(14, repeated=True) + includedFiles = _messages.StringField(15, repeated=True) + name = _messages.StringField(16) + pubsubConfig = _messages.MessageField('PubsubConfig', 17) + resourceName = _messages.StringField(18) + serviceAccount = _messages.StringField(19) + sourceToBuild = _messages.MessageField('GitRepoSource', 20) + substitutions = _messages.MessageField('SubstitutionsValue', 21) + tags = _messages.StringField(22, repeated=True) + triggerTemplate = _messages.MessageField('RepoSource', 23) + webhookConfig = _messages.MessageField('WebhookConfig', 24) class BuiltImage(_messages.Message): @@ -680,6 +1087,22 @@ class CancelOperationRequest(_messages.Message): r"""The request message for Operations.CancelOperation.""" +class CloudbuildLocationsRegionalWebhookRequest(_messages.Message): + r"""A CloudbuildLocationsRegionalWebhookRequest object. + + Fields: + httpBody: A HttpBody resource to be passed as the request body. + location: Required. The location where the webhook should be sent. + webhookKey: For GitHub Enterprise webhooks, this key is used to associate + the webhook request with the GitHubEnterpriseConfig to use for + validation. + """ + + httpBody = _messages.MessageField('HttpBody', 1) + location = _messages.StringField(2, required=True) + webhookKey = _messages.StringField(3) + + class CloudbuildOperationsCancelRequest(_messages.Message): r"""A CloudbuildOperationsCancelRequest object. @@ -703,6 +1126,20 @@ class CloudbuildOperationsGetRequest(_messages.Message): name = _messages.StringField(1, required=True) +class CloudbuildProjectsBuildsApproveRequest(_messages.Message): + r"""A CloudbuildProjectsBuildsApproveRequest object. + + Fields: + approveBuildRequest: A ApproveBuildRequest resource to be passed as the + request body. + name: Required. Name of the target build. For example: + "projects/{$project_id}/builds/{$build_id}" + """ + + approveBuildRequest = _messages.MessageField('ApproveBuildRequest', 1) + name = _messages.StringField(2, required=True) + + class CloudbuildProjectsBuildsCreateRequest(_messages.Message): r"""A CloudbuildProjectsBuildsCreateRequest object. @@ -756,204 +1193,1097 @@ class CloudbuildProjectsBuildsListRequest(_messages.Message): projectId = _messages.StringField(5, required=True) -class CloudbuildProjectsLocationsBuildsCreateRequest(_messages.Message): - r"""A CloudbuildProjectsLocationsBuildsCreateRequest object. +class CloudbuildProjectsGithubEnterpriseConfigsCreateRequest(_messages.Message): + r"""A CloudbuildProjectsGithubEnterpriseConfigsCreateRequest object. Fields: - build: A Build resource to be passed as the request body. - parent: The parent resource where this build will be created. Format: - `projects/{project}/locations/{location}` - projectId: Required. ID of the project. + gheConfigId: Optional. The ID to use for the GithubEnterpriseConfig, which + will become the final component of the GithubEnterpriseConfig's resource + name. ghe_config_id must meet the following requirements: + They must + contain only alphanumeric characters and dashes. + They can be 1-64 + characters long. + They must begin and end with an alphanumeric + character + gitHubEnterpriseConfig: A GitHubEnterpriseConfig resource to be passed as + the request body. + parent: Name of the parent project. For example: + projects/{$project_number} or projects/{$project_id} + projectId: ID of the project. """ - build = _messages.MessageField('Build', 1) - parent = _messages.StringField(2, required=True) + gheConfigId = _messages.StringField(1) + gitHubEnterpriseConfig = _messages.MessageField('GitHubEnterpriseConfig', 2) + parent = _messages.StringField(3, required=True) + projectId = _messages.StringField(4) + + +class CloudbuildProjectsGithubEnterpriseConfigsDeleteRequest(_messages.Message): + r"""A CloudbuildProjectsGithubEnterpriseConfigsDeleteRequest object. + + Fields: + configId: Unique identifier of the `GitHubEnterpriseConfig` + name: This field should contain the name of the enterprise config + resource. For example: + "projects/{$project_id}/githubEnterpriseConfigs/{$config_id}" + projectId: ID of the project + """ + + configId = _messages.StringField(1) + name = _messages.StringField(2, required=True) projectId = _messages.StringField(3) -class CloudbuildProjectsLocationsBuildsGetRequest(_messages.Message): - r"""A CloudbuildProjectsLocationsBuildsGetRequest object. +class CloudbuildProjectsGithubEnterpriseConfigsGetRequest(_messages.Message): + r"""A CloudbuildProjectsGithubEnterpriseConfigsGetRequest object. Fields: - id: Required. ID of the build. - name: The name of the `Build` to retrieve. Format: - `projects/{project}/locations/{location}/builds/{build}` - projectId: Required. ID of the project. + configId: Unique identifier of the `GitHubEnterpriseConfig` + name: This field should contain the name of the enterprise config + resource. For example: + "projects/{$project_id}/githubEnterpriseConfigs/{$config_id}" + projectId: ID of the project """ - id = _messages.StringField(1) + configId = _messages.StringField(1) name = _messages.StringField(2, required=True) projectId = _messages.StringField(3) -class CloudbuildProjectsLocationsBuildsListRequest(_messages.Message): - r"""A CloudbuildProjectsLocationsBuildsListRequest object. +class CloudbuildProjectsGithubEnterpriseConfigsListRequest(_messages.Message): + r"""A CloudbuildProjectsGithubEnterpriseConfigsListRequest object. Fields: - filter: The raw filter text to constrain the results. - pageSize: Number of results to return in the list. - pageToken: The page token for the next page of Builds. If unspecified, the - first page of results is returned. If the token is rejected for any - reason, INVALID_ARGUMENT will be thrown. In this case, the token should - be discarded, and pagination should be restarted from the first page of - results. See https://google.aip.dev/158 for more. - parent: The parent of the collection of `Builds`. Format: - `projects/{project}/locations/location` - projectId: Required. ID of the project. + parent: Name of the parent project. For example: + projects/{$project_number} or projects/{$project_id} + projectId: ID of the project """ - filter = _messages.StringField(1) - pageSize = _messages.IntegerField(2, variant=_messages.Variant.INT32) - pageToken = _messages.StringField(3) - parent = _messages.StringField(4, required=True) - projectId = _messages.StringField(5) + parent = _messages.StringField(1, required=True) + projectId = _messages.StringField(2) -class CloudbuildProjectsLocationsOperationsCancelRequest(_messages.Message): - r"""A CloudbuildProjectsLocationsOperationsCancelRequest object. +class CloudbuildProjectsGithubEnterpriseConfigsPatchRequest(_messages.Message): + r"""A CloudbuildProjectsGithubEnterpriseConfigsPatchRequest object. Fields: - cancelOperationRequest: A CancelOperationRequest resource to be passed as + gitHubEnterpriseConfig: A GitHubEnterpriseConfig resource to be passed as the request body. - name: The name of the operation resource to be cancelled. + name: Optional. The full resource name for the GitHubEnterpriseConfig For + example: "projects/{$project_id}/githubEnterpriseConfigs/{$config_id}" + updateMask: Update mask for the resource. If this is set, the server will + only update the fields specified in the field mask. Otherwise, a full + update of the mutable resource fields will be performed. """ - cancelOperationRequest = _messages.MessageField('CancelOperationRequest', 1) + gitHubEnterpriseConfig = _messages.MessageField('GitHubEnterpriseConfig', 1) name = _messages.StringField(2, required=True) + updateMask = _messages.StringField(3) -class CloudbuildProjectsLocationsOperationsGetRequest(_messages.Message): - r"""A CloudbuildProjectsLocationsOperationsGetRequest object. +class CloudbuildProjectsLocationsBitbucketServerConfigsAddBitbucketServerConnectedRepositoryRequest( + _messages.Message): + r"""A CloudbuildProjectsLocationsBitbucketServerConfigsAddBitbucketServerCon + nectedRepositoryRequest object. Fields: - name: The name of the operation resource. + addBitbucketServerConnectedRepositoryRequest: A + AddBitbucketServerConnectedRepositoryRequest resource to be passed as + the request body. + config: Required. The name of the `BitbucketServerConfig` to add a + connected repository. Format: `projects/{project}/locations/{location}/b + itbucketServerConfigs/{config}` + """ + + addBitbucketServerConnectedRepositoryRequest = _messages.MessageField( + 'AddBitbucketServerConnectedRepositoryRequest', 1) + config = _messages.StringField(2, required=True) + + +class CloudbuildProjectsLocationsBitbucketServerConfigsConnectedRepositoriesBatchCreateRequest( + _messages.Message): + r"""A CloudbuildProjectsLocationsBitbucketServerConfigsConnectedRepositories + BatchCreateRequest object. + + Fields: + batchCreateBitbucketServerConnectedRepositoriesRequest: A + BatchCreateBitbucketServerConnectedRepositoriesRequest resource to be + passed as the request body. + parent: The name of the `BitbucketServerConfig` that added connected + repository. Format: `projects/{project}/locations/{location}/bitbucketSe + rverConfigs/{config}` + """ + + batchCreateBitbucketServerConnectedRepositoriesRequest = _messages.MessageField( + 'BatchCreateBitbucketServerConnectedRepositoriesRequest', 1) + parent = _messages.StringField(2, required=True) + + +class CloudbuildProjectsLocationsBitbucketServerConfigsCreateRequest( + _messages.Message): + r"""A CloudbuildProjectsLocationsBitbucketServerConfigsCreateRequest object. + + Fields: + bitbucketServerConfig: A BitbucketServerConfig resource to be passed as + the request body. + bitbucketServerConfigId: Optional. The ID to use for the + BitbucketServerConfig, which will become the final component of the + BitbucketServerConfig's resource name. bitbucket_server_config_id must + meet the following requirements: + They must contain only alphanumeric + characters and dashes. + They can be 1-64 characters long. + They must + begin and end with an alphanumeric character. + parent: Required. Name of the parent resource. + """ + + bitbucketServerConfig = _messages.MessageField('BitbucketServerConfig', 1) + bitbucketServerConfigId = _messages.StringField(2) + parent = _messages.StringField(3, required=True) + + +class CloudbuildProjectsLocationsBitbucketServerConfigsDeleteRequest( + _messages.Message): + r"""A CloudbuildProjectsLocationsBitbucketServerConfigsDeleteRequest object. + + Fields: + name: Required. The config resource name. """ name = _messages.StringField(1, required=True) -class CloudbuildProjectsTriggersCreateRequest(_messages.Message): - r"""A CloudbuildProjectsTriggersCreateRequest object. +class CloudbuildProjectsLocationsBitbucketServerConfigsGetRequest( + _messages.Message): + r"""A CloudbuildProjectsLocationsBitbucketServerConfigsGetRequest object. Fields: - buildTrigger: A BuildTrigger resource to be passed as the request body. - projectId: Required. ID of the project for which to configure automatic - builds. + name: Required. The config resource name. """ - buildTrigger = _messages.MessageField('BuildTrigger', 1) - projectId = _messages.StringField(2, required=True) + name = _messages.StringField(1, required=True) -class CloudbuildProjectsTriggersDeleteRequest(_messages.Message): - r"""A CloudbuildProjectsTriggersDeleteRequest object. +class CloudbuildProjectsLocationsBitbucketServerConfigsListRequest( + _messages.Message): + r"""A CloudbuildProjectsLocationsBitbucketServerConfigsListRequest object. Fields: - projectId: Required. ID of the project that owns the trigger. - triggerId: Required. ID of the `BuildTrigger` to delete. + pageSize: The maximum number of configs to return. The service may return + fewer than this value. If unspecified, at most 50 configs will be + returned. The maximum value is 1000; values above 1000 will be coerced + to 1000. + pageToken: A page token, received from a previous + `ListBitbucketServerConfigsRequest` call. Provide this to retrieve the + subsequent page. When paginating, all other parameters provided to + `ListBitbucketServerConfigsRequest` must match the call that provided + the page token. + parent: Required. Name of the parent resource. """ - projectId = _messages.StringField(1, required=True) - triggerId = _messages.StringField(2, required=True) + pageSize = _messages.IntegerField(1, variant=_messages.Variant.INT32) + pageToken = _messages.StringField(2) + parent = _messages.StringField(3, required=True) -class CloudbuildProjectsTriggersGetRequest(_messages.Message): - r"""A CloudbuildProjectsTriggersGetRequest object. +class CloudbuildProjectsLocationsBitbucketServerConfigsPatchRequest( + _messages.Message): + r"""A CloudbuildProjectsLocationsBitbucketServerConfigsPatchRequest object. Fields: - projectId: Required. ID of the project that owns the trigger. - triggerId: Required. Identifier (`id` or `name`) of the `BuildTrigger` to - get. + bitbucketServerConfig: A BitbucketServerConfig resource to be passed as + the request body. + name: The resource name for the config. + updateMask: Update mask for the resource. If this is set, the server will + only update the fields specified in the field mask. Otherwise, a full + update of the mutable resource fields will be performed. """ - projectId = _messages.StringField(1, required=True) - triggerId = _messages.StringField(2, required=True) + bitbucketServerConfig = _messages.MessageField('BitbucketServerConfig', 1) + name = _messages.StringField(2, required=True) + updateMask = _messages.StringField(3) -class CloudbuildProjectsTriggersListRequest(_messages.Message): - r"""A CloudbuildProjectsTriggersListRequest object. +class CloudbuildProjectsLocationsBitbucketServerConfigsRemoveBitbucketServerConnectedRepositoryRequest( + _messages.Message): + r"""A CloudbuildProjectsLocationsBitbucketServerConfigsRemoveBitbucketServer + ConnectedRepositoryRequest object. Fields: - pageSize: Number of results to return in the list. - pageToken: Token to provide to skip to a particular spot in the list. - projectId: Required. ID of the project for which to list BuildTriggers. + config: Required. The name of the `BitbucketServerConfig` to remove a + connected repository. Format: `projects/{project}/locations/{location}/b + itbucketServerConfigs/{config}` + removeBitbucketServerConnectedRepositoryRequest: A + RemoveBitbucketServerConnectedRepositoryRequest resource to be passed as + the request body. + """ + + config = _messages.StringField(1, required=True) + removeBitbucketServerConnectedRepositoryRequest = _messages.MessageField( + 'RemoveBitbucketServerConnectedRepositoryRequest', 2) + + +class CloudbuildProjectsLocationsBitbucketServerConfigsReposListRequest( + _messages.Message): + r"""A CloudbuildProjectsLocationsBitbucketServerConfigsReposListRequest + object. + + Fields: + pageSize: The maximum number of configs to return. The service may return + fewer than this value. If unspecified, at most 50 configs will be + returned. The maximum value is 1000; values above 1000 will be coerced + to 1000. + pageToken: A page token, received from a previous + `ListBitbucketServerRepositoriesRequest` call. Provide this to retrieve + the subsequent page. When paginating, all other parameters provided to + `ListBitbucketServerConfigsRequest` must match the call that provided + the page token. + parent: Required. Name of the parent resource. """ pageSize = _messages.IntegerField(1, variant=_messages.Variant.INT32) pageToken = _messages.StringField(2) - projectId = _messages.StringField(3, required=True) + parent = _messages.StringField(3, required=True) -class CloudbuildProjectsTriggersPatchRequest(_messages.Message): - r"""A CloudbuildProjectsTriggersPatchRequest object. +class CloudbuildProjectsLocationsBuildsApproveRequest(_messages.Message): + r"""A CloudbuildProjectsLocationsBuildsApproveRequest object. Fields: - buildTrigger: A BuildTrigger resource to be passed as the request body. - projectId: Required. ID of the project that owns the trigger. - triggerId: Required. ID of the `BuildTrigger` to update. + approveBuildRequest: A ApproveBuildRequest resource to be passed as the + request body. + name: Required. Name of the target build. For example: + "projects/{$project_id}/builds/{$build_id}" """ - buildTrigger = _messages.MessageField('BuildTrigger', 1) - projectId = _messages.StringField(2, required=True) - triggerId = _messages.StringField(3, required=True) + approveBuildRequest = _messages.MessageField('ApproveBuildRequest', 1) + name = _messages.StringField(2, required=True) -class CloudbuildProjectsTriggersRunRequest(_messages.Message): - r"""A CloudbuildProjectsTriggersRunRequest object. +class CloudbuildProjectsLocationsBuildsCreateRequest(_messages.Message): + r"""A CloudbuildProjectsLocationsBuildsCreateRequest object. Fields: + build: A Build resource to be passed as the request body. + parent: The parent resource where this build will be created. Format: + `projects/{project}/locations/{location}` projectId: Required. ID of the project. - repoSource: A RepoSource resource to be passed as the request body. - triggerId: Required. ID of the trigger. """ - projectId = _messages.StringField(1, required=True) - repoSource = _messages.MessageField('RepoSource', 2) - triggerId = _messages.StringField(3, required=True) + build = _messages.MessageField('Build', 1) + parent = _messages.StringField(2, required=True) + projectId = _messages.StringField(3) + + +class CloudbuildProjectsLocationsBuildsGetRequest(_messages.Message): + r"""A CloudbuildProjectsLocationsBuildsGetRequest object. + + Fields: + id: Required. ID of the build. + name: The name of the `Build` to retrieve. Format: + `projects/{project}/locations/{location}/builds/{build}` + projectId: Required. ID of the project. + """ + + id = _messages.StringField(1) + name = _messages.StringField(2, required=True) + projectId = _messages.StringField(3) + + +class CloudbuildProjectsLocationsBuildsListRequest(_messages.Message): + r"""A CloudbuildProjectsLocationsBuildsListRequest object. + + Fields: + filter: The raw filter text to constrain the results. + pageSize: Number of results to return in the list. + pageToken: The page token for the next page of Builds. If unspecified, the + first page of results is returned. If the token is rejected for any + reason, INVALID_ARGUMENT will be thrown. In this case, the token should + be discarded, and pagination should be restarted from the first page of + results. See https://google.aip.dev/158 for more. + parent: The parent of the collection of `Builds`. Format: + `projects/{project}/locations/location` + projectId: Required. ID of the project. + """ + + filter = _messages.StringField(1) + pageSize = _messages.IntegerField(2, variant=_messages.Variant.INT32) + pageToken = _messages.StringField(3) + parent = _messages.StringField(4, required=True) + projectId = _messages.StringField(5) + + +class CloudbuildProjectsLocationsGithubEnterpriseConfigsCreateRequest( + _messages.Message): + r"""A CloudbuildProjectsLocationsGithubEnterpriseConfigsCreateRequest + object. + + Fields: + gheConfigId: Optional. The ID to use for the GithubEnterpriseConfig, which + will become the final component of the GithubEnterpriseConfig's resource + name. ghe_config_id must meet the following requirements: + They must + contain only alphanumeric characters and dashes. + They can be 1-64 + characters long. + They must begin and end with an alphanumeric + character + gitHubEnterpriseConfig: A GitHubEnterpriseConfig resource to be passed as + the request body. + parent: Name of the parent project. For example: + projects/{$project_number} or projects/{$project_id} + projectId: ID of the project. + """ + + gheConfigId = _messages.StringField(1) + gitHubEnterpriseConfig = _messages.MessageField('GitHubEnterpriseConfig', 2) + parent = _messages.StringField(3, required=True) + projectId = _messages.StringField(4) + + +class CloudbuildProjectsLocationsGithubEnterpriseConfigsDeleteRequest( + _messages.Message): + r"""A CloudbuildProjectsLocationsGithubEnterpriseConfigsDeleteRequest + object. + + Fields: + configId: Unique identifier of the `GitHubEnterpriseConfig` + name: This field should contain the name of the enterprise config + resource. For example: + "projects/{$project_id}/githubEnterpriseConfigs/{$config_id}" + projectId: ID of the project + """ + + configId = _messages.StringField(1) + name = _messages.StringField(2, required=True) + projectId = _messages.StringField(3) + + +class CloudbuildProjectsLocationsGithubEnterpriseConfigsGetRequest( + _messages.Message): + r"""A CloudbuildProjectsLocationsGithubEnterpriseConfigsGetRequest object. + + Fields: + configId: Unique identifier of the `GitHubEnterpriseConfig` + name: This field should contain the name of the enterprise config + resource. For example: + "projects/{$project_id}/githubEnterpriseConfigs/{$config_id}" + projectId: ID of the project + """ + + configId = _messages.StringField(1) + name = _messages.StringField(2, required=True) + projectId = _messages.StringField(3) + + +class CloudbuildProjectsLocationsGithubEnterpriseConfigsListRequest( + _messages.Message): + r"""A CloudbuildProjectsLocationsGithubEnterpriseConfigsListRequest object. + + Fields: + parent: Name of the parent project. For example: + projects/{$project_number} or projects/{$project_id} + projectId: ID of the project + """ + + parent = _messages.StringField(1, required=True) + projectId = _messages.StringField(2) + + +class CloudbuildProjectsLocationsGithubEnterpriseConfigsPatchRequest( + _messages.Message): + r"""A CloudbuildProjectsLocationsGithubEnterpriseConfigsPatchRequest object. + + Fields: + gitHubEnterpriseConfig: A GitHubEnterpriseConfig resource to be passed as + the request body. + name: Optional. The full resource name for the GitHubEnterpriseConfig For + example: "projects/{$project_id}/githubEnterpriseConfigs/{$config_id}" + updateMask: Update mask for the resource. If this is set, the server will + only update the fields specified in the field mask. Otherwise, a full + update of the mutable resource fields will be performed. + """ + + gitHubEnterpriseConfig = _messages.MessageField('GitHubEnterpriseConfig', 1) + name = _messages.StringField(2, required=True) + updateMask = _messages.StringField(3) + + +class CloudbuildProjectsLocationsOperationsCancelRequest(_messages.Message): + r"""A CloudbuildProjectsLocationsOperationsCancelRequest object. + + Fields: + cancelOperationRequest: A CancelOperationRequest resource to be passed as + the request body. + name: The name of the operation resource to be cancelled. + """ + + cancelOperationRequest = _messages.MessageField('CancelOperationRequest', 1) + name = _messages.StringField(2, required=True) + + +class CloudbuildProjectsLocationsOperationsGetRequest(_messages.Message): + r"""A CloudbuildProjectsLocationsOperationsGetRequest object. + + Fields: + name: The name of the operation resource. + """ + + name = _messages.StringField(1, required=True) + + +class CloudbuildProjectsLocationsTriggersCreateRequest(_messages.Message): + r"""A CloudbuildProjectsLocationsTriggersCreateRequest object. + + Fields: + buildTrigger: A BuildTrigger resource to be passed as the request body. + parent: The parent resource where this trigger will be created. Format: + `projects/{project}/locations/{location}` + projectId: Required. ID of the project for which to configure automatic + builds. + """ + + buildTrigger = _messages.MessageField('BuildTrigger', 1) + parent = _messages.StringField(2, required=True) + projectId = _messages.StringField(3) + + +class CloudbuildProjectsLocationsTriggersDeleteRequest(_messages.Message): + r"""A CloudbuildProjectsLocationsTriggersDeleteRequest object. + + Fields: + name: The name of the `Trigger` to delete. Format: + `projects/{project}/locations/{location}/triggers/{trigger}` + projectId: Required. ID of the project that owns the trigger. + triggerId: Required. ID of the `BuildTrigger` to delete. + """ + + name = _messages.StringField(1, required=True) + projectId = _messages.StringField(2) + triggerId = _messages.StringField(3) + + +class CloudbuildProjectsLocationsTriggersGetRequest(_messages.Message): + r"""A CloudbuildProjectsLocationsTriggersGetRequest object. + + Fields: + name: The name of the `Trigger` to retrieve. Format: + `projects/{project}/locations/{location}/triggers/{trigger}` + projectId: Required. ID of the project that owns the trigger. + triggerId: Required. Identifier (`id` or `name`) of the `BuildTrigger` to + get. + """ + + name = _messages.StringField(1, required=True) + projectId = _messages.StringField(2) + triggerId = _messages.StringField(3) + + +class CloudbuildProjectsLocationsTriggersListRequest(_messages.Message): + r"""A CloudbuildProjectsLocationsTriggersListRequest object. + + Fields: + pageSize: Number of results to return in the list. + pageToken: Token to provide to skip to a particular spot in the list. + parent: The parent of the collection of `Triggers`. Format: + `projects/{project}/locations/{location}` + projectId: Required. ID of the project for which to list BuildTriggers. + """ + + pageSize = _messages.IntegerField(1, variant=_messages.Variant.INT32) + pageToken = _messages.StringField(2) + parent = _messages.StringField(3, required=True) + projectId = _messages.StringField(4) + + +class CloudbuildProjectsLocationsTriggersPatchRequest(_messages.Message): + r"""A CloudbuildProjectsLocationsTriggersPatchRequest object. + + Fields: + buildTrigger: A BuildTrigger resource to be passed as the request body. + projectId: Required. ID of the project that owns the trigger. + resourceName: The `Trigger` name with format: + `projects/{project}/locations/{location}/triggers/{trigger}`, where + {trigger} is a unique identifier generated by the service. + triggerId: Required. ID of the `BuildTrigger` to update. + """ + + buildTrigger = _messages.MessageField('BuildTrigger', 1) + projectId = _messages.StringField(2) + resourceName = _messages.StringField(3, required=True) + triggerId = _messages.StringField(4) + + +class CloudbuildProjectsLocationsTriggersRunRequest(_messages.Message): + r"""A CloudbuildProjectsLocationsTriggersRunRequest object. + + Fields: + name: The name of the `Trigger` to run. Format: + `projects/{project}/locations/{location}/triggers/{trigger}` + runBuildTriggerRequest: A RunBuildTriggerRequest resource to be passed as + the request body. + """ + + name = _messages.StringField(1, required=True) + runBuildTriggerRequest = _messages.MessageField('RunBuildTriggerRequest', 2) + + +class CloudbuildProjectsLocationsTriggersWebhookRequest(_messages.Message): + r"""A CloudbuildProjectsLocationsTriggersWebhookRequest object. + + Fields: + httpBody: A HttpBody resource to be passed as the request body. + name: The name of the `ReceiveTriggerWebhook` to retrieve. Format: + `projects/{project}/locations/{location}/triggers/{trigger}` + projectId: Project in which the specified trigger lives + secret: Secret token used for authorization if an OAuth token isn't + provided. + trigger: Name of the trigger to run the payload against + """ + + httpBody = _messages.MessageField('HttpBody', 1) + name = _messages.StringField(2, required=True) + projectId = _messages.StringField(3) + secret = _messages.StringField(4) + trigger = _messages.StringField(5) + + +class CloudbuildProjectsLocationsWorkerPoolsCreateRequest(_messages.Message): + r"""A CloudbuildProjectsLocationsWorkerPoolsCreateRequest object. + + Fields: + parent: Required. The parent resource where this worker pool will be + created. Format: `projects/{project}/locations/{location}`. + validateOnly: If set, validate the request and preview the response, but + do not actually post it. + workerPool: A WorkerPool resource to be passed as the request body. + workerPoolId: Required. Immutable. The ID to use for the `WorkerPool`, + which will become the final component of the resource name. This value + should be 1-63 characters, and valid characters are /a-z-/. + """ + + parent = _messages.StringField(1, required=True) + validateOnly = _messages.BooleanField(2) + workerPool = _messages.MessageField('WorkerPool', 3) + workerPoolId = _messages.StringField(4) + + +class CloudbuildProjectsLocationsWorkerPoolsDeleteRequest(_messages.Message): + r"""A CloudbuildProjectsLocationsWorkerPoolsDeleteRequest object. + + Fields: + allowMissing: If set to true, and the `WorkerPool` is not found, the + request will succeed but no action will be taken on the server. + etag: Optional. If provided, it must match the server's etag on the + workerpool for the request to be processed. + name: Required. The name of the `WorkerPool` to delete. Format: + `projects/{project}/locations/{workerPool}/workerPools/{workerPool}`. + validateOnly: If set, validate the request and preview the response, but + do not actually post it. + """ + + allowMissing = _messages.BooleanField(1) + etag = _messages.StringField(2) + name = _messages.StringField(3, required=True) + validateOnly = _messages.BooleanField(4) + + +class CloudbuildProjectsLocationsWorkerPoolsGetRequest(_messages.Message): + r"""A CloudbuildProjectsLocationsWorkerPoolsGetRequest object. + + Fields: + name: Required. The name of the `WorkerPool` to retrieve. Format: + `projects/{project}/locations/{location}/workerPools/{workerPool}`. + """ + + name = _messages.StringField(1, required=True) + + +class CloudbuildProjectsLocationsWorkerPoolsListRequest(_messages.Message): + r"""A CloudbuildProjectsLocationsWorkerPoolsListRequest object. + + Fields: + pageSize: The maximum number of `WorkerPool`s to return. The service may + return fewer than this value. If omitted, the server will use a sensible + default. + pageToken: A page token, received from a previous `ListWorkerPools` call. + Provide this to retrieve the subsequent page. + parent: Required. The parent of the collection of `WorkerPools`. Format: + `projects/{project}/locations/{location}`. + """ + + pageSize = _messages.IntegerField(1, variant=_messages.Variant.INT32) + pageToken = _messages.StringField(2) + parent = _messages.StringField(3, required=True) + + +class CloudbuildProjectsLocationsWorkerPoolsPatchRequest(_messages.Message): + r"""A CloudbuildProjectsLocationsWorkerPoolsPatchRequest object. + + Fields: + name: Output only. The resource name of the `WorkerPool`, with format + `projects/{project}/locations/{location}/workerPools/{worker_pool}`. The + value of `{worker_pool}` is provided by `worker_pool_id` in + `CreateWorkerPool` request and the value of `{location}` is determined + by the endpoint accessed. + updateMask: A mask specifying which fields in `worker_pool` to update. + validateOnly: If set, validate the request and preview the response, but + do not actually post it. + workerPool: A WorkerPool resource to be passed as the request body. + """ + + name = _messages.StringField(1, required=True) + updateMask = _messages.StringField(2) + validateOnly = _messages.BooleanField(3) + workerPool = _messages.MessageField('WorkerPool', 4) + + +class CloudbuildProjectsTriggersCreateRequest(_messages.Message): + r"""A CloudbuildProjectsTriggersCreateRequest object. + + Fields: + buildTrigger: A BuildTrigger resource to be passed as the request body. + parent: The parent resource where this trigger will be created. Format: + `projects/{project}/locations/{location}` + projectId: Required. ID of the project for which to configure automatic + builds. + """ + + buildTrigger = _messages.MessageField('BuildTrigger', 1) + parent = _messages.StringField(2) + projectId = _messages.StringField(3, required=True) + + +class CloudbuildProjectsTriggersDeleteRequest(_messages.Message): + r"""A CloudbuildProjectsTriggersDeleteRequest object. + + Fields: + name: The name of the `Trigger` to delete. Format: + `projects/{project}/locations/{location}/triggers/{trigger}` + projectId: Required. ID of the project that owns the trigger. + triggerId: Required. ID of the `BuildTrigger` to delete. + """ + + name = _messages.StringField(1) + projectId = _messages.StringField(2, required=True) + triggerId = _messages.StringField(3, required=True) + + +class CloudbuildProjectsTriggersGetRequest(_messages.Message): + r"""A CloudbuildProjectsTriggersGetRequest object. + + Fields: + name: The name of the `Trigger` to retrieve. Format: + `projects/{project}/locations/{location}/triggers/{trigger}` + projectId: Required. ID of the project that owns the trigger. + triggerId: Required. Identifier (`id` or `name`) of the `BuildTrigger` to + get. + """ + + name = _messages.StringField(1) + projectId = _messages.StringField(2, required=True) + triggerId = _messages.StringField(3, required=True) + + +class CloudbuildProjectsTriggersListRequest(_messages.Message): + r"""A CloudbuildProjectsTriggersListRequest object. + + Fields: + pageSize: Number of results to return in the list. + pageToken: Token to provide to skip to a particular spot in the list. + parent: The parent of the collection of `Triggers`. Format: + `projects/{project}/locations/{location}` + projectId: Required. ID of the project for which to list BuildTriggers. + """ + + pageSize = _messages.IntegerField(1, variant=_messages.Variant.INT32) + pageToken = _messages.StringField(2) + parent = _messages.StringField(3) + projectId = _messages.StringField(4, required=True) + + +class CloudbuildProjectsTriggersPatchRequest(_messages.Message): + r"""A CloudbuildProjectsTriggersPatchRequest object. + + Fields: + buildTrigger: A BuildTrigger resource to be passed as the request body. + projectId: Required. ID of the project that owns the trigger. + triggerId: Required. ID of the `BuildTrigger` to update. + """ + + buildTrigger = _messages.MessageField('BuildTrigger', 1) + projectId = _messages.StringField(2, required=True) + triggerId = _messages.StringField(3, required=True) + + +class CloudbuildProjectsTriggersRunRequest(_messages.Message): + r"""A CloudbuildProjectsTriggersRunRequest object. + + Fields: + name: The name of the `Trigger` to run. Format: + `projects/{project}/locations/{location}/triggers/{trigger}` + projectId: Required. ID of the project. + repoSource: A RepoSource resource to be passed as the request body. + triggerId: Required. ID of the trigger. + """ + + name = _messages.StringField(1) + projectId = _messages.StringField(2, required=True) + repoSource = _messages.MessageField('RepoSource', 3) + triggerId = _messages.StringField(4, required=True) + + +class CloudbuildProjectsTriggersWebhookRequest(_messages.Message): + r"""A CloudbuildProjectsTriggersWebhookRequest object. + + Fields: + httpBody: A HttpBody resource to be passed as the request body. + name: The name of the `ReceiveTriggerWebhook` to retrieve. Format: + `projects/{project}/locations/{location}/triggers/{trigger}` + projectId: Project in which the specified trigger lives + secret: Secret token used for authorization if an OAuth token isn't + provided. + trigger: Name of the trigger to run the payload against + """ + + httpBody = _messages.MessageField('HttpBody', 1) + name = _messages.StringField(2) + projectId = _messages.StringField(3, required=True) + secret = _messages.StringField(4) + trigger = _messages.StringField(5, required=True) + + +class CloudbuildWebhookRequest(_messages.Message): + r"""A CloudbuildWebhookRequest object. + + Fields: + httpBody: A HttpBody resource to be passed as the request body. + webhookKey: For GitHub Enterprise webhooks, this key is used to associate + the webhook request with the GitHubEnterpriseConfig to use for + validation. + """ + + httpBody = _messages.MessageField('HttpBody', 1) + webhookKey = _messages.StringField(2) + + +class CreateBitbucketServerConfigOperationMetadata(_messages.Message): + r"""Metadata for `CreateBitbucketServerConfig` operation. + + Fields: + bitbucketServerConfig: The resource name of the BitbucketServerConfig to + be created. Format: + `projects/{project}/locations/{location}/bitbucketServerConfigs/{id}`. + completeTime: Time the operation was completed. + createTime: Time the operation was created. + """ + + bitbucketServerConfig = _messages.StringField(1) + completeTime = _messages.StringField(2) + createTime = _messages.StringField(3) + + +class CreateBitbucketServerConnectedRepositoryRequest(_messages.Message): + r"""Request to connect a repository from a connected Bitbucket Server host. + + Fields: + bitbucketServerConnectedRepository: Required. The Bitbucket Server + repository to connect. + parent: Required. The name of the `BitbucketServerConfig` that added + connected repository. Format: `projects/{project}/locations/{location}/b + itbucketServerConfigs/{config}` + """ + + bitbucketServerConnectedRepository = _messages.MessageField( + 'BitbucketServerConnectedRepository', 1) + parent = _messages.StringField(2) + + +class CreateGitHubEnterpriseConfigOperationMetadata(_messages.Message): + r"""Metadata for `CreateGithubEnterpriseConfig` operation. + + Fields: + completeTime: Time the operation was completed. + createTime: Time the operation was created. + githubEnterpriseConfig: The resource name of the GitHubEnterprise to be + created. Format: + `projects/{project}/locations/{location}/githubEnterpriseConfigs/{id}`. + """ + + completeTime = _messages.StringField(1) + createTime = _messages.StringField(2) + githubEnterpriseConfig = _messages.StringField(3) + + +class CreateGitLabConfigOperationMetadata(_messages.Message): + r"""Metadata for `CreateGitLabConfig` operation. + + Fields: + completeTime: Time the operation was completed. + createTime: Time the operation was created. + gitlabConfig: The resource name of the GitLabConfig to be created. Format: + `projects/{project}/locations/{location}/gitlabConfigs/{id}`. + """ + + completeTime = _messages.StringField(1) + createTime = _messages.StringField(2) + gitlabConfig = _messages.StringField(3) + + +class CreateWorkerPoolOperationMetadata(_messages.Message): + r"""Metadata for the `CreateWorkerPool` operation. + + Fields: + completeTime: Time the operation was completed. + createTime: Time the operation was created. + workerPool: The resource name of the `WorkerPool` to create. Format: + `projects/{project}/locations/{location}/workerPools/{worker_pool}`. + """ + + completeTime = _messages.StringField(1) + createTime = _messages.StringField(2) + workerPool = _messages.StringField(3) + + +class DeleteBitbucketServerConfigOperationMetadata(_messages.Message): + r"""Metadata for `DeleteBitbucketServerConfig` operation. + + Fields: + bitbucketServerConfig: The resource name of the BitbucketServerConfig to + be deleted. Format: + `projects/{project}/locations/{location}/bitbucketServerConfigs/{id}`. + completeTime: Time the operation was completed. + createTime: Time the operation was created. + """ + + bitbucketServerConfig = _messages.StringField(1) + completeTime = _messages.StringField(2) + createTime = _messages.StringField(3) + + +class DeleteGitHubEnterpriseConfigOperationMetadata(_messages.Message): + r"""Metadata for `DeleteGitHubEnterpriseConfig` operation. + + Fields: + completeTime: Time the operation was completed. + createTime: Time the operation was created. + githubEnterpriseConfig: The resource name of the GitHubEnterprise to be + deleted. Format: + `projects/{project}/locations/{location}/githubEnterpriseConfigs/{id}`. + """ + + completeTime = _messages.StringField(1) + createTime = _messages.StringField(2) + githubEnterpriseConfig = _messages.StringField(3) + + +class DeleteGitLabConfigOperationMetadata(_messages.Message): + r"""Metadata for `DeleteGitLabConfig` operation. + + Fields: + completeTime: Time the operation was completed. + createTime: Time the operation was created. + gitlabConfig: The resource name of the GitLabConfig to be created. Format: + `projects/{project}/locations/{location}/gitlabConfigs/{id}`. + """ + + completeTime = _messages.StringField(1) + createTime = _messages.StringField(2) + gitlabConfig = _messages.StringField(3) + + +class DeleteWorkerPoolOperationMetadata(_messages.Message): + r"""Metadata for the `DeleteWorkerPool` operation. + + Fields: + completeTime: Time the operation was completed. + createTime: Time the operation was created. + workerPool: The resource name of the `WorkerPool` being deleted. Format: + `projects/{project}/locations/{location}/workerPools/{worker_pool}`. + """ + + completeTime = _messages.StringField(1) + createTime = _messages.StringField(2) + workerPool = _messages.StringField(3) + + +class Empty(_messages.Message): + r"""A generic empty message that you can re-use to avoid defining duplicated + empty messages in your APIs. A typical example is to use it as the request + or the response type of an API method. For instance: service Foo { rpc + Bar(google.protobuf.Empty) returns (google.protobuf.Empty); } The JSON + representation for `Empty` is empty JSON object `{}`. + """ + + +class FailureInfo(_messages.Message): + r"""A fatal problem encountered during the execution of the build. + + Enums: + TypeValueValuesEnum: The name of the failure. + + Fields: + detail: Explains the failure issue in more detail using hard-coded text. + type: The name of the failure. + """ + class TypeValueValuesEnum(_messages.Enum): + r"""The name of the failure. + + Values: + FAILURE_TYPE_UNSPECIFIED: Type unspecified + PUSH_FAILED: Unable to push the image to the repository. + PUSH_IMAGE_NOT_FOUND: Final image not found. + PUSH_NOT_AUTHORIZED: Unauthorized push of the final image. + LOGGING_FAILURE: Backend logging failures. Should retry. + USER_BUILD_STEP: A build step has failed. + FETCH_SOURCE_FAILED: The source fetching has failed. + """ + FAILURE_TYPE_UNSPECIFIED = 0 + PUSH_FAILED = 1 + PUSH_IMAGE_NOT_FOUND = 2 + PUSH_NOT_AUTHORIZED = 3 + LOGGING_FAILURE = 4 + USER_BUILD_STEP = 5 + FETCH_SOURCE_FAILED = 6 + + detail = _messages.StringField(1) + type = _messages.EnumField('TypeValueValuesEnum', 2) + + +class FileHashes(_messages.Message): + r"""Container message for hashes of byte content of files, used in + SourceProvenance messages to verify integrity of source input to the build. + + Fields: + fileHash: Collection of file hashes. + """ + + fileHash = _messages.MessageField('Hash', 1, repeated=True) -class CloudbuildProjectsTriggersWebhookRequest(_messages.Message): - r"""A CloudbuildProjectsTriggersWebhookRequest object. +class GitFileSource(_messages.Message): + r"""GitFileSource describes a file within a (possibly remote) code + repository. + + Enums: + RepoTypeValueValuesEnum: See RepoType above. Fields: - httpBody: A HttpBody resource to be passed as the request body. - projectId: Project in which the specified trigger lives - secret: Secret token used for authorization if an OAuth token isn't - provided. - trigger: Name of the trigger to run the payload against + bitbucketServerConfig: The full resource name of the bitbucket server + config. Format: + `projects/{project}/locations/{location}/bitbucketServerConfigs/{id}`. + githubEnterpriseConfig: The full resource name of the github enterprise + config. Format: + `projects/{project}/locations/{location}/githubEnterpriseConfigs/{id}`. + `projects/{project}/githubEnterpriseConfigs/{id}`. + path: The path of the file, with the repo root as the root of the path. + repoType: See RepoType above. + revision: The branch, tag, arbitrary ref, or SHA version of the repo to + use when resolving the filename (optional). This field respects the same + syntax/resolution as described here: https://git- + scm.com/docs/gitrevisions If unspecified, the revision from which the + trigger invocation originated is assumed to be the revision from which + to read the specified path. + uri: The URI of the repo (optional). If unspecified, the repo from which + the trigger invocation originated is assumed to be the repo from which + to read the specified path. """ + class RepoTypeValueValuesEnum(_messages.Enum): + r"""See RepoType above. - httpBody = _messages.MessageField('HttpBody', 1) - projectId = _messages.StringField(2, required=True) - secret = _messages.StringField(3) - trigger = _messages.StringField(4, required=True) + Values: + UNKNOWN: The default, unknown repo type. + CLOUD_SOURCE_REPOSITORIES: A Google Cloud Source Repositories-hosted + repo. + GITHUB: A GitHub-hosted repo not necessarily on "github.com" (i.e. + GitHub Enterprise). + BITBUCKET_SERVER: A Bitbucket Server-hosted repo. + """ + UNKNOWN = 0 + CLOUD_SOURCE_REPOSITORIES = 1 + GITHUB = 2 + BITBUCKET_SERVER = 3 + bitbucketServerConfig = _messages.StringField(1) + githubEnterpriseConfig = _messages.StringField(2) + path = _messages.StringField(3) + repoType = _messages.EnumField('RepoTypeValueValuesEnum', 4) + revision = _messages.StringField(5) + uri = _messages.StringField(6) -class Empty(_messages.Message): - r"""A generic empty message that you can re-use to avoid defining duplicated - empty messages in your APIs. A typical example is to use it as the request - or the response type of an API method. For instance: service Foo { rpc - Bar(google.protobuf.Empty) returns (google.protobuf.Empty); } The JSON - representation for `Empty` is empty JSON object `{}`. + +class GitHubEnterpriseConfig(_messages.Message): + r"""GitHubEnterpriseConfig represents a configuration for a GitHub + Enterprise server. + + Fields: + appId: Required. The GitHub app id of the Cloud Build app on the GitHub + Enterprise server. + createTime: Output only. Time when the installation was associated with + the project. + displayName: Name to display for this config. + hostUrl: The URL of the github enterprise host the configuration is for. + name: Optional. The full resource name for the GitHubEnterpriseConfig For + example: "projects/{$project_id}/githubEnterpriseConfigs/{$config_id}" + peeredNetwork: Optional. The network to be used when reaching out to the + GitHub Enterprise server. The VPC network must be enabled for private + service connection. This should be set if the GitHub Enterprise server + is hosted on-premises and not reachable by public internet. If this + field is left empty, no network peering will occur and calls to the + GitHub Enterprise server will be made over the public internet. Must be + in the format `projects/{project}/global/networks/{network}`, where + {project} is a project number or id and {network} is the name of a VPC + network in the project. + secrets: Names of secrets in Secret Manager. + sslCa: Optional. SSL certificate to use for requests to GitHub Enterprise. + webhookKey: The key that should be attached to webhook calls to the + ReceiveWebhook endpoint. """ + appId = _messages.IntegerField(1) + createTime = _messages.StringField(2) + displayName = _messages.StringField(3) + hostUrl = _messages.StringField(4) + name = _messages.StringField(5) + peeredNetwork = _messages.StringField(6) + secrets = _messages.MessageField('GitHubEnterpriseSecrets', 7) + sslCa = _messages.StringField(8) + webhookKey = _messages.StringField(9) -class FileHashes(_messages.Message): - r"""Container message for hashes of byte content of files, used in - SourceProvenance messages to verify integrity of source input to the build. + +class GitHubEnterpriseSecrets(_messages.Message): + r"""GitHubEnterpriseSecrets represents the names of all necessary secrets in + Secret Manager for a GitHub Enterprise server. Format is: + projects//secrets/. Fields: - fileHash: Collection of file hashes. + oauthClientIdName: The resource name for the OAuth client ID secret in + Secret Manager. + oauthClientIdVersionName: The resource name for the OAuth client ID secret + version in Secret Manager. + oauthSecretName: The resource name for the OAuth secret in Secret Manager. + oauthSecretVersionName: The resource name for the OAuth secret secret + version in Secret Manager. + privateKeyName: The resource name for the private key secret. + privateKeyVersionName: The resource name for the private key secret + version. + webhookSecretName: The resource name for the webhook secret in Secret + Manager. + webhookSecretVersionName: The resource name for the webhook secret secret + version in Secret Manager. """ - fileHash = _messages.MessageField('Hash', 1, repeated=True) + oauthClientIdName = _messages.StringField(1) + oauthClientIdVersionName = _messages.StringField(2) + oauthSecretName = _messages.StringField(3) + oauthSecretVersionName = _messages.StringField(4) + privateKeyName = _messages.StringField(5) + privateKeyVersionName = _messages.StringField(6) + webhookSecretName = _messages.StringField(7) + webhookSecretVersionName = _messages.StringField(8) class GitHubEventsConfig(_messages.Message): r"""GitHubEventsConfig describes the configuration of a trigger that creates - a build whenever a GitHub event is received. This message is experimental. + a build whenever a GitHub event is received. Fields: + enterpriseConfigResourceName: Optional. The resource name of the github + enterprise config that should be applied to this installation. For + example: "projects/{$project_id}/githubEnterpriseConfigs/{$config_id}" installationId: The installationID that emits the GitHub event. name: Name of the repository. For example: The name for https://github.com/googlecloudplatform/cloud-builders is "cloud- @@ -965,11 +2295,80 @@ class GitHubEventsConfig(_messages.Message): push: filter to match changes in refs like branches, tags. """ - installationId = _messages.IntegerField(1) - name = _messages.StringField(2) - owner = _messages.StringField(3) - pullRequest = _messages.MessageField('PullRequestFilter', 4) - push = _messages.MessageField('PushFilter', 5) + enterpriseConfigResourceName = _messages.StringField(1) + installationId = _messages.IntegerField(2) + name = _messages.StringField(3) + owner = _messages.StringField(4) + pullRequest = _messages.MessageField('PullRequestFilter', 5) + push = _messages.MessageField('PushFilter', 6) + + +class GitRepoSource(_messages.Message): + r"""GitRepoSource describes a repo and ref of a code repository. + + Enums: + RepoTypeValueValuesEnum: See RepoType below. + + Fields: + bitbucketServerConfig: The full resource name of the bitbucket server + config. Format: + `projects/{project}/locations/{location}/bitbucketServerConfigs/{id}`. + githubEnterpriseConfig: The full resource name of the github enterprise + config. Format: + `projects/{project}/locations/{location}/githubEnterpriseConfigs/{id}`. + `projects/{project}/githubEnterpriseConfigs/{id}`. + ref: The branch or tag to use. Must start with "refs/" (required). + repoType: See RepoType below. + uri: The URI of the repo (required). + """ + class RepoTypeValueValuesEnum(_messages.Enum): + r"""See RepoType below. + + Values: + UNKNOWN: The default, unknown repo type. + CLOUD_SOURCE_REPOSITORIES: A Google Cloud Source Repositories-hosted + repo. + GITHUB: A GitHub-hosted repo not necessarily on "github.com" (i.e. + GitHub Enterprise). + BITBUCKET_SERVER: A Bitbucket Server-hosted repo. + """ + UNKNOWN = 0 + CLOUD_SOURCE_REPOSITORIES = 1 + GITHUB = 2 + BITBUCKET_SERVER = 3 + + bitbucketServerConfig = _messages.StringField(1) + githubEnterpriseConfig = _messages.StringField(2) + ref = _messages.StringField(3) + repoType = _messages.EnumField('RepoTypeValueValuesEnum', 4) + uri = _messages.StringField(5) + + +class GoogleDevtoolsCloudbuildV2OperationMetadata(_messages.Message): + r"""Represents the metadata of the long-running operation. + + Fields: + apiVersion: Output only. API version used to start the operation. + createTime: Output only. The time the operation was created. + endTime: Output only. The time the operation finished running. + requestedCancellation: Output only. Identifies whether the user has + requested cancellation of the operation. Operations that have + successfully been cancelled have Operation.error value with a + google.rpc.Status.code of 1, corresponding to `Code.CANCELLED`. + statusMessage: Output only. Human-readable status of the operation, if + any. + target: Output only. Server-defined resource path for the target of the + operation. + verb: Output only. Name of the verb executed by the operation. + """ + + apiVersion = _messages.StringField(1) + createTime = _messages.StringField(2) + endTime = _messages.StringField(3) + requestedCancellation = _messages.BooleanField(4) + statusMessage = _messages.StringField(5) + target = _messages.StringField(6) + verb = _messages.StringField(7) class HTTPDelivery(_messages.Message): @@ -1121,6 +2520,35 @@ class AdditionalProperty(_messages.Message): kmsKeyName = _messages.StringField(2) +class ListBitbucketServerConfigsResponse(_messages.Message): + r"""RPC response object returned by ListBitbucketServerConfigs RPC method. + + Fields: + bitbucketServerConfigs: A list of BitbucketServerConfigs + nextPageToken: A token that can be sent as `page_token` to retrieve the + next page. If this field is omitted, there are no subsequent pages. + """ + + bitbucketServerConfigs = _messages.MessageField( + 'BitbucketServerConfig', 1, repeated=True) + nextPageToken = _messages.StringField(2) + + +class ListBitbucketServerRepositoriesResponse(_messages.Message): + r"""RPC response object returned by the ListBitbucketServerRepositories RPC + method. + + Fields: + bitbucketServerRepositories: List of Bitbucket Server repositories. + nextPageToken: A token that can be sent as `page_token` to retrieve the + next page. If this field is omitted, there are no subsequent pages. + """ + + bitbucketServerRepositories = _messages.MessageField( + 'BitbucketServerRepository', 1, repeated=True) + nextPageToken = _messages.StringField(2) + + class ListBuildTriggersResponse(_messages.Message): r"""Response containing existing `BuildTriggers`. @@ -1147,6 +2575,67 @@ class ListBuildsResponse(_messages.Message): nextPageToken = _messages.StringField(2) +class ListGithubEnterpriseConfigsResponse(_messages.Message): + r"""RPC response object returned by ListGithubEnterpriseConfigs RPC method. + + Fields: + configs: A list of GitHubEnterpriseConfigs + """ + + configs = _messages.MessageField('GitHubEnterpriseConfig', 1, repeated=True) + + +class ListWorkerPoolsResponse(_messages.Message): + r"""Response containing existing `WorkerPools`. + + Fields: + nextPageToken: Continuation token used to page through large result sets. + Provide this value in a subsequent ListWorkerPoolsRequest to return the + next page of results. + workerPools: `WorkerPools` for the specified project. + """ + + nextPageToken = _messages.StringField(1) + workerPools = _messages.MessageField('WorkerPool', 2, repeated=True) + + +class NetworkConfig(_messages.Message): + r"""Defines the network configuration for the pool. + + Enums: + EgressOptionValueValuesEnum: Option to configure network egress for the + workers. + + Fields: + egressOption: Option to configure network egress for the workers. + peeredNetwork: Required. Immutable. The network definition that the + workers are peered to. If this section is left empty, the workers will + be peered to `WorkerPool.project_id` on the service producer network. + Must be in the format `projects/{project}/global/networks/{network}`, + where `{project}` is a project number, such as `12345`, and `{network}` + is the name of a VPC network in the project. See [Understanding network + configuration options](https://cloud.google.com/build/docs/private- + pools/set-up-private-pool-environment) + """ + class EgressOptionValueValuesEnum(_messages.Enum): + r"""Option to configure network egress for the workers. + + Values: + EGRESS_OPTION_UNSPECIFIED: If set, defaults to PUBLIC_EGRESS. + NO_PUBLIC_EGRESS: If set, workers are created without any public + address, which prevents network egress to public IPs unless a network + proxy is configured. + PUBLIC_EGRESS: If set, workers are created with a public address which + allows for public internet egress. + """ + EGRESS_OPTION_UNSPECIFIED = 0 + NO_PUBLIC_EGRESS = 1 + PUBLIC_EGRESS = 2 + + egressOption = _messages.EnumField('EgressOptionValueValuesEnum', 1) + peeredNetwork = _messages.StringField(2) + + class Notification(_messages.Message): r"""Notification is the container which holds the data that is relevant to this particular notification. @@ -1377,6 +2866,117 @@ class AdditionalProperty(_messages.Message): response = _messages.MessageField('ResponseValue', 5) +class OperationMetadata(_messages.Message): + r"""Represents the metadata of the long-running operation. + + Fields: + apiVersion: Output only. API version used to start the operation. + cancelRequested: Output only. Identifies whether the user has requested + cancellation of the operation. Operations that have been cancelled + successfully have Operation.error value with a google.rpc.Status.code of + 1, corresponding to `Code.CANCELLED`. + createTime: Output only. The time the operation was created. + endTime: Output only. The time the operation finished running. + statusDetail: Output only. Human-readable status of the operation, if any. + target: Output only. Server-defined resource path for the target of the + operation. + verb: Output only. Name of the verb executed by the operation. + """ + + apiVersion = _messages.StringField(1) + cancelRequested = _messages.BooleanField(2) + createTime = _messages.StringField(3) + endTime = _messages.StringField(4) + statusDetail = _messages.StringField(5) + target = _messages.StringField(6) + verb = _messages.StringField(7) + + +class PoolOption(_messages.Message): + r"""Details about how a build should be executed on a `WorkerPool`. See + [running builds in a private + pool](https://cloud.google.com/build/docs/private-pools/run-builds-in- + private-pool) for more information. + + Fields: + name: The `WorkerPool` resource to execute the build on. You must have + `cloudbuild.workerpools.use` on the project hosting the WorkerPool. + Format + projects/{project}/locations/{location}/workerPools/{workerPoolId} + """ + + name = _messages.StringField(1) + + +class PrivatePoolV1Config(_messages.Message): + r"""Configuration for a V1 `PrivatePool`. + + Fields: + networkConfig: Network configuration for the pool. + workerConfig: Machine configuration for the workers in the pool. + """ + + networkConfig = _messages.MessageField('NetworkConfig', 1) + workerConfig = _messages.MessageField('WorkerConfig', 2) + + +class ProcessAppManifestCallbackOperationMetadata(_messages.Message): + r"""Metadata for `ProcessAppManifestCallback` operation. + + Fields: + completeTime: Time the operation was completed. + createTime: Time the operation was created. + githubEnterpriseConfig: The resource name of the GitHubEnterprise to be + created. Format: + `projects/{project}/locations/{location}/githubEnterpriseConfigs/{id}`. + """ + + completeTime = _messages.StringField(1) + createTime = _messages.StringField(2) + githubEnterpriseConfig = _messages.StringField(3) + + +class PubsubConfig(_messages.Message): + r"""PubsubConfig describes the configuration of a trigger that creates a + build whenever a Pub/Sub message is published. + + Enums: + StateValueValuesEnum: Potential issues with the underlying Pub/Sub + subscription configuration. Only populated on get requests. + + Fields: + serviceAccountEmail: Service account that will make the push request. + state: Potential issues with the underlying Pub/Sub subscription + configuration. Only populated on get requests. + subscription: Output only. Name of the subscription. Format is + `projects/{project}/subscriptions/{subscription}`. + topic: The name of the topic from which this subscription is receiving + messages. Format is `projects/{project}/topics/{topic}`. + """ + class StateValueValuesEnum(_messages.Enum): + r"""Potential issues with the underlying Pub/Sub subscription + configuration. Only populated on get requests. + + Values: + STATE_UNSPECIFIED: The subscription configuration has not been checked. + OK: The Pub/Sub subscription is properly configured. + SUBSCRIPTION_DELETED: The subscription has been deleted. + TOPIC_DELETED: The topic has been deleted. + SUBSCRIPTION_MISCONFIGURED: Some of the subscription's field are + misconfigured. + """ + STATE_UNSPECIFIED = 0 + OK = 1 + SUBSCRIPTION_DELETED = 2 + TOPIC_DELETED = 3 + SUBSCRIPTION_MISCONFIGURED = 4 + + serviceAccountEmail = _messages.StringField(1) + state = _messages.EnumField('StateValueValuesEnum', 2) + subscription = _messages.StringField(3) + topic = _messages.StringField(4) + + class PullRequestFilter(_messages.Message): r"""PullRequestFilter contains filter properties for matching GitHub Pull Requests. @@ -1441,6 +3041,17 @@ class ReceiveTriggerWebhookResponse(_messages.Message): """ +class RemoveBitbucketServerConnectedRepositoryRequest(_messages.Message): + r"""RPC request object accepted by RemoveBitbucketServerConnectedRepository + RPC method. + + Fields: + connectedRepository: The connected repository to remove. + """ + + connectedRepository = _messages.MessageField('BitbucketServerRepositoryId', 1) + + class RepoSource(_messages.Message): r"""Location of the source in a Google Cloud Source Repository. @@ -1545,6 +3156,48 @@ class RetryBuildRequest(_messages.Message): projectId = _messages.StringField(3) +class RunBuildTriggerRequest(_messages.Message): + r"""Specifies a build trigger to run and the source to use. + + Fields: + projectId: Required. ID of the project. + source: Source to build against this trigger. Branch and tag names cannot + consist of regular expressions. + triggerId: Required. ID of the trigger. + """ + + projectId = _messages.StringField(1) + source = _messages.MessageField('RepoSource', 2) + triggerId = _messages.StringField(3) + + +class RunWorkflowCustomOperationMetadata(_messages.Message): + r"""Represents the custom metadata of the RunWorkflow long-running + operation. + + Fields: + apiVersion: Output only. API version used to start the operation. + createTime: Output only. The time the operation was created. + endTime: Output only. The time the operation finished running. + pipelineRunId: Output only. ID of the pipeline run created by RunWorkflow. + requestedCancellation: Output only. Identifies whether the user has + requested cancellation of the operation. Operations that have + successfully been cancelled have Operation.error value with a + google.rpc.Status.code of 1, corresponding to `Code.CANCELLED`. + target: Output only. Server-defined resource path for the target of the + operation. + verb: Output only. Name of the verb executed by the operation. + """ + + apiVersion = _messages.StringField(1) + createTime = _messages.StringField(2) + endTime = _messages.StringField(3) + pipelineRunId = _messages.StringField(4) + requestedCancellation = _messages.BooleanField(5) + target = _messages.StringField(6) + verb = _messages.StringField(7) + + class SMTPDelivery(_messages.Message): r"""SMTPDelivery is the delivery configuration for an SMTP (email) notification. @@ -1675,10 +3328,15 @@ class Source(_messages.Message): Source Repository. storageSource: If provided, get the source from this location in Google Cloud Storage. + storageSourceManifest: If provided, get the source from this manifest in + Google Cloud Storage. This feature is in Preview; see description + [here](https://github.com/GoogleCloudPlatform/cloud- + builders/tree/master/gcs-fetcher). """ repoSource = _messages.MessageField('RepoSource', 1) storageSource = _messages.MessageField('StorageSource', 2) + storageSourceManifest = _messages.MessageField('StorageSourceManifest', 3) class SourceProvenance(_messages.Message): @@ -1708,6 +3366,9 @@ class SourceProvenance(_messages.Message): with any revisions resolved. resolvedStorageSource: A copy of the build's `source.storage_source`, if exists, with any generations resolved. + resolvedStorageSourceManifest: A copy of the build's + `source.storage_source_manifest`, if exists, with any revisions + resolved. This feature is in Preview. """ @encoding.MapUnrecognizedFields('additionalProperties') class FileHashesValue(_messages.Message): @@ -1742,6 +3403,8 @@ class AdditionalProperty(_messages.Message): fileHashes = _messages.MessageField('FileHashesValue', 1) resolvedRepoSource = _messages.MessageField('RepoSource', 2) resolvedStorageSource = _messages.MessageField('StorageSource', 3) + resolvedStorageSourceManifest = _messages.MessageField( + 'StorageSourceManifest', 4) class StandardQueryParameters(_messages.Message): @@ -1866,7 +3529,29 @@ class StorageSource(_messages.Message): generation: Google Cloud Storage generation for the object. If the generation is omitted, the latest generation will be used. object: Google Cloud Storage object containing the source. This object - must be a gzipped archive file (`.tar.gz`) containing source to build. + must be a zipped (`.zip`) or gzipped archive file (`.tar.gz`) containing + source to build. + """ + + bucket = _messages.StringField(1) + generation = _messages.IntegerField(2) + object = _messages.StringField(3) + + +class StorageSourceManifest(_messages.Message): + r"""Location of the source manifest in Google Cloud Storage. This feature is + in Preview; see description + [here](https://github.com/GoogleCloudPlatform/cloud- + builders/tree/master/gcs-fetcher). + + Fields: + bucket: Google Cloud Storage bucket containing the source manifest (see + [Bucket Name Requirements](https://cloud.google.com/storage/docs/bucket- + naming#requirements)). + generation: Google Cloud Storage generation for the object. If the + generation is omitted, the latest generation will be used. + object: Google Cloud Storage object containing the source manifest. This + object must be a JSON file. """ bucket = _messages.StringField(1) @@ -1886,6 +3571,68 @@ class TimeSpan(_messages.Message): startTime = _messages.StringField(2) +class UpdateBitbucketServerConfigOperationMetadata(_messages.Message): + r"""Metadata for `UpdateBitbucketServerConfig` operation. + + Fields: + bitbucketServerConfig: The resource name of the BitbucketServerConfig to + be updated. Format: + `projects/{project}/locations/{location}/bitbucketServerConfigs/{id}`. + completeTime: Time the operation was completed. + createTime: Time the operation was created. + """ + + bitbucketServerConfig = _messages.StringField(1) + completeTime = _messages.StringField(2) + createTime = _messages.StringField(3) + + +class UpdateGitHubEnterpriseConfigOperationMetadata(_messages.Message): + r"""Metadata for `UpdateGitHubEnterpriseConfig` operation. + + Fields: + completeTime: Time the operation was completed. + createTime: Time the operation was created. + githubEnterpriseConfig: The resource name of the GitHubEnterprise to be + updated. Format: + `projects/{project}/locations/{location}/githubEnterpriseConfigs/{id}`. + """ + + completeTime = _messages.StringField(1) + createTime = _messages.StringField(2) + githubEnterpriseConfig = _messages.StringField(3) + + +class UpdateGitLabConfigOperationMetadata(_messages.Message): + r"""Metadata for `UpdateGitLabConfig` operation. + + Fields: + completeTime: Time the operation was completed. + createTime: Time the operation was created. + gitlabConfig: The resource name of the GitLabConfig to be created. Format: + `projects/{project}/locations/{location}/gitlabConfigs/{id}`. + """ + + completeTime = _messages.StringField(1) + createTime = _messages.StringField(2) + gitlabConfig = _messages.StringField(3) + + +class UpdateWorkerPoolOperationMetadata(_messages.Message): + r"""Metadata for the `UpdateWorkerPool` operation. + + Fields: + completeTime: Time the operation was completed. + createTime: Time the operation was created. + workerPool: The resource name of the `WorkerPool` being updated. Format: + `projects/{project}/locations/{location}/workerPools/{worker_pool}`. + """ + + completeTime = _messages.StringField(1) + createTime = _messages.StringField(2) + workerPool = _messages.StringField(3) + + class Volume(_messages.Message): r"""Volume describes a Docker container volume which is mounted into build steps in order to persist files across build step execution. @@ -1903,6 +3650,184 @@ class Volume(_messages.Message): path = _messages.StringField(2) +class Warning(_messages.Message): + r"""A non-fatal problem encountered during the execution of the build. + + Enums: + PriorityValueValuesEnum: The priority for this warning. + + Fields: + priority: The priority for this warning. + text: Explanation of the warning generated. + """ + class PriorityValueValuesEnum(_messages.Enum): + r"""The priority for this warning. + + Values: + PRIORITY_UNSPECIFIED: Should not be used. + INFO: e.g. deprecation warnings and alternative feature highlights. + WARNING: e.g. automated detection of possible issues with the build. + ALERT: e.g. alerts that a feature used in the build is pending removal + """ + PRIORITY_UNSPECIFIED = 0 + INFO = 1 + WARNING = 2 + ALERT = 3 + + priority = _messages.EnumField('PriorityValueValuesEnum', 1) + text = _messages.StringField(2) + + +class WebhookConfig(_messages.Message): + r"""WebhookConfig describes the configuration of a trigger that creates a + build whenever a webhook is sent to a trigger's webhook URL. + + Enums: + StateValueValuesEnum: Potential issues with the underlying Pub/Sub + subscription configuration. Only populated on get requests. + + Fields: + secret: Required. Resource name for the secret required as a URL + parameter. + state: Potential issues with the underlying Pub/Sub subscription + configuration. Only populated on get requests. + """ + class StateValueValuesEnum(_messages.Enum): + r"""Potential issues with the underlying Pub/Sub subscription + configuration. Only populated on get requests. + + Values: + STATE_UNSPECIFIED: The webhook auth configuration not been checked. + OK: The auth configuration is properly setup. + SECRET_DELETED: The secret provided in auth_method has been deleted. + """ + STATE_UNSPECIFIED = 0 + OK = 1 + SECRET_DELETED = 2 + + secret = _messages.StringField(1) + state = _messages.EnumField('StateValueValuesEnum', 2) + + +class WorkerConfig(_messages.Message): + r"""Defines the configuration to be used for creating workers in the pool. + + Fields: + diskSizeGb: Size of the disk attached to the worker, in GB. See [Worker + pool config file](https://cloud.google.com/build/docs/private- + pools/worker-pool-config-file-schema). Specify a value of up to 1000. If + `0` is specified, Cloud Build will use a standard disk size. + machineType: Machine type of a worker, such as `e2-medium`. See [Worker + pool config file](https://cloud.google.com/build/docs/private- + pools/worker-pool-config-file-schema). If left blank, Cloud Build will + use a sensible default. + """ + + diskSizeGb = _messages.IntegerField(1) + machineType = _messages.StringField(2) + + +class WorkerPool(_messages.Message): + r"""Configuration for a `WorkerPool`. Cloud Build owns and maintains a pool + of workers for general use and have no access to a project's private + network. By default, builds submitted to Cloud Build will use a worker from + this pool. If your build needs access to resources on a private network, + create and use a `WorkerPool` to run your builds. Private `WorkerPool`s give + your builds access to any single VPC network that you administer, including + any on-prem resources connected to that VPC network. For an overview of + private pools, see [Private pools + overview](https://cloud.google.com/build/docs/private-pools/private-pools- + overview). + + Enums: + StateValueValuesEnum: Output only. `WorkerPool` state. + + Messages: + AnnotationsValue: User specified annotations. See + https://google.aip.dev/128#annotations for more details such as format + and size limitations. + + Fields: + annotations: User specified annotations. See + https://google.aip.dev/128#annotations for more details such as format + and size limitations. + createTime: Output only. Time at which the request to create the + `WorkerPool` was received. + deleteTime: Output only. Time at which the request to delete the + `WorkerPool` was received. + displayName: A user-specified, human-readable name for the `WorkerPool`. + If provided, this value must be 1-63 characters. + etag: Output only. Checksum computed by the server. May be sent on update + and delete requests to ensure that the client has an up-to-date value + before proceeding. + name: Output only. The resource name of the `WorkerPool`, with format + `projects/{project}/locations/{location}/workerPools/{worker_pool}`. The + value of `{worker_pool}` is provided by `worker_pool_id` in + `CreateWorkerPool` request and the value of `{location}` is determined + by the endpoint accessed. + privatePoolV1Config: Legacy Private Pool configuration. + state: Output only. `WorkerPool` state. + uid: Output only. A unique identifier for the `WorkerPool`. + updateTime: Output only. Time at which the request to update the + `WorkerPool` was received. + """ + class StateValueValuesEnum(_messages.Enum): + r"""Output only. `WorkerPool` state. + + Values: + STATE_UNSPECIFIED: State of the `WorkerPool` is unknown. + CREATING: `WorkerPool` is being created. + RUNNING: `WorkerPool` is running. + DELETING: `WorkerPool` is being deleted: cancelling builds and draining + workers. + DELETED: `WorkerPool` is deleted. + UPDATING: `WorkerPool` is being updated; new builds cannot be run. + """ + STATE_UNSPECIFIED = 0 + CREATING = 1 + RUNNING = 2 + DELETING = 3 + DELETED = 4 + UPDATING = 5 + + @encoding.MapUnrecognizedFields('additionalProperties') + class AnnotationsValue(_messages.Message): + r"""User specified annotations. See https://google.aip.dev/128#annotations + for more details such as format and size limitations. + + Messages: + AdditionalProperty: An additional property for a AnnotationsValue + object. + + Fields: + additionalProperties: Additional properties of type AnnotationsValue + """ + class AdditionalProperty(_messages.Message): + r"""An additional property for a AnnotationsValue object. + + Fields: + key: Name of the additional property. + value: A string attribute. + """ + + key = _messages.StringField(1) + value = _messages.StringField(2) + + additionalProperties = _messages.MessageField( + 'AdditionalProperty', 1, repeated=True) + + annotations = _messages.MessageField('AnnotationsValue', 1) + createTime = _messages.StringField(2) + deleteTime = _messages.StringField(3) + displayName = _messages.StringField(4) + etag = _messages.StringField(5) + name = _messages.StringField(6) + privatePoolV1Config = _messages.MessageField('PrivatePoolV1Config', 7) + state = _messages.EnumField('StateValueValuesEnum', 8) + uid = _messages.StringField(9) + updateTime = _messages.StringField(10) + + encoding.AddCustomJsonFieldMapping( StandardQueryParameters, 'f__xgafv', '$.xgafv') encoding.AddCustomJsonEnumMapping( diff --git a/sdks/python/apache_beam/runners/portability/sdk_container_builder.py b/sdks/python/apache_beam/runners/portability/sdk_container_builder.py index 7b4e81f21fae..d06b005a4f0d 100644 --- a/sdks/python/apache_beam/runners/portability/sdk_container_builder.py +++ b/sdks/python/apache_beam/runners/portability/sdk_container_builder.py @@ -45,6 +45,7 @@ from apache_beam.options.pipeline_options import GoogleCloudOptions from apache_beam.options.pipeline_options import PipelineOptions # pylint: disable=unused-import from apache_beam.options.pipeline_options import SetupOptions +from apache_beam.options.pipeline_options import WorkerOptions from apache_beam.portability import common_urns from apache_beam.portability.api import beam_runner_api_pb2 from apache_beam.runners.dataflow.internal.clients import cloudbuild @@ -74,7 +75,7 @@ def __init__(self, options): beam_version.__version__ if 'dev' not in beam_version.__version__ else 'latest') self._base_image = ( - self._options.view_as(SetupOptions).prebuild_sdk_container_base_image or + self._options.view_as(WorkerOptions).sdk_container_image or 'apache/beam_python%s.%s_sdk:%s' % (sys.version_info[0], sys.version_info[1], version)) self._temp_src_dir = None @@ -203,6 +204,8 @@ class _SdkContainerImageCloudBuilder(SdkContainerImageBuilder): def __init__(self, options): super().__init__(options) self._google_cloud_options = options.view_as(GoogleCloudOptions) + self._cloud_build_machine_type = self._get_cloud_build_machine_type_enum( + options.view_as(SetupOptions).cloud_build_machine_type) if self._google_cloud_options.no_auth: credentials = None else: @@ -243,6 +246,9 @@ def _invoke_docker_build_and_push(self, container_image_name): self._upload_to_gcs(tarball_path, gcs_location) build = cloudbuild.Build() + if self._cloud_build_machine_type: + build.options = cloudbuild.BuildOptions() + build.options.machineType = self._cloud_build_machine_type build.steps = [] step = cloudbuild.BuildStep() step.name = 'gcr.io/kaniko-project/executor:latest' @@ -276,6 +282,7 @@ def _invoke_docker_build_and_push(self, container_image_name): cloudbuild.CloudbuildProjectsBuildsGetRequest( id=build_id, projectId=project_id)) while response.status in [cloudbuild.Build.StatusValueValuesEnum.QUEUED, + cloudbuild.Build.StatusValueValuesEnum.PENDING, cloudbuild.Build.StatusValueValuesEnum.WORKING]: time.sleep(10) response = self._cloudbuild_client.projects_builds.Get( @@ -338,3 +345,24 @@ def _get_gcs_bucket_and_name(gcs_location): def _make_tarfile(output_filename, source_dir): with tarfile.open(output_filename, "w:gz") as tar: tar.add(source_dir, arcname=SOURCE_FOLDER) + + @staticmethod + def _get_cloud_build_machine_type_enum(machine_type: str): + if not machine_type: + return None + mappings = { + 'n1-highcpu-8': cloudbuild.BuildOptions.MachineTypeValueValuesEnum. + N1_HIGHCPU_8, + 'n1-highcpu-32': cloudbuild.BuildOptions.MachineTypeValueValuesEnum. + N1_HIGHCPU_32, + 'e2-highcpu-8': cloudbuild.BuildOptions.MachineTypeValueValuesEnum. + E2_HIGHCPU_8, + 'e2-highcpu-32': cloudbuild.BuildOptions.MachineTypeValueValuesEnum. + E2_HIGHCPU_32 + } + if machine_type.lower() in mappings: + return mappings[machine_type.lower()] + else: + raise ValueError( + 'Unknown Cloud Build Machine Type option, please specify one of ' + '[n1-highcpu-8, n1-highcpu-32, e2-highcpu-8, e2-highcpu-32].') From d6e64906f2b9b940aefa0265e8126f14c2535b20 Mon Sep 17 00:00:00 2001 From: nancyxu123 Date: Wed, 9 Mar 2022 09:30:45 -0800 Subject: [PATCH 60/68] Merge pull request #17036 from [BEAM-12164] Convert all static instances to be transient in the connector in order to enable concurrent testing * Convert all static instances to be transient in the connector in order to enable concurrent testing * Initialized fields to null * nullness * Suppress uninitialized warnings * Remove resetting dao factory fields in SpannerChangeStreamErrorTest.java * Add validation package --- .../changestreams/action/ActionFactory.java | 16 ++++++++-------- .../spanner/changestreams/dao/DaoFactory.java | 10 +++++----- .../changestreams/mapper/MapperFactory.java | 8 ++++---- .../SpannerChangeStreamErrorTest.java | 17 ----------------- 4 files changed, 17 insertions(+), 34 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ActionFactory.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ActionFactory.java index c6cd7256ff60..5277c368f7c6 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ActionFactory.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ActionFactory.java @@ -29,17 +29,17 @@ * Factory class for creating instances that will handle each type of record within a change stream * query. The instances created are all singletons. */ -// static fields are un-initialized, because we start them during the first fetch call (with the -// singleton pattern) -@SuppressWarnings("initialization.static.fields.uninitialized") +// transient fields are un-initialized, because we start them during the first fetch call (with the +// singleton pattern). +@SuppressWarnings("initialization.fields.uninitialized") public class ActionFactory implements Serializable { private static final long serialVersionUID = -4060958761369602619L; - private static DataChangeRecordAction dataChangeRecordActionInstance; - private static HeartbeatRecordAction heartbeatRecordActionInstance; - private static ChildPartitionsRecordAction childPartitionsRecordActionInstance; - private static QueryChangeStreamAction queryChangeStreamActionInstance; - private static DetectNewPartitionsAction detectNewPartitionsActionInstance; + private transient DataChangeRecordAction dataChangeRecordActionInstance; + private transient HeartbeatRecordAction heartbeatRecordActionInstance; + private transient ChildPartitionsRecordAction childPartitionsRecordActionInstance; + private transient QueryChangeStreamAction queryChangeStreamActionInstance; + private transient DetectNewPartitionsAction detectNewPartitionsActionInstance; /** * Creates and returns a singleton instance of an action class capable of processing {@link diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dao/DaoFactory.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dao/DaoFactory.java index 28655f69a26a..7c94720c7875 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dao/DaoFactory.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dao/DaoFactory.java @@ -27,17 +27,17 @@ * Factory class to create data access objects to perform change stream queries and access the * metadata tables. The instances created are all singletons. */ -// static fields are un-initialized, because we start them during the first fetch call (with the +// transient fields are un-initialized, because we start them during the first fetch call (with the // singleton pattern) // nullness checks for metadata instance and database are handled in the constructor -@SuppressWarnings({"initialization.static.fields.uninitialized", "nullness"}) +@SuppressWarnings({"initialization.fields.uninitialized", "nullness"}) public class DaoFactory implements Serializable { private static final long serialVersionUID = 7929063669009832487L; - private static PartitionMetadataAdminDao partitionMetadataAdminDao; - private static PartitionMetadataDao partitionMetadataDaoInstance; - private static ChangeStreamDao changeStreamDaoInstance; + private transient PartitionMetadataAdminDao partitionMetadataAdminDao; + private transient PartitionMetadataDao partitionMetadataDaoInstance; + private transient ChangeStreamDao changeStreamDaoInstance; private final SpannerConfig changeStreamSpannerConfig; private final SpannerConfig metadataSpannerConfig; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/mapper/MapperFactory.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/mapper/MapperFactory.java index 52274635c922..6ad51d23a1be 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/mapper/MapperFactory.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/mapper/MapperFactory.java @@ -23,15 +23,15 @@ * Factory class for creating instances that will map a struct to a connector model. The instances * created are all singletons. */ -// static fields are un-initialized, because we start them during the first fetch call (with the +// transient fields are un-initialized, because we start them during the first fetch call (with the // singleton pattern) -@SuppressWarnings("initialization.static.fields.uninitialized") +@SuppressWarnings("initialization.fields.uninitialized") public class MapperFactory implements Serializable { private static final long serialVersionUID = -813434573067800902L; - private static ChangeStreamRecordMapper changeStreamRecordMapperInstance; - private static PartitionMetadataMapper partitionMetadataMapperInstance; + private transient ChangeStreamRecordMapper changeStreamRecordMapperInstance; + private transient PartitionMetadataMapper partitionMetadataMapperInstance; /** * Creates and returns a singleton instance of a mapper class capable of transforming a {@link diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/SpannerChangeStreamErrorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/SpannerChangeStreamErrorTest.java index 050993391b78..4789c6c56751 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/SpannerChangeStreamErrorTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/SpannerChangeStreamErrorTest.java @@ -54,7 +54,6 @@ import org.apache.beam.sdk.Pipeline.PipelineExecutionException; import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; -import org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.DaoFactory; import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionMetadata.State; import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.testing.TestPipeline; @@ -102,7 +101,6 @@ public void tearDown() throws NoSuchFieldException, IllegalAccessException { serviceHelper.reset(); serviceHelper.stop(); mockSpannerService.reset(); - resetDaoFactoryFields(); } @Test @@ -427,21 +425,6 @@ private SpannerConfig getSpannerConfig() { .withDatabaseId(TEST_DATABASE); } - private static void resetDaoFactoryFields() throws NoSuchFieldException, IllegalAccessException { - java.lang.reflect.Field partitionMetadataAdminDaoField = - DaoFactory.class.getDeclaredField("partitionMetadataAdminDao"); - partitionMetadataAdminDaoField.setAccessible(true); - partitionMetadataAdminDaoField.set(null, null); - java.lang.reflect.Field partitionMetadataDaoInstanceField = - DaoFactory.class.getDeclaredField("partitionMetadataDaoInstance"); - partitionMetadataDaoInstanceField.setAccessible(true); - partitionMetadataDaoInstanceField.set(null, null); - java.lang.reflect.Field changeStreamDaoInstanceField = - DaoFactory.class.getDeclaredField("changeStreamDaoInstance"); - changeStreamDaoInstanceField.setAccessible(true); - changeStreamDaoInstanceField.set(null, null); - } - private static final ResultSetMetadata PARTITION_METADATA_RESULT_SET_METADATA = ResultSetMetadata.newBuilder() .setRowType( From 8de937a6ad6066d3e91a0c9e72b2bff2505b9908 Mon Sep 17 00:00:00 2001 From: reuvenlax Date: Wed, 9 Mar 2022 09:43:16 -0800 Subject: [PATCH 61/68] fix variable reference (#16991) --- .../org/apache/beam/sdk/transforms/ParDoLifecycleTest.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java index 337b9315b992..1184ad2120ca 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java @@ -386,10 +386,7 @@ private static void validate(CallState... requiredCallStates) { List states = Arrays.stream(requiredCallStates).collect(Collectors.toList()); assertThat( - "At least one bundle should contain " - + states - + ", got " - + ExceptionThrowingFn.callStateMap.values(), + "At least one bundle should contain " + states + ", got " + callStates.values(), callStates.values().stream() .anyMatch(tracker -> tracker.callStateVisited.equals(states))); } From ae104e29226908c1ff076c6159bdab061f1623ea Mon Sep 17 00:00:00 2001 From: Nancy Date: Wed, 9 Mar 2022 12:50:38 -0800 Subject: [PATCH 62/68] Committed changes --- .../changestreams/it/SpannerChangeStreamOrderedWithinKeyIT.java | 1 + .../it/SpannerChangeStreamTransactionBoundariesIT.java | 2 ++ 2 files changed, 3 insertions(+) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyIT.java index 53566f0d6489..9758cad812d9 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyIT.java @@ -83,6 +83,7 @@ public static void setup() throws InterruptedException, ExecutionException, Time @Test public void testOrderedWithinKey() { + LOG.info("Test pipeline: " + pipeline.toString()); final SpannerConfig spannerConfig = SpannerConfig.create() .withProjectId(projectId) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamTransactionBoundariesIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamTransactionBoundariesIT.java index 71b233e4daea..eb5b9e3ba151 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamTransactionBoundariesIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamTransactionBoundariesIT.java @@ -85,11 +85,13 @@ public static void setup() throws InterruptedException, ExecutionException, Time @Test public void testTransactionBoundaries() { + LOG.info("Test pipeline: " + pipeline.toString()); final SpannerConfig spannerConfig = SpannerConfig.create() .withProjectId(projectId) .withInstanceId(instanceId) .withDatabaseId(databaseId); + // Commit a initial transaction to get the timestamp to start reading from. List mutations = new ArrayList<>(); mutations.add(insertRecordMutation(0, "FirstName0", "LastName0")); From da52b07ade748e155a3fe19007722d75c2147ca5 Mon Sep 17 00:00:00 2001 From: Nancy Date: Wed, 9 Mar 2022 16:56:37 -0800 Subject: [PATCH 63/68] Print more logging --- ...hangeStreamOrderedWithinKeyGloballyIT.java | 73 +++++++++---------- 1 file changed, 34 insertions(+), 39 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java index f50b1b6e7e59..1c83a85c3116 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java @@ -32,7 +32,7 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; import javax.annotation.Nullable; -import org.apache.beam.sdk.coders.BooleanCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.DataChangeRecord; @@ -130,7 +130,7 @@ public void testOrderedWithinKey() { // per key. com.google.cloud.Timestamp endTimestamp = writeTransactionsToDatabase(); - LOG.debug( + LOG.info( "Reading change streams from {} to {}", startTimestamp.toString(), endTimestamp.toString()); final PCollection tokens = @@ -146,7 +146,7 @@ public void testOrderedWithinKey() { .apply(ParDo.of(new KeyByIdFn())) .apply(ParDo.of(new KeyValueByCommitTimestampAndTransactionIdFn<>())) .apply( - ParDo.of(new BufferKeyUntilOutputTimestamp(endTimestamp, timeIncrementInSeconds))) + ParDo.of(new BufferKeyUntilOutputTimestamp(timeIncrementInSeconds))) .apply(ParDo.of(new ToStringFn())); // Assert that the returned PCollection contains one entry per key for the committed @@ -279,16 +279,9 @@ private static class BufferKeyUntilOutputTimestamp private static final long serialVersionUID = 5050535558953049259L; private final long incrementIntervalInSeconds; - private final @Nullable Instant pipelineEndTime; - private BufferKeyUntilOutputTimestamp( - @Nullable com.google.cloud.Timestamp endTimestamp, long incrementIntervalInSeconds) { + private BufferKeyUntilOutputTimestamp(long incrementIntervalInSeconds) { this.incrementIntervalInSeconds = incrementIntervalInSeconds; - if (endTimestamp != null) { - this.pipelineEndTime = new Instant(endTimestamp.toSqlTimestamp()); - } else { - pipelineEndTime = null; - } } @SuppressWarnings("unused") @@ -302,8 +295,8 @@ private BufferKeyUntilOutputTimestamp( buffer = StateSpecs.bag(); @SuppressWarnings("unused") - @StateId("keySeen") - private final StateSpec> keySeen = StateSpecs.value(BooleanCoder.of()); + @StateId("seenKey") + private final StateSpec> seenKey = StateSpecs.value(StringUtf8Coder.of()); @ProcessElement public void process( @@ -314,20 +307,20 @@ public void process( BagState> buffer, @TimerId("timer") Timer timer, - @StateId("keySeen") ValueState keySeen) { + @StateId("seenKey") ValueState seenKey) { buffer.add(element.getValue()); // Only set the timer if this is the first time we are receiving a data change record // with this key. - Boolean hasKeyBeenSeen = keySeen.read(); + String hasKeyBeenSeen = seenKey.read(); if (hasKeyBeenSeen == null) { Instant commitTimestamp = new Instant(element.getValue().getValue().getCommitTimestamp().toSqlTimestamp()); Instant outputTimestamp = commitTimestamp.plus(Duration.standardSeconds(incrementIntervalInSeconds)); - LOG.debug("Setting timer at {} for key {}", outputTimestamp.toString(), element.getKey()); + LOG.info("Setting timer at {} for key {}", outputTimestamp.toString(), element.getKey()); timer.set(outputTimestamp); - keySeen.write(true); + seenKey.write(element.getKey()); } } @@ -337,7 +330,9 @@ public void onExpiry( @StateId("buffer") BagState> buffer, - @TimerId("timer") Timer timer) { + @TimerId("timer") Timer timer, + @StateId("seenKey") ValueState seenKey) { + Instant timerContextTimestamp = context.timestamp(); if (!buffer.isEmpty().read()) { final List> records = @@ -359,18 +354,18 @@ public void onExpiry( // have been processed and successfully committed. Since the timer fires when the // watermark passes the expiration time, we should only output records with event time // < expiration time. - if (recordCommitTimestamp.isBefore(context.timestamp())) { - LOG.debug( + if (recordCommitTimestamp.isBefore(timerContextTimestamp)) { + LOG.info( "Outputting record with key {} and value \"{}\" at expiration timestamp {}", record.getValue().getMods().get(0).getKeysJson(), recordString, - context.timestamp().toString()); + timerContextTimestamp.toString()); recordsToOutput.add(record); } else { - LOG.debug( + LOG.info( "Expired at {} but adding record with key {} and value {} back to buffer " + "due to commit timestamp {}", - context.timestamp().toString(), + timerContextTimestamp.toString(), record.getValue().getMods().get(0).getKeysJson(), recordString, recordCommitTimestamp.toString()); @@ -384,26 +379,26 @@ public void onExpiry( KV.of( recordsToOutput.get(0).getValue().getMods().get(0).getKeysJson(), recordsToOutput), - context.timestamp()); - LOG.debug( - "Expired at {}, outputting records for key {}", - context.timestamp().toString(), + timerContextTimestamp); + LOG.info( + "Expired at {}, outputting records for key and context timestamp {}", + timerContextTimestamp.toString(), recordsToOutput.get(0).getValue().getMods().get(0).getKeysJson()); } else { - LOG.debug("Expired at {} with no records", context.timestamp().toString()); + LOG.info("Expired at {} with no records", timerContextTimestamp.toString()); } } Instant nextTimer = - context.timestamp().plus(Duration.standardSeconds(incrementIntervalInSeconds)); - if (pipelineEndTime == null || context.timestamp().isBefore(pipelineEndTime)) { - // If the current timer's timestamp is before the pipeline end time, or there is no - // pipeline end time, we still have data left to process. - LOG.debug("Setting next timer to {}", nextTimer.toString()); + timerContextTimestamp.plus(Duration.standardSeconds(incrementIntervalInSeconds)); + String keyString = seenKey.read(); + if (buffer.isEmpty() != null && !buffer.isEmpty().read()) { + LOG.info("Setting next timer to {} for key ", nextTimer.toString(), keyString); timer.set(nextTimer); } else { - LOG.debug( - "Timer not being set as exceeded pipeline end time: " + pipelineEndTime.toString()); + LOG.info( + "Timer not being set since the buffer is empty for key {} ", keyString); + seenKey.clear(); } } } @@ -500,27 +495,27 @@ private com.google.cloud.Timestamp writeTransactionsToDatabase() { mutations.add(insertRecordMutation(1)); mutations.add(insertRecordMutation(2)); com.google.cloud.Timestamp t1 = databaseClient.write(mutations); - LOG.debug("The first transaction committed with timestamp: " + t1.toString()); + LOG.info("The first transaction committed with timestamp: " + t1.toString()); mutations.clear(); // 2. Commmit a transaction to insert Singer 3 and remove Singer 1 from the table. mutations.add(insertRecordMutation(3)); mutations.add(deleteRecordMutation(1)); com.google.cloud.Timestamp t2 = databaseClient.write(mutations); - LOG.debug("The second transaction committed with timestamp: " + t2.toString()); + LOG.info("The second transaction committed with timestamp: " + t2.toString()); mutations.clear(); // 3. Commit a transaction to delete Singer 2 and Singer 3 from the table. mutations.add(deleteRecordMutation(2)); mutations.add(deleteRecordMutation(3)); com.google.cloud.Timestamp t3 = databaseClient.write(mutations); - LOG.debug("The third transaction committed with timestamp: " + t3.toString()); + LOG.info("The third transaction committed with timestamp: " + t3.toString()); mutations.clear(); // 4. Commit a transaction to delete Singer 0. mutations.add(deleteRecordMutation(0)); com.google.cloud.Timestamp t4 = databaseClient.write(mutations); - LOG.debug("The fourth transaction committed with timestamp: " + t4.toString()); + LOG.info("The fourth transaction committed with timestamp: " + t4.toString()); return t4; } From bd1cb0776692ce7c406c22b6bddca1d51b520d4f Mon Sep 17 00:00:00 2001 From: Nancy Date: Wed, 9 Mar 2022 16:59:43 -0800 Subject: [PATCH 64/68] More logging --- .../it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java index 1c83a85c3116..bab819b3785f 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java @@ -332,7 +332,10 @@ public void onExpiry( buffer, @TimerId("timer") Timer timer, @StateId("seenKey") ValueState seenKey) { + String keyForTimer = seenKey.read(); Instant timerContextTimestamp = context.timestamp(); + LOG.info("Timer reached expiration time for key {} and for timestamp {}", keyForTimer, + timerContextTimestamp)'' if (!buffer.isEmpty().read()) { final List> records = @@ -391,9 +394,8 @@ public void onExpiry( Instant nextTimer = timerContextTimestamp.plus(Duration.standardSeconds(incrementIntervalInSeconds)); - String keyString = seenKey.read(); if (buffer.isEmpty() != null && !buffer.isEmpty().read()) { - LOG.info("Setting next timer to {} for key ", nextTimer.toString(), keyString); + LOG.info("Setting next timer to {} for key ", nextTimer.toString(), keyForTimer); timer.set(nextTimer); } else { LOG.info( From e702eaac738fd95ebd56c6d40edad28254c67345 Mon Sep 17 00:00:00 2001 From: Nancy Xu Date: Thu, 17 Mar 2022 16:23:24 +0000 Subject: [PATCH 65/68] Made pipelines streaming --- ...annerChangeStreamOrderedByTimestampAndTransactionIdIT.java | 4 ++-- .../it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java | 3 ++- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.java index 01ae1f055add..cafe9afc6a78 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.java @@ -193,8 +193,8 @@ public void testTransactionBoundaries() { // Delete Singers 2 and 3. + "{\"SingerId\":\"2\"}{\"SingerId\":\"3\"},DELETE\n"); - final PipelineResult pipelineResult = pipeline.run(); - pipelineResult.waitUntilFinish(); + pipeline.runWithAdditionalOptionArgs(Collections.singletonList("--streaming")) + .waitUntiFinish(); } // KeyByTransactionIdFn takes in a DataChangeRecord and outputs a key-value pair of diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java index bab819b3785f..cff69546ae69 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java @@ -193,7 +193,8 @@ public void testOrderedWithinKey() { + "{\"FirstName\":\"Inserting mutation 3\",\"LastName\":null,\"SingerInfo\":null};" + "Deleted record;"); - pipeline.run().waitUntilFinish(); + pipeline.runWithAdditionalOptionArgs(Collections.singletonList("--streaming")) + .waitUntiFinish(); } // Data change records may contain multiple mods if there are multiple primary keys. From 4a18c08f141c56ec229209c8afb0a8ef3302535f Mon Sep 17 00:00:00 2001 From: Nancy Xu Date: Thu, 17 Mar 2022 16:26:53 +0000 Subject: [PATCH 66/68] Made small fixes --- ...pannerChangeStreamOrderedByTimestampAndTransactionIdIT.java | 3 ++- .../it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.java index cafe9afc6a78..1aa1b06cc706 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.java @@ -24,6 +24,7 @@ import com.google.cloud.spanner.Mutation; import java.io.Serializable; import java.util.ArrayList; +import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Objects; @@ -194,7 +195,7 @@ public void testTransactionBoundaries() { + "{\"SingerId\":\"2\"}{\"SingerId\":\"3\"},DELETE\n"); pipeline.runWithAdditionalOptionArgs(Collections.singletonList("--streaming")) - .waitUntiFinish(); + .waitUntilFinish(); } // KeyByTransactionIdFn takes in a DataChangeRecord and outputs a key-value pair of diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java index cff69546ae69..78078e0034dc 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java @@ -194,7 +194,7 @@ public void testOrderedWithinKey() { + "Deleted record;"); pipeline.runWithAdditionalOptionArgs(Collections.singletonList("--streaming")) - .waitUntiFinish(); + .waitUntilFinish(); } // Data change records may contain multiple mods if there are multiple primary keys. From 7bcd6b52df81b3066101f3a5eccb5098338ce75f Mon Sep 17 00:00:00 2001 From: Nancy Xu Date: Thu, 17 Mar 2022 16:51:24 +0000 Subject: [PATCH 67/68] Small fixes --- .../it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java index 78078e0034dc..4e0125264be6 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java @@ -336,7 +336,7 @@ public void onExpiry( String keyForTimer = seenKey.read(); Instant timerContextTimestamp = context.timestamp(); LOG.info("Timer reached expiration time for key {} and for timestamp {}", keyForTimer, - timerContextTimestamp)'' + timerContextTimestamp); if (!buffer.isEmpty().read()) { final List> records = @@ -396,11 +396,11 @@ public void onExpiry( Instant nextTimer = timerContextTimestamp.plus(Duration.standardSeconds(incrementIntervalInSeconds)); if (buffer.isEmpty() != null && !buffer.isEmpty().read()) { - LOG.info("Setting next timer to {} for key ", nextTimer.toString(), keyForTimer); + LOG.info("Setting next timer to {} for key {}", nextTimer.toString(), keyForTimer); timer.set(nextTimer); } else { LOG.info( - "Timer not being set since the buffer is empty for key {} ", keyString); + "Timer not being set since the buffer is empty for key {} ", keyForTimer); seenKey.clear(); } } From f47357ae8ce0cc5ee6ba217b3e1f4a4cb514c9cb Mon Sep 17 00:00:00 2001 From: Nancy Xu Date: Thu, 17 Mar 2022 16:55:36 +0000 Subject: [PATCH 68/68] Ran spotless Apply --- ...StreamOrderedByTimestampAndTransactionIdIT.java | 4 ++-- ...nnerChangeStreamOrderedWithinKeyGloballyIT.java | 14 +++++++------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.java index 1aa1b06cc706..0d70a063b750 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedByTimestampAndTransactionIdIT.java @@ -33,7 +33,6 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; import javax.annotation.Nullable; -import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.coders.BooleanCoder; import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; @@ -194,7 +193,8 @@ public void testTransactionBoundaries() { // Delete Singers 2 and 3. + "{\"SingerId\":\"2\"}{\"SingerId\":\"3\"},DELETE\n"); - pipeline.runWithAdditionalOptionArgs(Collections.singletonList("--streaming")) + pipeline + .runWithAdditionalOptionArgs(Collections.singletonList("--streaming")) .waitUntilFinish(); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java index 4e0125264be6..78f742f3f380 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamOrderedWithinKeyGloballyIT.java @@ -31,7 +31,6 @@ import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; import java.util.stream.StreamSupport; -import javax.annotation.Nullable; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; @@ -145,8 +144,7 @@ public void testOrderedWithinKey() { .apply(ParDo.of(new BreakRecordByModFn())) .apply(ParDo.of(new KeyByIdFn())) .apply(ParDo.of(new KeyValueByCommitTimestampAndTransactionIdFn<>())) - .apply( - ParDo.of(new BufferKeyUntilOutputTimestamp(timeIncrementInSeconds))) + .apply(ParDo.of(new BufferKeyUntilOutputTimestamp(timeIncrementInSeconds))) .apply(ParDo.of(new ToStringFn())); // Assert that the returned PCollection contains one entry per key for the committed @@ -193,7 +191,8 @@ public void testOrderedWithinKey() { + "{\"FirstName\":\"Inserting mutation 3\",\"LastName\":null,\"SingerInfo\":null};" + "Deleted record;"); - pipeline.runWithAdditionalOptionArgs(Collections.singletonList("--streaming")) + pipeline + .runWithAdditionalOptionArgs(Collections.singletonList("--streaming")) .waitUntilFinish(); } @@ -335,7 +334,9 @@ public void onExpiry( @StateId("seenKey") ValueState seenKey) { String keyForTimer = seenKey.read(); Instant timerContextTimestamp = context.timestamp(); - LOG.info("Timer reached expiration time for key {} and for timestamp {}", keyForTimer, + LOG.info( + "Timer reached expiration time for key {} and for timestamp {}", + keyForTimer, timerContextTimestamp); if (!buffer.isEmpty().read()) { final List> @@ -399,8 +400,7 @@ public void onExpiry( LOG.info("Setting next timer to {} for key {}", nextTimer.toString(), keyForTimer); timer.set(nextTimer); } else { - LOG.info( - "Timer not being set since the buffer is empty for key {} ", keyForTimer); + LOG.info("Timer not being set since the buffer is empty for key {} ", keyForTimer); seenKey.clear(); } }