diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java index f4303886c7ab..bcd3db4730c7 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java @@ -17,8 +17,6 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; - import com.google.api.client.googleapis.json.GoogleJsonError; import com.google.api.client.googleapis.json.GoogleJsonResponseException; import com.google.api.client.googleapis.services.AbstractGoogleClientRequest; @@ -56,7 +54,6 @@ import com.google.api.services.bigquery.model.Table; import com.google.api.services.bigquery.model.TableDataInsertAllRequest; import com.google.api.services.bigquery.model.TableDataInsertAllResponse; -import com.google.api.services.bigquery.model.TableDataInsertAllResponse.InsertErrors; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; import com.google.auth.Credentials; @@ -966,14 +963,22 @@ public void deleteDataset(String projectId, String datasetId) ALWAYS_RETRY); } - static class InsertBatchofRowsCallable implements Callable> { + static class InsertBatchofRowsCallable + implements Callable> { + private static final Logger LOG = LoggerFactory.getLogger(InsertBatchofRowsCallable.class); private final TableReference ref; private final Boolean skipInvalidRows; - private final Boolean ignoreUnkownValues; + private final Boolean ignoreUnknownValues; private final Bigquery client; private final FluentBackoff rateLimitBackoffFactory; private final List rows; + private final List> rowsToPublish; + private final List idsToPublish; + private final InsertRetryPolicy retryPolicy; + private final ErrorContainer errorContainer; + private final List> failedInserts; + private final boolean ignoreInsertIds; private final AtomicLong maxThrottlingMsec; private final Sleeper sleeper; private final StreamingInsertsMetrics result; @@ -985,15 +990,27 @@ static class InsertBatchofRowsCallable implements Callable> { Bigquery client, FluentBackoff rateLimitBackoffFactory, List rows, + List> rowsToPublish, + List idsToPublish, + InsertRetryPolicy retryPolicy, + ErrorContainer errorContainer, + List> failedInserts, + boolean ignoreInsertIds, AtomicLong maxThrottlingMsec, Sleeper sleeper, StreamingInsertsMetrics result) { this.ref = ref; this.skipInvalidRows = skipInvalidRows; - this.ignoreUnkownValues = ignoreUnknownValues; + this.ignoreUnknownValues = ignoreUnknownValues; this.client = client; this.rateLimitBackoffFactory = rateLimitBackoffFactory; this.rows = rows; + this.rowsToPublish = rowsToPublish; + this.idsToPublish = idsToPublish; + this.retryPolicy = retryPolicy; + this.errorContainer = errorContainer; + this.failedInserts = failedInserts; + this.ignoreInsertIds = ignoreInsertIds; this.maxThrottlingMsec = maxThrottlingMsec; this.sleeper = sleeper; this.result = result; @@ -1004,7 +1021,7 @@ public List call() throws Exception { TableDataInsertAllRequest content = new TableDataInsertAllRequest(); content.setRows(rows); content.setSkipInvalidRows(skipInvalidRows); - content.setIgnoreUnknownValues(ignoreUnkownValues); + content.setIgnoreUnknownValues(ignoreUnknownValues); final Bigquery.Tabledata.InsertAll insert = client @@ -1012,8 +1029,7 @@ public List call() throws Exception { .insertAll(ref.getProjectId(), ref.getDatasetId(), ref.getTableId(), content) .setPrettyPrint(false); - // A backoff for rate limit exceeded errors. - BackOff backoff1 = BackOffAdapter.toGcpBackOff(rateLimitBackoffFactory.backoff()); + BackOff backoff = BackOffAdapter.toGcpBackOff(rateLimitBackoffFactory.backoff()); long totalBackoffMillis = 0L; while (true) { ServiceCallMetric serviceCallMetric = BigQueryUtils.writeCallMetric(ref); @@ -1021,6 +1037,18 @@ public List call() throws Exception { try { List response = insert.execute().getInsertErrors(); + List retryErrors = new ArrayList<>(); + if (response != null && !response.isEmpty()) { + for (TableDataInsertAllResponse.InsertErrors error : response) { + if (error.getIndex() != null + && retryPolicy.shouldRetry(new InsertRetryPolicy.Context(error))) { + retryErrors.add(error); + } else if (error.getIndex() != null) { + errorContainer.add( + failedInserts, error, ref, rowsToPublish.get(error.getIndex().intValue())); + } + } + } if (response == null || response.isEmpty()) { serviceCallMetric.call("ok"); } else { @@ -1031,57 +1059,70 @@ public List call() throws Exception { } } result.updateSuccessfulRpcMetrics(start, Instant.now()); - return response; - } catch (IOException e) { - GoogleJsonError.ErrorInfo errorInfo = getErrorInfo(e); - if (errorInfo == null) { - serviceCallMetric.call(ServiceCallMetric.CANONICAL_STATUS_UNKNOWN); - result.updateFailedRpcMetrics(start, start, BigQuerySinkMetrics.UNKNOWN); - throw e; - } - String errorReason = errorInfo.getReason(); + return retryErrors; + } catch (GoogleJsonResponseException e) { + GoogleJsonError.ErrorInfo errorInfo = DatasetServiceImpl.getErrorInfo(e); + String errorReason = + errorInfo != null ? errorInfo.getReason() : "http_error_" + e.getStatusCode(); serviceCallMetric.call(errorReason); result.updateFailedRpcMetrics(start, Instant.now(), errorReason); - /** - * TODO(BEAM-10584): Check for QUOTA_EXCEEDED error will be replaced by - * ApiErrorExtractor.INSTANCE.quotaExceeded(e) after the next release of - * GoogleCloudDataproc/hadoop-connectors - */ - if (!ApiErrorExtractor.INSTANCE.rateLimited(e) - && !errorInfo.getReason().equals(QUOTA_EXCEEDED)) { - if (ApiErrorExtractor.INSTANCE.badRequest(e) - && e.getMessage().contains(NO_ROWS_PRESENT)) { - LOG.error( - "No rows present in the request error likely caused by BigQuery Insert" - + " timing out. Update BigQueryOptions.setHTTPWriteTimeout to be longer," - + " or 0 to disable timeouts", - e.getCause()); - } - throw e; + + List batchErrors = new ArrayList<>(); + for (int i = 0; i < rows.size(); i++) { + batchErrors.add( + new TableDataInsertAllResponse.InsertErrors() + .setIndex((long) i) + .setErrors(ImmutableList.of(new ErrorProto().setReason(errorReason)))); } - try (QuotaEventCloseable qec = - new QuotaEvent.Builder() - .withOperation("insert_all") - .withFullResourceName(BigQueryHelpers.toTableFullResourceName(ref)) - .create()) { - LOG.info( - String.format( - "BigQuery insertAll error, retrying: %s", - ApiErrorExtractor.INSTANCE.getErrorMessage(e))); + List retryErrors = new ArrayList<>(); + for (TableDataInsertAllResponse.InsertErrors error : batchErrors) { + if (retryPolicy.shouldRetry(new InsertRetryPolicy.Context(error, e))) { + retryErrors.add(error); + } else { + errorContainer.add( + failedInserts, error, ref, rowsToPublish.get(error.getIndex().intValue())); + } } - try { - long nextBackOffMillis = backoff1.nextBackOffMillis(); + if (!retryErrors.isEmpty()) { + try (QuotaEventCloseable qec = + new QuotaEvent.Builder() + .withOperation("insert_all") + .withFullResourceName(BigQueryHelpers.toTableFullResourceName(ref)) + .create()) { + LOG.info( + String.format( + "BigQuery insertAll error, retrying: %s", + ApiErrorExtractor.INSTANCE.getErrorMessage(e))); + } + long nextBackOffMillis = backoff.nextBackOffMillis(); if (nextBackOffMillis == BackOff.STOP) { - throw e; + return retryErrors; } sleeper.sleep(nextBackOffMillis); totalBackoffMillis += nextBackOffMillis; - final long totalBackoffMillisSoFar = totalBackoffMillis; - maxThrottlingMsec.getAndUpdate(current -> Math.max(current, totalBackoffMillisSoFar)); + long currentBackoffMillis = totalBackoffMillis; + maxThrottlingMsec.getAndUpdate(current -> Math.max(current, currentBackoffMillis)); result.updateRetriedRowsWithStatus(errorReason, rows.size()); - } catch (InterruptedException interrupted) { - throw new IOException("Interrupted while waiting before retrying insertAll"); + continue; + } + if (ApiErrorExtractor.INSTANCE.badRequest(e) + && e.getMessage().contains("No rows present in the request.")) { + LOG.error( + "No rows present in the request error likely caused by BigQuery Insert" + + " timing out. Update BigQueryOptions.setHTTPWriteTimeout to be longer," + + " or 0 to disable timeouts", + e.getCause()); } + return retryErrors; + } catch (IOException e) { + GoogleJsonError.ErrorInfo errorInfo = DatasetServiceImpl.getErrorInfo(e); + String errorReason = + errorInfo != null + ? errorInfo.getReason() + : ServiceCallMetric.CANONICAL_STATUS_UNKNOWN; + serviceCallMetric.call(errorReason); + result.updateFailedRpcMetrics(start, Instant.now(), errorReason); + throw e; } } } @@ -1103,7 +1144,7 @@ long insertAll( boolean ignoreInsertIds, List> successfulRows) throws IOException, InterruptedException { - checkNotNull(ref, "ref"); + Preconditions.checkNotNull(ref, "ref"); if (executor == null) { this.executor = new BoundedExecutorService( @@ -1121,8 +1162,6 @@ long insertAll( final Set failedIndices = new HashSet<>(); long retTotalDataSize = 0; List allErrors = 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> rowsToPublish = rowList; List idsToPublish = null; if (!ignoreInsertIds) { @@ -1130,17 +1169,11 @@ long insertAll( } while (true) { - List> retryRows = new ArrayList<>(); - List retryIds = (idsToPublish != null) ? new ArrayList<>() : null; - int strideIndex = 0; - // Upload in batches. List rows = new ArrayList<>(); long dataSize = 0L; - List>> futures = new ArrayList<>(); List strideIndices = new ArrayList<>(); - // Store the longest throttled time across all parallel threads final AtomicLong maxThrottlingMsec = new AtomicLong(); int rowIndex = 0; @@ -1153,18 +1186,11 @@ long insertAll( throw new RuntimeException("Failed to convert the row to JSON", ex); } - // The following scenario must be *extremely* rare. - // If this row's encoding by itself is larger than the maximum row payload, then it's - // impossible to insert into BigQuery, and so we send it out through the dead-letter - // queue. if (nextRowSize >= MAX_BQ_ROW_PAYLOAD_BYTES) { - InsertErrors error = - new InsertErrors() + TableDataInsertAllResponse.InsertErrors error = + new TableDataInsertAllResponse.InsertErrors() .setErrors(ImmutableList.of(new ErrorProto().setReason("row-too-large"))); - // We verify whether the retryPolicy parameter expects us to retry. If it does, then - // it will return true. Otherwise it will return false. if (retryPolicy.shouldRetry(new InsertRetryPolicy.Context(error))) { - // Obtain table schema TableSchema tableSchema = null; try { String tableSpec = BigQueryHelpers.toTableSpec(ref); @@ -1182,20 +1208,17 @@ long insertAll( LOG.warn("Failed to get table schema", e); } - // Validate row schema String rowDetails = ""; if (tableSchema != null) { rowDetails = validateRowSchema(row, tableSchema); } - // Basic log to return String bqLimitLog = String.format( "We have observed a row of size %s bytes exceeding the " + "BigQueryIO limit of %s.", nextRowSize, MAX_BQ_ROW_PAYLOAD_DESC); - // Add on row schema diff details if present if (!rowDetails.isEmpty()) { bqLimitLog += String.format( @@ -1213,24 +1236,24 @@ long insertAll( } } - // If adding the next row will push the request above BQ row limits, or - // if the current batch of elements is larger than the targeted request size, - // we immediately go and issue the data insertion. if (dataSize + nextRowSize >= MAX_BQ_ROW_PAYLOAD_BYTES || dataSize >= maxRowBatchSize || rows.size() + 1 > maxRowsPerBatch) { - // If the row does not fit into the insert buffer, then we take the current buffer, - // issue the insert call, and we retry adding the same row to the troublesome buffer. - // Add a future to insert the current batch into BQ. futures.add( executor.submit( - new InsertBatchofRowsCallable( + new InsertBatchofRowsCallable<>( ref, skipInvalidRows, ignoreUnknownValues, client, rateLimitBackoffFactory, rows, + rowsToPublish.subList(strideIndex, rowIndex), + idsToPublish == null ? null : idsToPublish.subList(strideIndex, rowIndex), + retryPolicy, + errorContainer, + failedInserts, + ignoreInsertIds, maxThrottlingMsec, sleeper, streamingInsertsResults))); @@ -1240,10 +1263,8 @@ long insertAll( rows = new ArrayList<>(); dataSize = 0L; } - // If the row fits into the insert buffer, then we add it to the buffer to be inserted - // later, and we move onto the next row. TableDataInsertAllRequest.Rows out = new TableDataInsertAllRequest.Rows(); - if (idsToPublish != null) { + if (idsToPublish != null && !ignoreInsertIds) { out.setInsertId(idsToPublish.get(rowIndex)); } out.setJson(row.getUnknownKeys()); @@ -1255,95 +1276,97 @@ long insertAll( if (rows.size() > 0) { futures.add( executor.submit( - new InsertBatchofRowsCallable( + new InsertBatchofRowsCallable<>( ref, skipInvalidRows, ignoreUnknownValues, client, rateLimitBackoffFactory, rows, + rowsToPublish.subList(strideIndex, rowIndex), + idsToPublish == null ? null : idsToPublish.subList(strideIndex, rowIndex), + retryPolicy, + errorContainer, + failedInserts, + ignoreInsertIds, maxThrottlingMsec, sleeper, streamingInsertsResults))); strideIndices.add(strideIndex); retTotalDataSize += dataSize; - rows = new ArrayList<>(); } try { + List> retryRows = new ArrayList<>(); + List retryIds = + (idsToPublish != null && !ignoreInsertIds) ? new ArrayList<>() : null; for (int i = 0; i < futures.size(); i++) { - List errors = futures.get(i).get(); - if (errors == null) { - continue; + List errors; + try { + errors = futures.get(i).get(); + } catch (ExecutionException e) { + throw new IOException("Failed to execute insert batch", e); } - - for (TableDataInsertAllResponse.InsertErrors error : errors) { - if (error.getIndex() == null) { - throw new IOException("Insert failed: " + error + ", other errors: " + allErrors); - } - int errorIndex = error.getIndex().intValue() + strideIndices.get(i); - failedIndices.add(errorIndex); - if (retryPolicy.shouldRetry(new InsertRetryPolicy.Context(error))) { + if (errors != null && !errors.isEmpty()) { + for (TableDataInsertAllResponse.InsertErrors error : errors) { + if (error.getIndex() == null) { + throw new IOException("Insert failed: " + error + ", other errors: " + allErrors); + } + int globalIndex = error.getIndex().intValue() + strideIndices.get(i); + failedIndices.add(globalIndex); allErrors.add(error); - retryRows.add(rowsToPublish.get(errorIndex)); - // TODO (https://github.com/apache/beam/issues/20891): Select the retry rows(using - // errorIndex) from the batch of rows which attempted insertion in this call. - // Not the entire set of rows in rowsToPublish. + retryRows.add(rowsToPublish.get(globalIndex)); if (retryIds != null) { - retryIds.add(idsToPublish.get(errorIndex)); + retryIds.add(idsToPublish.get(globalIndex)); } - } else { - numFailedRows += 1; - errorContainer.add(failedInserts, error, ref, rowsToPublish.get(errorIndex)); } } } - // Accumulate the longest throttled time across all parallel threads throttlingMsecs.inc(maxThrottlingMsec.get()); + + if (allErrors.isEmpty()) { + break; + } + + rowsToPublish = retryRows; + idsToPublish = retryIds; + streamingInsertsResults.updateRetriedRowsWithStatus( + BigQuerySinkMetrics.INTERNAL, retryRows.size()); + int numErrorToLog = Math.min(allErrors.size(), 5); + LOG.info( + "Retrying {} failed inserts to BigQuery. First {} fails: {}", + rowsToPublish.size(), + numErrorToLog, + allErrors.subList(0, numErrorToLog)); + allErrors.clear(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - throw new IOException("Interrupted while inserting " + rowsToPublish); - } catch (ExecutionException e) { - streamingInsertsResults.updateStreamingInsertsMetrics( - ref, rowList.size(), rowList.size()); - throw new RuntimeException(e.getCause()); + throw new IOException( + "Interrupted while waiting before retrying insert of " + rowsToPublish); } - if (allErrors.isEmpty()) { - break; - } - long nextBackoffMillis = backoff.nextBackOffMillis(); - if (nextBackoffMillis == BackOff.STOP) { + long nextBackOffMillis = backoff.nextBackOffMillis(); + if (nextBackOffMillis == BackOff.STOP) { break; } try { - sleeper.sleep(nextBackoffMillis); + sleeper.sleep(nextBackOffMillis); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - throw new IOException("Interrupted while waiting before retrying insert of " + retryRows); + throw new IOException( + "Interrupted while waiting before retrying insert of " + rowsToPublish); } - rowsToPublish = retryRows; - idsToPublish = retryIds; - streamingInsertsResults.updateRetriedRowsWithStatus( - BigQuerySinkMetrics.INTERNAL, retryRows.size()); - // print first 5 failures - int numErrorToLog = Math.min(allErrors.size(), 5); - LOG.info( - "Retrying {} failed inserts to BigQuery. First {} fails: {}", - rowsToPublish.size(), - numErrorToLog, - allErrors.subList(0, numErrorToLog)); - allErrors.clear(); } + if (successfulRows != null) { - for (int i = 0; i < rowsToPublish.size(); i++) { + for (int i = 0; i < rowList.size(); i++) { if (!failedIndices.contains(i)) { successfulRows.add( ValueInSingleWindow.of( - rowsToPublish.get(i).getValue(), - rowsToPublish.get(i).getTimestamp(), - rowsToPublish.get(i).getWindow(), - rowsToPublish.get(i).getPaneInfo())); + rowList.get(i).getValue(), + rowList.get(i).getTimestamp(), + rowList.get(i).getWindow(), + rowList.get(i).getPaneInfo())); } } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/InsertRetryPolicy.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/InsertRetryPolicy.java index ff21d5c0e546..bfc943ccde89 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/InsertRetryPolicy.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/InsertRetryPolicy.java @@ -17,45 +17,54 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; +import com.google.api.client.googleapis.json.GoogleJsonResponseException; import com.google.api.services.bigquery.model.ErrorProto; import com.google.api.services.bigquery.model.TableDataInsertAllResponse; import java.io.Serializable; import java.util.Set; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; +import org.checkerframework.checker.nullness.qual.Nullable; /** * A retry policy for streaming BigQuery inserts. * - *

This retry policy currently applies only to per-element errors within successful (200 OK) - * BigQuery responses. Non-200 responses (e.g., 400 Bad Request, 500 Internal Server Error) will - * result in a {@code RuntimeException} and bundle failure. The subsequent handling of the failed - * bundle (e.g., retry or final failure) is determined by the specific Runner's fault tolerance - * mechanisms. + *

This retry policy applies to both per-element errors within successful (200 OK) BigQuery + * responses and non-200 HTTP responses (e.g., 400 Bad Request, 429 Too Many Requests, 500 Internal + * Server Error). Non-200 responses are handled by passing a {@code GoogleJsonResponseException} to + * the retry policy, allowing per-row retry decisions based on HTTP status codes. Rows that should + * not be retried are sent to the Dead Letter Queue (DLQ). * - * @see org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.DatasetServiceImpl#insertAll( - * TableReference, java.util.List, java.util.List, BackOff, - * org.apache.beam.sdk.util.FluentBackoff, Sleeper, - * org.apache.beam.sdk.io.gcp.bigquery.InsertRetryPolicy, java.util.List, - * org.apache.beam.sdk.io.gcp.bigquery.ErrorContainer, boolean, boolean, boolean, - * java.util.List) + * @see org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.DatasetServiceImpl#insertAll */ public abstract class InsertRetryPolicy implements Serializable { /** * Contains information about a failed insert. * - *

Currently only the list of errors returned from BigQuery. In the future this may contain - * more information - e.g. how many times this insert has been retried, and for how long. + *

Includes either per-row errors from a 200 OK response or a {@code + * GoogleJsonResponseException} for non-200 HTTP responses. Future extensions may include retry + * attempt counts or durations. */ public static class Context { - // A list of all errors corresponding to an attempted insert of a single record. - final TableDataInsertAllResponse.InsertErrors errors; + private final TableDataInsertAllResponse.@Nullable InsertErrors errors; + private final @Nullable GoogleJsonResponseException exception; - public TableDataInsertAllResponse.InsertErrors getInsertErrors() { - return errors; + public Context(TableDataInsertAllResponse.@Nullable InsertErrors errors) { + this(errors, null); } - public Context(TableDataInsertAllResponse.InsertErrors errors) { + public Context( + TableDataInsertAllResponse.@Nullable InsertErrors errors, + @Nullable GoogleJsonResponseException exception) { this.errors = errors; + this.exception = exception; + } + + public TableDataInsertAllResponse.@Nullable InsertErrors getInsertErrors() { + return errors; + } + + public @Nullable GoogleJsonResponseException getHttpException() { + return exception; } } @@ -76,29 +85,43 @@ public boolean shouldRetry(Context context) { }; } - /** Always retry all failures. */ + /** Always retry all failures, including transient HTTP errors (429, 503). */ public static InsertRetryPolicy alwaysRetry() { return new InsertRetryPolicy() { @Override public boolean shouldRetry(Context context) { - return true; + GoogleJsonResponseException httpException = context.getHttpException(); + if (httpException != null) { + int statusCode = httpException.getStatusCode(); + return statusCode == 429 || statusCode == 503; + } + TableDataInsertAllResponse.InsertErrors insertErrors = context.getInsertErrors(); + return insertErrors != null; } }; } - /** Retry all failures except for known persistent errors. */ + /** Retry all failures except for known persistent errors and non-retryable HTTP errors. */ public static InsertRetryPolicy retryTransientErrors() { return new InsertRetryPolicy() { @Override public boolean shouldRetry(Context context) { - if (context.getInsertErrors().getErrors() != null) { - for (ErrorProto error : context.getInsertErrors().getErrors()) { - if (error.getReason() != null && PERSISTENT_ERRORS.contains(error.getReason())) { + GoogleJsonResponseException httpException = context.getHttpException(); + if (httpException != null) { + int statusCode = httpException.getStatusCode(); + return statusCode == 429 || statusCode == 503; + } + TableDataInsertAllResponse.InsertErrors insertErrors = context.getInsertErrors(); + if (insertErrors != null && insertErrors.getErrors() != null) { + for (ErrorProto error : insertErrors.getErrors()) { + String reason = error.getReason(); + if (reason != null && PERSISTENT_ERRORS.contains(reason)) { return false; } } + return true; } - return true; + return false; } }; }