ihji commented on a change in pull request #14501:
URL: https://github.com/apache/beam/pull/14501#discussion_r625333795
##########
File path:
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
##########
@@ -446,8 +457,9 @@ public Job getJob(JobReference jobRef, Sleeper sleeper,
BackOff backoff)
@VisibleForTesting
static class DatasetServiceImpl implements DatasetService {
+ // Backoff: 200ms * 1.5 ^ n, n=[1,10]
private static final FluentBackoff INSERT_BACKOFF_FACTORY =
-
FluentBackoff.DEFAULT.withInitialBackoff(Duration.millis(200)).withMaxRetries(5);
+
FluentBackoff.DEFAULT.withInitialBackoff(Duration.millis(200)).withMaxRetries(10);
Review comment:
Any reason to increase max retries?
##########
File path:
runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/ResourceIdentifiers.java
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.runners.core.metrics;
+
+/**
+ * Helper functions to generate resource labels strings for GCP entitites
These can be used on
Review comment:
Is this GCP specific resource identifier helper? Would it be better to
put GCP in the class name? (something like `GCPResourceIdentifier`)
##########
File path:
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
##########
@@ -880,52 +889,83 @@ public void deleteDataset(String projectId, String
datasetId)
}
if (dataSize >= maxRowBatchSize
- || rows.size() >= maxRowsPerBatch
+ || rowsToInsertForRequest.size() >= maxRowsPerBatch
|| i == rowsToPublish.size() - 1) {
TableDataInsertAllRequest content = new
TableDataInsertAllRequest();
- content.setRows(rows);
+ content.setRows(rowsToInsertForRequest);
content.setSkipInvalidRows(skipInvalidRows);
content.setIgnoreUnknownValues(ignoreUnkownValues);
+ HashMap<String, String> baseLabels = new HashMap<String, String>();
+ // TODO(ajamato): Add Ptransform label. Populate it as empty for
now to prevent the
+ // SpecMonitoringInfoValidator from dropping the MonitoringInfo.
+ baseLabels.put(MonitoringInfoConstants.Labels.PTRANSFORM, "");
+ baseLabels.put(MonitoringInfoConstants.Labels.SERVICE, "BigQuery");
+ baseLabels.put(MonitoringInfoConstants.Labels.METHOD,
"BigQueryBatchWrite");
+ baseLabels.put(
+ MonitoringInfoConstants.Labels.RESOURCE,
+ ResourceIdentifiers.bigQueryTable(
+ ref.getProjectId(), ref.getDatasetId(), ref.getTableId()));
+ baseLabels.put(MonitoringInfoConstants.Labels.BIGQUERY_PROJECT_ID,
ref.getProjectId());
+ baseLabels.put(MonitoringInfoConstants.Labels.BIGQUERY_DATASET,
ref.getDatasetId());
+ baseLabels.put(MonitoringInfoConstants.Labels.BIGQUERY_TABLE,
ref.getTableId());
+
+ ServiceCallMetric serviceCallMetric =
+ new
ServiceCallMetric(MonitoringInfoConstants.Urns.API_REQUEST_COUNT, baseLabels);
Review comment:
Could we create `ServiceCallMetric` instance outside the loop? The
object configuration doesn't seem to change in each iteration.
##########
File path:
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
##########
@@ -842,21 +844,25 @@ public void deleteDataset(String projectId, String
datasetId)
}
long retTotalDataSize = 0;
- List<TableDataInsertAllResponse.InsertErrors> allErrors = new
ArrayList<>();
+ List<TableDataInsertAllResponse.InsertErrors> rowsFailedToInsertErrors =
new ArrayList<>();
// These lists contain the rows to publish. Initially the contain the
entire list.
// If there are failures, they will contain only the failed rows to be
retried.
List<FailsafeValueInSingleWindow<TableRow, TableRow>> rowsToPublish =
rowList;
List<String> idsToPublish = null;
+
if (!ignoreInsertIds) {
idsToPublish = insertIdList;
}
+
+ List<InsertAttemptParams> insertAttemptParamsForEachRequest = new
ArrayList<>();
Review comment:
This change looks irrelevant to the PR description. Would it be better
to split into a separate PR?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]