From 1151f9b70d3e06a28a846ecbc49ce351e6ddca88 Mon Sep 17 00:00:00 2001 From: Radek Stankiewicz Date: Wed, 4 Mar 2026 19:50:44 +0100 Subject: [PATCH 1/4] Fix Slf4j formatting errors in sdks/java/io/google-cloud-platform --- .../sdk/io/gcp/bigquery/BigQueryHelpers.java | 2 +- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 3 +- .../io/gcp/bigquery/BigQueryServicesImpl.java | 14 +++---- .../bigquery/DynamicDestinationsHelpers.java | 2 +- .../StorageApiFinalizeWritesDoFn.java | 16 ++++---- .../StorageApiFlushAndFinalizeDoFn.java | 11 ++---- .../StorageApiWriteUnshardedRecords.java | 2 +- .../StorageApiWritesShardedRecords.java | 15 +++----- .../sdk/io/gcp/bigquery/TableSchemaCache.java | 4 +- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 6 +-- .../gcp/bigtable/BigtableServiceFactory.java | 14 +++---- .../action/DetectNewPartitionsAction.java | 8 ++-- .../changestreams/dao/MetadataTableDao.java | 4 +- .../changestreams/dofn/InitializeDoFn.java | 6 +-- .../sdk/io/gcp/datastore/EntityToRow.java | 2 +- .../sdk/io/gcp/datastore/RowToEntity.java | 2 +- .../io/gcp/firestore/FirestoreV1WriteFn.java | 2 +- .../beam/sdk/io/gcp/healthcare/FhirIO.java | 38 +++++++++---------- .../healthcare/FhirIOPatientEverything.java | 7 +--- .../beam/sdk/io/gcp/healthcare/HL7v2IO.java | 34 ++++++----------- .../healthcare/HttpHealthcareApiClient.java | 11 ++---- .../io/gcp/pubsub/AddTimestampAttribute.java | 7 ++-- .../sdk/io/gcp/spanner/BatchSpannerRead.java | 2 +- .../sdk/io/gcp/spanner/NaiveSpannerRead.java | 2 +- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 15 ++++---- .../sdk/io/gcp/testing/BigqueryClient.java | 16 ++++---- .../dao/MetadataTableDaoTest.java | 2 +- .../it/BigtableChangeStreamIT.java | 2 +- .../changestreams/it/IntegrationTestEnv.java | 14 +++---- ...mOrderedByTimestampAndTransactionIdIT.java | 15 ++++---- ...hangeStreamOrderedWithinKeyGloballyIT.java | 12 +++--- ...SpannerChangeStreamOrderedWithinKeyIT.java | 16 ++++---- ...erChangeStreamTransactionBoundariesIT.java | 16 ++++---- .../changestreams/util/TestJsonMapper.java | 2 +- 34 files changed, 144 insertions(+), 180 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java index d468ffbea43c..55c703438f02 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java @@ -296,7 +296,7 @@ void logBigQueryError(@Nullable Job job) { ? String.format(" due to: %s", jobErrors.get(jobErrors.size() - 1).getMessage()) : ""; - LOG.error(String.format("BigQuery Error : %s %s", finalError, causativeError)); + LOG.error("BigQuery Error : {} {}", finalError, causativeError); } } 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 960e3e2747f8..23ca5c1ce213 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 @@ -781,8 +781,7 @@ public static TypedRead read(SerializableFunction par (writer, reader) -> new GenericDatumTransformer<>(parseFn, jsonTableSchema, writer); } catch (IOException e) { - LOG.warn( - String.format("Error while converting table schema %s to JSON!", input), e); + LOG.warn("Error while converting table schema {} to JSON!", input, e); return null; } }) 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 36906aee15b9..60593da55093 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 @@ -383,7 +383,7 @@ static void startJob( return; // SUCCEEDED } catch (IOException e) { if (errorExtractor.itemAlreadyExists(e)) { - LOG.info("BigQuery job " + jobRef + " already exists, will not retry inserting it:", e); + LOG.info("BigQuery job {} already exists, will not retry inserting it:", jobRef, e); return; // SUCCEEDED } @@ -395,7 +395,7 @@ static void startJob( .create() : null) { // ignore and retry - LOG.info("Failed to insert job " + jobRef + ", will retry:", e); + LOG.info("Failed to insert job {}, will retry:", jobRef, e); } lastException = e; } @@ -431,12 +431,11 @@ static void startJobStream( } catch (IOException e) { if (errorExtractor.itemAlreadyExists(e)) { LOG.info( - "BigQuery job " + jobReference + " already exists, will not retry inserting it:", - e); + "BigQuery job {} already exists, will not retry inserting it:", jobReference, e); return; // SUCCEEDED } // ignore and retry - LOG.info("Failed to insert job " + jobReference + ", will retry:", e); + LOG.info("Failed to insert job {}, will retry:", jobReference, e); exception = e; } } while (nextBackOff(sleeper, backOff)); @@ -1065,9 +1064,8 @@ public List call() throws Exception { .withFullResourceName(BigQueryHelpers.toTableFullResourceName(ref)) .create()) { LOG.info( - String.format( - "BigQuery insertAll error, retrying: %s", - ApiErrorExtractor.INSTANCE.getErrorMessage(e))); + "BigQuery insertAll error, retrying: {}", + ApiErrorExtractor.INSTANCE.getErrorMessage(e)); } try { long nextBackOffMillis = backoff1.nextBackOffMillis(); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java index 52b5b954a095..6370244c268c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java @@ -451,7 +451,7 @@ private MatchTableDynamicDestinations( try (DatasetService datasetService = bqServices.getDatasetService(bqOptions)) { return datasetService.getTable(tableReference); } catch (InterruptedException | IOException e) { - LOG.info("Failed to get BigQuery table " + tableReference); + LOG.info("Failed to get BigQuery table {}", tableReference); } } catch (Exception e) { throw new RuntimeException(e); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFinalizeWritesDoFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFinalizeWritesDoFn.java index cbe018ba739c..bdd0792288bb 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFinalizeWritesDoFn.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFinalizeWritesDoFn.java @@ -120,7 +120,7 @@ public void process(PipelineOptions pipelineOptions, @Element KV contexts -> { RetryManager.Operation.Context firstContext = Preconditions.checkArgumentNotNull(Iterables.getFirst(contexts, null)); - LOG.error("Finalize of stream " + streamId + " failed with " + firstContext.getError()); + LOG.error("Finalize of stream {} failed", streamId, firstContext.getError()); finalizeOperationsFailed.inc(); BigQuerySinkMetrics.reportFailedRPCMetrics( firstContext, BigQuerySinkMetrics.RpcMethod.FINALIZE_STREAM); @@ -132,7 +132,7 @@ public void process(PipelineOptions pipelineOptions, @Element KV Preconditions.checkArgumentNotNull( c.getResult(), "Finalize of write stream " + streamId + " finished, but with null result"); - LOG.debug("Finalize of stream " + streamId + " finished with " + response); + LOG.debug("Finalize of stream {} finished with {}", streamId, response); rowsFinalized.inc(response.getRowCount()); finalizeOperationsSucceeded.inc(); @@ -169,17 +169,15 @@ public void finishBundle(PipelineOptions pipelineOptions) throws Exception { RetryManager.Operation.Context firstContext = Preconditions.checkArgumentNotNull(Iterables.getFirst(contexts, null)); LOG.error( - "BatchCommit failed. tableId " - + tableId - + " streamNames " - + streamNames - + " error: " - + firstContext.getError()); + "BatchCommit failed. tableId {} streamNames {}", + tableId, + streamNames, + firstContext.getError()); batchCommitOperationsFailed.inc(); return RetryType.RETRY_ALL_OPERATIONS; }, c -> { - LOG.info("BatchCommit succeeded for tableId " + tableId + " response " + c.getResult()); + LOG.info("BatchCommit succeeded for tableId {} response {}", tableId, c.getResult()); batchCommitOperationsSucceeded.inc(); }, response -> { 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 fd3853d15e0f..93383fd50507 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 @@ -212,8 +212,7 @@ public void process(PipelineOptions pipelineOptions, @Element KV failedContext = Preconditions.checkArgumentNotNull(Iterables.getFirst(contexts, null)); Throwable error = failedContext.getError(); - LOG.warn( - "Flush of stream " + streamId + " to offset " + offset + " failed with " + error); + LOG.warn("Flush of stream {} to offset {} failed", streamId, offset, error); flushOperationsFailed.inc(); BigQuerySinkMetrics.reportFailedRPCMetrics( failedContext, BigQuerySinkMetrics.RpcMethod.FLUSH_ROWS); @@ -286,11 +285,9 @@ public void process(PipelineOptions pipelineOptions, @Element KV { LOG.warn( - "Finalize of stream " - + streamId - + " failed with " - + Preconditions.checkArgumentNotNull(Iterables.getFirst(contexts, null)) - .getError()); + "Finalize of stream {} failed", + streamId, + Preconditions.checkArgumentNotNull(Iterables.getFirst(contexts, null)).getError()); finalizeOperationsFailed.inc(); @Nullable Context firstContext = Iterables.getFirst(contexts, null); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java index 41bf06d7af23..fc26647777ac 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java @@ -135,7 +135,7 @@ public class StorageApiWriteUnshardedRecords .expireAfterAccess(15, TimeUnit.MINUTES) .removalListener( (RemovalNotification removal) -> { - LOG.info("Expiring append client for " + removal.getKey()); + LOG.info("Expiring append client for {}", removal.getKey()); final @Nullable AppendClientInfo appendClientInfo = removal.getValue(); if (appendClientInfo != null) { appendClientInfo.close(); 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 03a5924cacb3..be9495ca19e5 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 @@ -799,7 +799,7 @@ public void process( // Don't log errors for expected offset mismatch. These will be logged as warnings // below. LOG.error( - "Got error " + failedContext.getError() + " closing " + failedContext.streamName); + "Got error {} closing {}", failedContext.getError(), failedContext.streamName); } try { @@ -833,11 +833,9 @@ public void process( if (offsetMismatch || streamDoesNotExist) { appendOffsetFailures.inc(); LOG.warn( - "Append to " - + failedContext - + " failed with " - + failedContext.getError() - + " Will retry with a new stream"); + "Append to {} failed. Will retry with a new stream", + failedContext, + failedContext.getError()); // Finalize the stream and clear streamName so a new stream will be created. o.get(flushTag) .output( @@ -901,9 +899,8 @@ public void process( // the ProtoRows iterable at 2MB and the max request size is 10MB, this scenario seems // nearly impossible. LOG.error( - "A request containing more than one row is over the request size limit of " - + maxRequestSize - + ". This is unexpected. All rows in the request will be sent to the failed-rows PCollection."); + "A request containing more than one row is over the request size limit of {}. This is unexpected. All rows in the request will be sent to the failed-rows PCollection.", + maxRequestSize); } for (int i = 0; i < splitValue.getProtoRows().getSerializedRowsCount(); ++i) { org.joda.time.Instant timestamp = splitValue.getTimestamps().get(i); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableSchemaCache.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableSchemaCache.java index 0013cd3d16ae..1e80a91824ad 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableSchemaCache.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableSchemaCache.java @@ -294,7 +294,7 @@ public void refreshThread() { // Since this is a daemon thread, don't exit until it is explicitly shut down. Exiting early // can cause the // pipeline to stall. - LOG.error("Caught exception in BigQuery's table schema cache refresh thread: " + e); + LOG.error("Caught exception in BigQuery's table schema cache refresh thread", e); } this.refreshExecutor.submit(this::refreshThread); } @@ -313,7 +313,7 @@ private Map refreshAll(Map tables) if (table == null) { throw new RuntimeException("Did not get value for table " + tableReference); } - LOG.info("Refreshed BigQuery schema for " + entry.getKey()); + LOG.info("Refreshed BigQuery schema for {}", entry.getKey()); schemas.put(entry.getKey(), table.getSchema()); } return schemas; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 58d73af7effb..100078d32e70 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -1545,7 +1545,7 @@ private void checkForFailures() throws IOException { String.format( "At least %d errors occurred writing to Bigtable. First %d errors: %s", i + failures.size(), i, logEntry.toString()); - LOG.error(message); + LOG.error("{}", message); IOException exception = new IOException(message); for (BigtableWriteException e : suppressed) { exception.addSuppressed(e); @@ -2462,7 +2462,7 @@ private void createOrUpdateMetadataTable( // Only try to create or update metadata table if option is set to true. Otherwise, just // check if the table exists. if (shouldCreateOrUpdateMetadataTable && metadataTableAdminDao.createMetadataTable()) { - LOG.info("Created metadata table: " + metadataTableId); + LOG.info("Created metadata table: {}", metadataTableId); } } @@ -2645,7 +2645,7 @@ public static boolean createOrUpdateReadChangeStreamMetadataTable( // Only try to create or update metadata table if option is set to true. Otherwise, just // check if the table exists. if (metadataTableAdminDao.createMetadataTable()) { - LOG.info("Created metadata table: " + metadataTableAdminDao.getTableId()); + LOG.info("Created metadata table: {}", metadataTableAdminDao.getTableId()); } return metadataTableAdminDao.doesMetadataTableExist(); } finally { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java index 33b742d2db57..0a9185ee0b93 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java @@ -81,10 +81,10 @@ public void close() { refCounts.getOrDefault(getConfigId().id(), new AtomicInteger(0)).decrementAndGet(); if (refCount < 0) { LOG.error( - "close() Ref count is < 0, configId=" + getConfigId().id() + " refCount=" + refCount); + "close() Ref count is < 0, configId={} refCount={}", getConfigId().id(), refCount); } LOG.debug( - "close() is called for config id " + getConfigId().id() + ", ref count is " + refCount); + "close() is called for config id {}, ref count is {}", getConfigId().id(), refCount); if (refCount == 0) { entries.remove(getConfigId().id()); refCounts.remove(getConfigId().id()); @@ -101,14 +101,14 @@ BigtableServiceEntry getServiceForReading( PipelineOptions pipelineOptions) throws IOException { synchronized (lock) { - LOG.debug("getServiceForReading(), config id: " + configId.id()); + LOG.debug("getServiceForReading(), config id: {}", configId.id()); BigtableServiceEntry entry = entries.get(configId.id()); if (entry != null) { // When entry is not null, refCount.get(configId.id()) should always exist. // Doing a putIfAbsent to avoid NPE. AtomicInteger count = refCounts.putIfAbsent(configId.id(), new AtomicInteger(0)); if (count == null) { - LOG.error("entry is not null but refCount of config Id " + configId.id() + " is null."); + LOG.error("entry is not null but refCount of config Id {} is null.", configId.id()); } refCounts.get(configId.id()).getAndIncrement(); LOG.debug("getServiceForReading() returning an existing service entry"); @@ -147,13 +147,13 @@ BigtableServiceEntry getServiceForWriting( throws IOException { synchronized (lock) { BigtableServiceEntry entry = entries.get(configId.id()); - LOG.debug("getServiceForWriting(), config id: " + configId.id()); + LOG.debug("getServiceForWriting(), config id: {}", configId.id()); if (entry != null) { // When entry is not null, refCount.get(configId.id()) should always exist. // Doing a putIfAbsent to avoid NPE. AtomicInteger count = refCounts.putIfAbsent(configId.id(), new AtomicInteger(0)); if (count == null) { - LOG.error("entry is not null but refCount of config Id " + configId.id() + " is null."); + LOG.error("entry is not null but refCount of config Id {} is null.", configId.id()); } refCounts.get(configId.id()).getAndIncrement(); LOG.debug("getServiceForWriting() returning an existing service entry"); @@ -201,7 +201,7 @@ boolean checkTableExists(BigtableConfig config, PipelineOptions pipelineOptions, return false; } String message = String.format("Error checking whether table %s exists", tableId); - LOG.error(message, e); + LOG.error("Error checking whether table {} exists", tableId, e); throw new IOException(message, e); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/action/DetectNewPartitionsAction.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/action/DetectNewPartitionsAction.java index 8354b7efc43d..08f310515f43 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/action/DetectNewPartitionsAction.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/action/DetectNewPartitionsAction.java @@ -158,7 +158,7 @@ private Optional getNewWatermark( List missingPartitions = getMissingPartitionsFromEntireKeySpace(partitions); if (missingPartitions.isEmpty()) { - LOG.info("DNP: Updating watermark: " + lowWatermark); + LOG.info("DNP: Updating watermark: {}", lowWatermark); return Optional.of(lowWatermark); } LOG.warn( @@ -253,8 +253,8 @@ public ProcessContinuation run( ManualWatermarkEstimator watermarkEstimator, InitialPipelineState initialPipelineState) throws Exception { - LOG.debug("DNP: Watermark: " + watermarkEstimator.getState()); - LOG.debug("DNP: CurrentTracker: " + tracker.currentRestriction().getFrom()); + LOG.debug("DNP: Watermark: {}", watermarkEstimator.getState()); + LOG.debug("DNP: CurrentTracker: {}", tracker.currentRestriction().getFrom()); if (tracker.currentRestriction().getFrom() == 0L) { if (!tracker.tryClaim(0L)) { LOG.error( @@ -298,7 +298,7 @@ public ProcessContinuation run( } if (!tracker.tryClaim(tracker.currentRestriction().getFrom())) { - LOG.warn("DNP: Checkpointing, stopping this run: " + tracker.currentRestriction()); + LOG.warn("DNP: Checkpointing, stopping this run: {}", tracker.currentRestriction()); return ProcessContinuation.stop(); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/dao/MetadataTableDao.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/dao/MetadataTableDao.java index 9363fc8c8689..6a5ac4354cae 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/dao/MetadataTableDao.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/dao/MetadataTableDao.java @@ -413,8 +413,8 @@ public boolean deleteNewPartition(NewPartition newPartition) { public List readStreamPartitionsWithWatermark() throws InvalidProtocolBufferException { LOG.debug( - "Reading stream partitions from metadata table: " - + getFullStreamPartitionPrefix().toStringUtf8()); + "Reading stream partitions from metadata table: {}", + getFullStreamPartitionPrefix().toStringUtf8()); Filter filterForWatermark = FILTERS .chain() diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/dofn/InitializeDoFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/dofn/InitializeDoFn.java index c8ee12772cc0..1bf2b5788592 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/dofn/InitializeDoFn.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/dofn/InitializeDoFn.java @@ -50,9 +50,9 @@ public InitializeDoFn( @ProcessElement public void processElement(OutputReceiver receiver) throws IOException { - LOG.info(daoFactory.getStreamTableDebugString()); - LOG.info(daoFactory.getMetadataTableDebugString()); - LOG.info("ChangeStreamName: " + daoFactory.getChangeStreamName()); + LOG.info("{}", daoFactory.getStreamTableDebugString()); + LOG.info("{}", daoFactory.getMetadataTableDebugString()); + LOG.info("ChangeStreamName: {}", daoFactory.getChangeStreamName()); boolean resume = false; DetectNewPartitionsState detectNewPartitionsState = diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/EntityToRow.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/EntityToRow.java index 1c0bef6f2200..6bc3883746db 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/EntityToRow.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/EntityToRow.java @@ -57,7 +57,7 @@ private EntityToRow(Schema schema, String keyField) { + "` should of type `BYTES`. Please change the type or specify a field to" + " store the KEY value."); } - LOG.info("Entity KEY will be stored under `" + keyField + "` field."); + LOG.info("Entity KEY will be stored under `{}` field.", keyField); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/RowToEntity.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/RowToEntity.java index 2b1c596d4cc1..dbeb07e3f80a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/RowToEntity.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/RowToEntity.java @@ -72,7 +72,7 @@ public PCollection expand(PCollection input) { + "` should of type `VARBINARY`. Please change the type or specify a field to" + " write the KEY value from via TableProperties."); } - LOG.info("Field to use as Entity KEY is set to: `" + keyField + "`."); + LOG.info("Field to use as Entity KEY is set to: `{}`.", keyField); } return input.apply(ParDo.of(new RowToEntity.RowToEntityConverter(isFieldPresent))); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1WriteFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1WriteFn.java index ab33d8e5c166..b95dfd78d144 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1WriteFn.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1WriteFn.java @@ -285,7 +285,7 @@ public void processElement(ProcessContext context, BoundedWindow window) throws .setMessage(message) .build()), window)), - () -> LOG.info(message)); + () -> LOG.info("{}", message)); } else { writes.offer(new WriteElement(queueNextEntryPriority++, write, window)); flushBatch(/* finishingBundle */ false, contextAdapter); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java index 5b8eda8d6347..c9df8e7ec9b3 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java @@ -84,7 +84,6 @@ import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.TypeDescriptors; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Throwables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; @@ -420,13 +419,14 @@ private static void incrementLroCounters( com.google.api.services.healthcare.v1.model.Status error = operation.getError(); if (error == null) { operationSuccessCounter.inc(); - LOG.debug(String.format("Operation %s finished successfully.", operation.getName())); + LOG.debug("Operation {} finished successfully.", operation.getName()); } else { operationFailureCounter.inc(); LOG.error( - String.format( - "Operation %s failed with error code: %d and message: %s.", - operation.getName(), error.getCode(), error.getMessage())); + "Operation {} failed with error code: {} and message: {}.", + operation.getName(), + error.getCode(), + error.getMessage()); } // Update resource counters. @@ -441,11 +441,11 @@ private static void incrementLroCounters( Long numFailures = Long.parseLong(counters.get(LRO_FAILURE_KEY)); resourceFailureCounter.inc(numFailures); if (numFailures > 0) { - LOG.error("Operation " + operation.getName() + " had " + numFailures + " failures."); + LOG.error("Operation {} had {} failures.", operation.getName(), numFailures); } } } catch (Exception e) { - LOG.error("failed to increment LRO counters, error message: " + e.getMessage()); + LOG.error("failed to increment LRO counters, error message", e); } } } @@ -607,10 +607,9 @@ public void processElement(ProcessContext context) { } catch (Exception e) { READ_RESOURCE_ERRORS.inc(); LOG.warn( - String.format( - "Error fetching Fhir resource with ID %s writing to Dead Letter " - + "Queue. Cause: %s Stack Trace: %s", - resourceId, e.getMessage(), Throwables.getStackTraceAsString(e))); + "Error fetching Fhir resource with ID {} writing to Dead Letter Queue. ", + resourceId, + e); context.output(FhirIO.Read.DEAD_LETTER, HealthcareIOError.of(resourceId, e)); } } @@ -1137,7 +1136,7 @@ public void delete(@Element Metadata path, ProcessContext context) { Collections.singleton(path.resourceId()), StandardMoveOptions.IGNORE_MISSING_FILES); } catch (IOException e) { - LOG.error("error cleaning up tempGcsDir: %s", e); + LOG.error("error cleaning up tempGcsDir", e); } } })) @@ -1210,7 +1209,7 @@ public void addToFile(ProcessContext context, BoundedWindow window) throws IOExc "Failed to parse payload: %s as json at: %s : %s." + "Dropping resource from batch import.", httpBody, e.getLocation().getCharOffset(), e.getMessage()); - LOG.warn(resource); + LOG.warn("{}", resource); context.output( Write.FAILED_BODY, HealthcareIOError.of(httpBody, new IOException(resource))); } @@ -1344,9 +1343,10 @@ public void importBatch(FinishBundleContext context) throws IOException { ResourceId deadLetterResourceId = FileSystems.matchNewResource(deadLetterGcsPath.get(), true); LOG.warn( - String.format( - "Failed to import %s with error: %s. Moving to deadletter path %s", - importUri, e.getMessage(), deadLetterResourceId.toString())); + "Failed to import {} with error. Moving to deadletter path {}", + importUri, + deadLetterResourceId.toString(), + e); IMPORT_OPERATION_ERRORS.inc(); FileSystems.rename(tempDestinations, deadLetterDestinations); @@ -1988,11 +1988,7 @@ public void processElement(ProcessContext context) { fhirSearchParameters.getResourceType(), fhirSearchParameters.getQueries()))); } catch (IllegalArgumentException | NoSuchElementException e) { searchResourcesErrorCount.inc(); - log.warn( - String.format( - "Error search FHIR resources writing to Dead Letter " - + "Queue. Cause: %s Stack Trace: %s", - e.getMessage(), Throwables.getStackTraceAsString(e))); + log.warn("Error search FHIR resources writing to Dead Letter Queue.", e); context.output( FhirIO.Search.DEAD_LETTER, HealthcareIOError.of(fhirSearchParameters.toString(), e)); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOPatientEverything.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOPatientEverything.java index e2a5937a67ed..e0d3647d25e6 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOPatientEverything.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOPatientEverything.java @@ -46,7 +46,6 @@ import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Throwables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; @@ -207,10 +206,8 @@ public void processElement(ProcessContext context) { } catch (IllegalArgumentException | NoSuchElementException e) { GET_PATIENT_EVERYTHING_ERROR_COUNT.inc(); LOG.warn( - String.format( - "Error executing GetPatientEverything: FHIR resources writing to Dead Letter " - + "Queue. Cause: %s Stack Trace: %s", - e.getMessage(), Throwables.getStackTraceAsString(e))); + "Error executing GetPatientEverything: FHIR resources writing to Dead Letter Queue.", + e); context.output(DEAD_LETTER, HealthcareIOError.of(patientEverythingParameter.toString(), e)); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java index 16269b7ab33a..3647ef7671eb 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java @@ -54,7 +54,6 @@ import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.sdk.values.TypeDescriptors; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Throwables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.FluentIterable; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; @@ -564,10 +563,7 @@ private HL7v2Message fetchMessage(String msgId) } catch (Exception e) { failedMessageGets.inc(); LOG.warn( - String.format( - "Error fetching HL7v2 message with ID %s writing to Dead Letter " - + "Queue. Cause: %s Stack Trace: %s", - msgId, e.getMessage(), Throwables.getStackTraceAsString(e))); + "Error fetching HL7v2 message with ID {} writing to Dead Letter Queue. ", msgId, e); throw e; } } @@ -696,19 +692,14 @@ public void split(@Restriction OffsetRange timeRange, OutputReceiver err = HealthcareIOError.of(msg, e); - LOG.warn(String.format("%s %s", err.getErrorMessage(), err.getStackTrace())); + LOG.warn("{} {}", err.getErrorMessage(), err.getStackTrace()); context.output(err); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java index 1b67e516bfbe..d46662dd4e4f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java @@ -313,9 +313,7 @@ public Instant getEarliestHL7v2SendTime(String hl7v2Store, @Nullable String filt String sendTime = response.getHl7V2Messages().get(0).getSendTime(); if (Strings.isNullOrEmpty(sendTime)) { LOG.warn( - String.format( - "Earliest message in %s has null or empty sendTime defaulting to Epoch.", - hl7v2Store)); + "Earliest message in {} has null or empty sendTime defaulting to Epoch.", hl7v2Store); return Instant.ofEpochMilli(0); } // sendTime is conveniently RFC3339 UTC "Zulu" @@ -349,9 +347,8 @@ public Instant getLatestHL7v2SendTime(String hl7v2Store, @Nullable String filter } String sendTime = response.getHl7V2Messages().get(0).getSendTime(); if (Strings.isNullOrEmpty(sendTime)) { - LOG.warn( - String.format( - "Latest message in %s has null or empty sendTime defaulting to now.", hl7v2Store)); + LOG.warn("Latest message in {} has null or empty sendTime defaulting to now.", hl7v2Store); + return Instant.now(); } // sendTime is conveniently RFC3339 UTC "Zulu" @@ -560,7 +557,7 @@ public Operation deidentifyFhirStore( @Override public Operation pollOperation(Operation operation, Long sleepMs) throws InterruptedException, IOException { - LOG.debug(String.format("Operation %s started, polling until complete.", operation.getName())); + LOG.debug("Operation {} started, polling until complete.", operation.getName()); while (operation.getDone() == null || !operation.getDone()) { // Update the status of the operation with another request. Thread.sleep(sleepMs); // Pause between requests. diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/AddTimestampAttribute.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/AddTimestampAttribute.java index 6abd9c0b9622..0ecafc82f90b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/AddTimestampAttribute.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/AddTimestampAttribute.java @@ -57,10 +57,9 @@ public PCollection expand(PCollection input) { // Warn the user if they're writing data to TIMESTAMP_FIELD, but event timestamp is mapped // to publish time. The data will be dropped. LOG.warn( - String.format( - "Dropping output field '%s' before writing to PubSub because this is a read-only " - + "column. To preserve this information you must configure a timestamp attribute.", - TIMESTAMP_FIELD)); + "Dropping output field '{}' before writing to PubSub because this is a read-only " + + "column. To preserve this information you must configure a timestamp attribute.", + TIMESTAMP_FIELD); } rows = withTimestamp.apply(DropFields.fields(TIMESTAMP_FIELD)); } else { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/BatchSpannerRead.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/BatchSpannerRead.java index 42022fca9658..d1c6f894dd7c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/BatchSpannerRead.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/BatchSpannerRead.java @@ -255,7 +255,7 @@ public void processElement(ProcessContext c) throws Exception { } catch (SpannerException e) { serviceCallMetric.call(e.getErrorCode().getGrpcStatusCode().toString()); LOG.error( - "Error while reading partition for operation: " + op.getReadOperation().toString(), e); + "Error while reading partition for operation: {}", op.getReadOperation().toString(), e); throw (e); } serviceCallMetric.call("ok"); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerRead.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerRead.java index 3d818bd30df5..feb8a1e4cc05 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerRead.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerRead.java @@ -121,7 +121,7 @@ public void processElement(ProcessContext c) throws Exception { } } catch (SpannerException e) { serviceCallMetric.call(e.getErrorCode().getGrpcStatusCode().toString()); - LOG.error("Error while reading operation: " + op, e); + LOG.error("Error while reading operation: {}", op, e); throw (e); } serviceCallMetric.call("ok"); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index 3a69d1177f4a..62ba95dbd5fd 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -1973,10 +1973,9 @@ && getInclusiveStartAt().toSqlTimestamp().after(getInclusiveEndAt().toSqlTimesta final Dialect metadataDatabaseDialect = getDialect(partitionMetadataSpannerConfig, input.getPipeline().getOptions()); LOG.info( - "The Spanner database " - + changeStreamDatabaseId - + " has dialect " - + changeStreamDatabaseDialect); + "The Spanner database {} has dialect {}", + changeStreamDatabaseId, + changeStreamDatabaseDialect); PartitionMetadataTableNames partitionMetadataTableNames = Optional.ofNullable(getMetadataTable()) .map( @@ -2000,7 +1999,7 @@ && getInclusiveStartAt().toSqlTimestamp().after(getInclusiveEndAt().toSqlTimesta final boolean isMutableChangeStream = isMutableChangeStream( spannerAccessor.getDatabaseClient(), changeStreamDatabaseDialect, changeStreamName); - LOG.info("The change stream " + changeStreamName + " is mutable: " + isMutableChangeStream); + LOG.info("The change stream {} is mutable: {}", changeStreamName, isMutableChangeStream); final DaoFactory daoFactory = new DaoFactory( changeStreamSpannerConfig, @@ -2029,8 +2028,8 @@ && getInclusiveStartAt().toSqlTimestamp().after(getInclusiveEndAt().toSqlTimesta new PostProcessingMetricsDoFn(metrics); LOG.info( - "Partition metadata table that will be used is " - + partitionMetadataTableNames.getTableName()); + "Partition metadata table that will be used is {}", + partitionMetadataTableNames.getTableName()); final PCollection impulseOut = input.apply(Impulse.create()); final PCollection partitionsOut = @@ -2533,7 +2532,7 @@ public void processElement(ProcessContext c) throws Exception { mutationGroupsWriteSuccess.inc(); } catch (SpannerException e) { mutationGroupsWriteFail.inc(); - LOG.warn("Failed to write the mutation group: " + mg, e); + LOG.warn("Failed to write the mutation group: {}", mg, e); c.output(failedTag, mg); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClient.java index 0e9476e6a226..a42c367670cb 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClient.java @@ -444,7 +444,7 @@ public void createNewDataset( .setLocation(location)) .execute(); if (response != null) { - LOG.info("Successfully created new dataset : " + response.getId()); + LOG.info("Successfully created new dataset : {}", response.getId()); return; } else { lastException = @@ -467,9 +467,9 @@ public void createNewDataset( public void deleteTable(String projectId, String datasetId, String tableName) { try { bqClient.tables().delete(projectId, datasetId, tableName).execute(); - LOG.info("Successfully deleted table: " + tableName); + LOG.info("Successfully deleted table: {}", tableName); } catch (Exception e) { - LOG.debug("Exception caught when deleting table: " + e.getMessage()); + LOG.debug("Exception caught when deleting table", e); } } @@ -480,14 +480,14 @@ public void deleteDataset(String projectId, String datasetId) { this.deleteTable(projectId, datasetId, table.getTableReference().getTableId()); } } catch (Exception e) { - LOG.debug("Exceptions caught when listing all tables: " + e.getMessage()); + LOG.debug("Exceptions caught when listing all tables", e); } try { bqClient.datasets().delete(projectId, datasetId).execute(); - LOG.info("Successfully deleted dataset: " + datasetId); + LOG.info("Successfully deleted dataset: {}", datasetId); } catch (Exception e) { - LOG.debug("Exceptions caught when deleting dataset: " + e.getMessage()); + LOG.debug("Exceptions caught when deleting dataset", e); } } @@ -503,7 +503,7 @@ public void createNewTable(String projectId, String datasetId, Table newTable) try { Table response = this.bqClient.tables().insert(projectId, datasetId, newTable).execute(); if (response != null) { - LOG.info("Successfully created new table: " + response.getId()); + LOG.info("Successfully created new table: {}", response.getId()); return; } else { lastException = @@ -547,7 +547,7 @@ public void insertDataToTable( .execute(); if (response != null && (response.getInsertErrors() == null || response.getInsertErrors().isEmpty())) { - LOG.info("Successfully inserted data into table : " + tableName); + LOG.info("Successfully inserted data into table : {}", tableName); return; } else { if (response == null || response.getInsertErrors() == null) { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/dao/MetadataTableDaoTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/dao/MetadataTableDaoTest.java index 9c0fbcfec440..f1ae915a012e 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/dao/MetadataTableDaoTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/dao/MetadataTableDaoTest.java @@ -181,7 +181,7 @@ public void run() { locked = true; } } catch (InterruptedException e) { - LOG.error(e.toString()); + LOG.error("Failed to wait for futures", e); } } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/it/BigtableChangeStreamIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/it/BigtableChangeStreamIT.java index f38b4b5cd9bc..d1e0ace7eb8f 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/it/BigtableChangeStreamIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/it/BigtableChangeStreamIT.java @@ -88,7 +88,7 @@ public static void beforeClass() throws IOException { appProfileId = "default"; bigtableClientOverride = new BigtableClientIntegrationTestOverride(); - LOG.info(bigtableClientOverride.toString()); + LOG.info("{}", bigtableClientOverride); BigtableDataSettings.Builder dataSettingsBuilder = BigtableDataSettings.newBuilder(); BigtableTableAdminSettings.Builder tableAdminSettingsBuilder = diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/IntegrationTestEnv.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/IntegrationTestEnv.java index 581b7f3cc2f5..12f264148f47 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/IntegrationTestEnv.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/IntegrationTestEnv.java @@ -129,7 +129,7 @@ protected void after() { .get(TIMEOUT_MINUTES, TimeUnit.MINUTES); } } catch (Exception e) { - LOG.error("Failed to drop change stream " + changeStream + ". Skipping...", e); + LOG.error("Failed to drop change stream {}. Skipping...", changeStream, e); } } @@ -150,14 +150,14 @@ protected void after() { .get(TIMEOUT_MINUTES, TimeUnit.MINUTES); } } catch (Exception e) { - LOG.error("Failed to drop table " + table + ". Skipping...", e); + LOG.error("Failed to drop table {}. Skipping...", table, e); } } try { databaseAdminClient.dropDatabase(instanceId, databaseId); } catch (Exception e) { - LOG.error("Failed to drop database " + databaseId + ". Skipping...", e); + LOG.error("Failed to drop database {}. Skipping...", databaseId, e); } if (useSeparateMetadataDb) { databaseAdminClient.dropDatabase(instanceId, metadataDatabaseId); @@ -172,7 +172,7 @@ void createMetadataDatabase() throws ExecutionException, InterruptedException, T String createSingersTable() throws InterruptedException, ExecutionException, TimeoutException { final String tableName = generateTableName(SINGERS_TABLE_NAME_PREFIX); - LOG.info("Creating table " + tableName); + LOG.info("Creating table {}", tableName); if (this.isPostgres) { databaseAdminClient .updateDatabaseDdl( @@ -229,7 +229,7 @@ String createGSQLTableDDL(String tableName) { String createChangeStreamFor(String tableName) throws InterruptedException, ExecutionException, TimeoutException { final String changeStreamName = generateChangeStreamName(); - LOG.info("CREATE CHANGE STREAM \"" + changeStreamName + "\" FOR \"" + tableName + "\""); + LOG.info("CREATE CHANGE STREAM \"{}\" FOR \"{}\"", changeStreamName, tableName); if (this.isPostgres) { databaseAdminClient .updateDatabaseDdl( @@ -324,7 +324,7 @@ private void recreateDatabase( throws ExecutionException, InterruptedException, TimeoutException { // Drops the database if it already exists databaseAdminClient.dropDatabase(instanceId, databaseId); - LOG.info("Creating database " + databaseId + ", isPostgres=" + isPostgres); + LOG.info("Creating database {}, isPostgres={}", databaseId, isPostgres); if (isPostgres) { databaseAdminClient .createDatabase( @@ -347,7 +347,7 @@ private void recreateDatabase( private String generateTableName(String prefix) { int maxTableNameLength = MAX_POSTGRES_TABLE_NAME_LENGTH; - LOG.info("Max table length: " + maxTableNameLength); + LOG.info("Max table length: {}", maxTableNameLength); return prefix + "_" + RandomStringUtils.randomAlphanumeric(maxTableNameLength - 1 - prefix.length()); 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 21f6eef79362..88bee4ed7c0d 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 @@ -114,7 +114,7 @@ public void testTransactionBoundaries() { try { Thread.sleep(timeIncrementInSeconds * 1000); } catch (InterruptedException e) { - LOG.error(e.toString(), e); + LOG.error("Interrupted while waiting", e); } // This will be the second batch of transactions that will have strict timestamp ordering @@ -125,7 +125,7 @@ public void testTransactionBoundaries() { try { Thread.sleep(timeIncrementInSeconds * 1000); } catch (InterruptedException e) { - LOG.error(e.toString(), e); + LOG.error("Interrupted while waiting", e); } // This will be the final batch of transactions that will have strict timestamp ordering @@ -413,8 +413,7 @@ record : records) { 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()); + LOG.debug("Timer not being set as exceeded pipeline end time: {}", pipelineEndTime); } } } @@ -475,27 +474,27 @@ private Timestamp writeTransactionsToDatabase() { 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()); + LOG.debug("The first transaction committed with timestamp: {}", t1); 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()); + LOG.debug("The second transaction committed with timestamp: {}", t2); 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()); + LOG.debug("The third transaction committed with timestamp: {}", t3); 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()); + LOG.debug("The fourth transaction committed with timestamp: {}", t4); return t4; } 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 2b2c134032b3..b93d74eff3d8 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 @@ -114,7 +114,7 @@ public void testOrderedWithinKey() { try { Thread.sleep(timeIncrementInSeconds * 1000); } catch (InterruptedException e) { - LOG.error(e.toString(), e); + LOG.error("Interrupted while waiting", e); } // This will be the second batch of transactions that will have strict timestamp ordering @@ -125,7 +125,7 @@ public void testOrderedWithinKey() { try { Thread.sleep(timeIncrementInSeconds * 1000); } catch (InterruptedException e) { - LOG.error(e.toString(), e); + LOG.error("Interrupted while waiting", e); } // This will be the final batch of transactions that will have strict timestamp ordering @@ -497,27 +497,27 @@ private com.google.cloud.Timestamp writeTransactionsToDatabase() { mutations.add(insertRecordMutation(1)); mutations.add(insertRecordMutation(2)); com.google.cloud.Timestamp t1 = databaseClient.write(mutations); - LOG.info("The first transaction committed with timestamp: " + t1.toString()); + LOG.info("The first transaction committed with timestamp: {}", t1); 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.info("The second transaction committed with timestamp: " + t2.toString()); + LOG.info("The second transaction committed with timestamp: {}", t2); 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.info("The third transaction committed with timestamp: " + t3.toString()); + LOG.info("The third transaction committed with timestamp: {}", t3); mutations.clear(); // 4. Commit a transaction to delete Singer 0. mutations.add(deleteRecordMutation(0)); com.google.cloud.Timestamp t4 = databaseClient.write(mutations); - LOG.info("The fourth transaction committed with timestamp: " + t4.toString()); + LOG.info("The fourth transaction committed with timestamp: {}", t4); return t4; } 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 11d91e7695f1..009505c334c4 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 @@ -86,7 +86,7 @@ public static void setup() throws InterruptedException, ExecutionException, Time @Test public void testOrderedWithinKey() { - LOG.info("Test pipeline: " + pipeline.toString()); + LOG.info("Test pipeline: {}", pipeline); final SpannerConfig spannerConfig = SpannerConfig.create() .withProjectId(projectId) @@ -319,14 +319,14 @@ private static 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.debug("The first transaction committed with timestamp: {}", t1); 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)); com.google.cloud.Timestamp t2 = databaseClient.write(mutations); - LOG.debug("The second transaction committed with timestamp: " + t2.toString()); + LOG.debug("The second transaction committed with timestamp: {}", t2); mutations.clear(); // 3. Commit a transaction to insert Singer 3 and Singer 5. @@ -335,14 +335,14 @@ private static com.google.cloud.Timestamp writeTransactionsToDatabase() { mutations.add(insertRecordMutation(5)); mutations.add(updateRecordMutation(5)); com.google.cloud.Timestamp t3 = databaseClient.write(mutations); - LOG.debug("The third transaction committed with timestamp: " + t3.toString()); + LOG.debug("The third transaction committed with timestamp: {}", t3); mutations.clear(); // 4. Commit a transaction to update Singer 3 and Singer 2 in the table. mutations.add(updateRecordMutation(3)); mutations.add(updateRecordMutation(2)); com.google.cloud.Timestamp t4 = databaseClient.write(mutations); - LOG.debug("The fourth transaction committed with timestamp: " + t4.toString()); + LOG.debug("The fourth transaction committed with timestamp: {}", t4); mutations.clear(); // 5. Commit a transaction to delete 4, insert 1, delete 3, update 5. @@ -352,7 +352,7 @@ private static com.google.cloud.Timestamp writeTransactionsToDatabase() { mutations.add(updateRecordMutation(5)); com.google.cloud.Timestamp t5 = databaseClient.write(mutations); - LOG.debug("The fifth transaction committed with timestamp: " + t5.toString()); + LOG.debug("The fifth transaction committed with timestamp: {}", t5); mutations.clear(); // 6. Commit a transaction to delete Singers 5, insert singers 6. @@ -360,7 +360,7 @@ private static com.google.cloud.Timestamp writeTransactionsToDatabase() { 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()); + LOG.debug("The sixth transaction committed with timestamp: {}", t6); mutations.clear(); // 7. Delete remaining rows from database. @@ -368,7 +368,7 @@ private static com.google.cloud.Timestamp writeTransactionsToDatabase() { 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()); + LOG.debug("The seventh transaction committed with timestamp: {}", t7); return t7; } 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 4ff4f9a81e73..77a680d15779 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 @@ -88,7 +88,7 @@ public static void setup() throws InterruptedException, ExecutionException, Time @Test public void testTransactionBoundaries() { - LOG.info("Test pipeline: " + pipeline.toString()); + LOG.info("Test pipeline: {}", pipeline); final SpannerConfig spannerConfig = SpannerConfig.create() .withProjectId(projectId) @@ -323,14 +323,14 @@ private Timestamp writeTransactionsToDatabase() { 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()); + LOG.debug("The first transaction committed with timestamp: {}", t1); 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()); + LOG.debug("The second transaction committed with timestamp: {}", t2); mutations.clear(); // 3. Commit a transaction to insert Singer 4 and Singer 5 and Singer 6 into the table. @@ -338,21 +338,21 @@ private Timestamp writeTransactionsToDatabase() { 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()); + LOG.debug("The third transaction committed with timestamp: {}", t3); 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()); + LOG.debug("The fourth transaction committed with timestamp: {}", t4); 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()); + LOG.debug("The fifth transaction committed with timestamp: {}", t5); mutations.clear(); // 6. Commit a transaction to delete Singers 3, 4, 5. @@ -361,7 +361,7 @@ private Timestamp writeTransactionsToDatabase() { mutations.add(deleteRecordMutation(5)); Timestamp t6 = databaseClient.write(mutations); mutations.clear(); - LOG.debug("The sixth transaction committed with timestamp: " + t6.toString()); + LOG.debug("The sixth transaction committed with timestamp: {}", t6); // 7. Commit a transaction to delete Singers 0, 2, 6, 7. mutations.add(deleteRecordMutation(0)); @@ -369,7 +369,7 @@ private Timestamp writeTransactionsToDatabase() { mutations.add(deleteRecordMutation(6)); mutations.add(deleteRecordMutation(7)); Timestamp t7 = databaseClient.write(mutations); - LOG.debug("The seventh transaction committed with timestamp: " + t7.toString()); + LOG.debug("The seventh transaction committed with timestamp: {}", t7); return t7; } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/util/TestJsonMapper.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/util/TestJsonMapper.java index fefc1b639d1b..e033c0a751e2 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/util/TestJsonMapper.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/util/TestJsonMapper.java @@ -57,7 +57,7 @@ public class TestJsonMapper { .omittingInsignificantWhitespace() .print(value); } catch (InvalidProtocolBufferException exc) { - LOG.info("Failed to convert record to JSON: " + record.toString()); + LOG.info("Failed to convert record to JSON: {}", record); } return null; } From 0c04fbadd55583ebaf1f9f95032c4d2e2b32fe66 Mon Sep 17 00:00:00 2001 From: Radek Stankiewicz Date: Wed, 4 Mar 2026 20:01:48 +0100 Subject: [PATCH 2/4] enable Slf4jFormatShouldBeConst --- .../beam/it/common/utils/IORedirectUtil.java | 2 +- .../it/jdbc/AbstractJDBCResourceManager.java | 2 +- .../runners/core/metrics/MetricsLogger.java | 2 +- .../core/metrics/SimpleExecutionState.java | 2 +- ...ycleManagerRemovingTransformEvaluator.java | 2 +- .../FlinkStreamingPipelineTranslator.java | 2 +- .../FnApiControlClientPoolService.java | 2 +- .../fnexecution/logging/Slf4jLogWriter.java | 1 + .../status/WorkerStatusClient.java | 6 +-- .../control/RemoteExecutionTest.java | 4 +- .../jobsubmission/InMemoryJobService.java | 26 ++++--------- .../runners/jobsubmission/JobInvocation.java | 3 +- .../beam/runners/jet/JetPipelineResult.java | 2 +- .../apache/beam/runners/jet/JetRunner.java | 2 +- .../runners/samza/SamzaPipelineResult.java | 2 +- .../runners/samza/SamzaPipelineRunner.java | 2 +- .../beam/runners/samza/SamzaRunner.java | 2 +- .../samza/adapter/BoundedSourceSystem.java | 2 +- .../samza/adapter/UnboundedSourceSystem.java | 2 +- .../samza/container/ContainerCfgLoader.java | 4 +- .../samza/runtime/AsyncDoFnRunner.java | 2 +- .../samza/translation/ConfigBuilder.java | 6 +-- .../samza/translation/TranslationContext.java | 7 ++-- .../runners/spark/SparkPipelineRunner.java | 2 +- .../translation/SparkContextFactory.java | 2 +- .../spark/util/SideInputBroadcast.java | 2 +- .../spark/metrics/SparkMetricsPusherTest.java | 2 +- .../providers/LoggingTransformProvider.java | 6 +-- .../apache/beam/sdk/util/WindowTracing.java | 3 +- .../construction/PTransformTranslation.java | 7 ++-- .../util/construction/TransformUpgrader.java | 4 +- .../beam/sdk/testing/ExpectedLogsTest.java | 22 +++++------ .../sdk/transforms/join/CoGbkResultTest.java | 2 +- .../expansion/service/ExpansionService.java | 10 ++--- .../avro/io/AvroGeneratedUserFactory.java | 2 +- .../avro/schemas/TestAvroFactory.java | 2 +- .../extensions/gcp/options/GcpOptions.java | 19 ++++------ .../gcp/util/RetryHttpRequestInitializer.java | 8 ++-- .../ordered/GlobalSequencesProcessorDoFn.java | 14 +++---- .../sdk/extensions/ordered/ProcessorDoFn.java | 11 +++--- .../ordered/SequencePerKeyProcessorDoFn.java | 4 +- .../combiner/DefaultSequenceCombiner.java | 2 +- .../sdk/extensions/python/PythonService.java | 6 +-- .../fn/harness/ExternalWorkerService.java | 2 +- .../control/ExecutionStateSampler.java | 37 +++++++++---------- .../harness/control/ProcessBundleHandler.java | 2 +- .../beam/fn/harness/debug/DataSampler.java | 3 +- .../aws2/common/AsyncBatchWriteHandler.java | 2 +- .../io/aws2/kinesis/EFOShardSubscriber.java | 2 +- .../apache/beam/sdk/io/aws2/sns/SnsIO.java | 2 +- .../apache/beam/sdk/io/amqp/AmqpIOTest.java | 2 +- .../sdk/io/elasticsearch/ElasticsearchIO.java | 2 +- .../sdk/io/hbase/HBaseSharedConnection.java | 2 +- .../org/apache/beam/sdk/io/jdbc/JdbcUtil.java | 4 +- .../org/apache/beam/sdk/io/jms/JmsIO.java | 2 +- .../sdk/io/kafka/KafkaUnboundedReader.java | 19 +++++----- .../apache/beam/sdk/io/kafka/KafkaWriter.java | 2 +- .../apache/beam/sdk/io/kafka/KafkaIOIT.java | 2 +- .../apache/beam/sdk/io/kudu/KuduIOTest.java | 4 ++ .../apache/beam/sdk/io/mongodb/MongoDbIO.java | 2 +- .../org/apache/beam/sdk/io/neo4j/Neo4jIO.java | 19 ++++------ .../beam/sdk/io/snowflake/SnowflakeIO.java | 6 +-- .../sdk/io/solace/broker/SessionService.java | 14 +++---- .../io/solace/it/SolaceContainerManager.java | 2 +- .../org/apache/beam/sdk/io/solr/SolrIO.java | 2 +- .../RabbitMqReceiverWithOffset.java | 4 +- .../io/sparkreceiver/SparkReceiverIOIT.java | 2 +- .../apache/beam/sdk/io/thrift/ThriftIO.java | 6 +-- .../openai/OpenAIModelHandlerIT.java | 2 +- .../testing/watermarks/WatermarkLatency.java | 2 +- .../launcher/TransformServiceLauncher.java | 12 +++--- 71 files changed, 178 insertions(+), 201 deletions(-) diff --git a/it/common/src/main/java/org/apache/beam/it/common/utils/IORedirectUtil.java b/it/common/src/main/java/org/apache/beam/it/common/utils/IORedirectUtil.java index 9ed6471525d7..0ed2934a2d48 100644 --- a/it/common/src/main/java/org/apache/beam/it/common/utils/IORedirectUtil.java +++ b/it/common/src/main/java/org/apache/beam/it/common/utils/IORedirectUtil.java @@ -41,7 +41,7 @@ public static void redirectLinesLog(InputStream inputStream, Logger log) { String line; while ((line = bis.readLine()) != null) { - log.info(line); + log.info("{}", line); } } catch (Exception e) { log.error("Error redirecting stream", e); diff --git a/it/jdbc/src/main/java/org/apache/beam/it/jdbc/AbstractJDBCResourceManager.java b/it/jdbc/src/main/java/org/apache/beam/it/jdbc/AbstractJDBCResourceManager.java index 73c55d001381..03079faf530a 100644 --- a/it/jdbc/src/main/java/org/apache/beam/it/jdbc/AbstractJDBCResourceManager.java +++ b/it/jdbc/src/main/java/org/apache/beam/it/jdbc/AbstractJDBCResourceManager.java @@ -210,7 +210,7 @@ public boolean write(String tableName, List> rows) sql.append(String.join(",", valueList)).append(")"); try { - LOG.info("Running SQL statement: " + sql); + LOG.info("Running SQL statement: {}", sql); stmt.executeUpdate(sql.toString()); } catch (SQLException e) { throw new JDBCResourceManagerException( diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsLogger.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsLogger.java index 75478d52b335..a7214b64c155 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsLogger.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsLogger.java @@ -61,7 +61,7 @@ public void tryLoggingMetrics( long currentTimeMillis = System.currentTimeMillis(); long lastReported = lastReportedMillis.get(); if (currentTimeMillis - lastReported > minimumLoggingFrequencyMillis) { - LOG.info(generateLogMessage(header, allowedMetricUrns, lastReported)); + LOG.info("{}", generateLogMessage(header, allowedMetricUrns, lastReported)); lastReportedMillis.set(currentTimeMillis); } } finally { diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleExecutionState.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleExecutionState.java index 821a7e06c526..14278678a364 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleExecutionState.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleExecutionState.java @@ -151,7 +151,7 @@ public String getLullMessage(Thread trackedThread, Duration millis) { @Override public void reportLull(Thread trackedThread, long millis) { - LOG.warn(getLullMessage(trackedThread, Duration.millis(millis))); + LOG.warn("{}", getLullMessage(trackedThread, Duration.millis(millis))); } @VisibleForTesting diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java index 6508cb2a78b2..6a6c8d8ecf18 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java @@ -84,7 +84,7 @@ private void onException(Exception e, String msg) { if (removalException instanceof InterruptedException) { Thread.currentThread().interrupt(); } - LOG.error(msg, removalException); + LOG.error("{}", msg, removalException); e.addSuppressed(removalException); } } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java index 0607838987f1..113cdbd91290 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java @@ -133,7 +133,7 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { if (translator == null || !applyCanTranslate(transform, node, translator)) { String transformUrn = PTransformTranslation.urnForTransform(transform); - LOG.info(transformUrn); + LOG.info("{}", transformUrn); throw new UnsupportedOperationException( "The transform " + transformUrn + " is currently not supported."); } diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolService.java index e75efc6d325f..541d26118080 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolService.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolService.java @@ -117,7 +117,7 @@ public void getProcessBundleDescriptor( String msg = String.format("ProcessBundleDescriptor with id %s not found", bundleDescriptorId); responseObserver.onError(new StatusException(Status.NOT_FOUND.withDescription(msg))); - LOG.error(msg); + LOG.error("{}", msg); } else { responseObserver.onNext(descriptor); responseObserver.onCompleted(); diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/logging/Slf4jLogWriter.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/logging/Slf4jLogWriter.java index 41a2cee196c0..f2e4a147cd54 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/logging/Slf4jLogWriter.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/logging/Slf4jLogWriter.java @@ -35,6 +35,7 @@ public static LogWriter getDefault() { private Slf4jLogWriter() {} @Override + @SuppressWarnings("Slf4jFormatShouldBeConst") public void log(LogEntry entry) { Logger log; String location = entry.getLogLocation(); diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClient.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClient.java index 210db0cb9f7b..29c58c516878 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClient.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClient.java @@ -141,9 +141,9 @@ public void onNext(WorkerStatusResponse response) { future.complete(response); } else { LOG.warn( - String.format( - "Received response for status with unknown response id %s and status %s", - response.getId(), response.getStatusInfo())); + "Received response for status with unknown response id {} and status {}", + response.getId(), + response.getStatusInfo()); } } diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java index 996c206a7e97..7a10a6ff5a26 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java @@ -376,8 +376,8 @@ public void process(ProcessContext ctxt) { @ProcessElement public void process(ProcessContext ctxt) { org.slf4j.Logger logger = LoggerFactory.getLogger(RemoteExecutionTest.class); - logger.warn("TEST" + ctxt.element()); - logger.error("TEST_EXCEPTION" + ctxt.element(), new Exception()); + logger.warn("TEST{}", ctxt.element()); + logger.error("TEST_EXCEPTION{}", ctxt.element(), new Exception()); } })) .apply("addKeys", WithKeys.of("foo")) diff --git a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/InMemoryJobService.java b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/InMemoryJobService.java index eb57af5ab50c..8413f184115e 100644 --- a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/InMemoryJobService.java +++ b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/InMemoryJobService.java @@ -276,9 +276,7 @@ public void run(RunJobRequest request, StreamObserver responseOb LOG.warn("Encountered Status Exception", e); responseObserver.onError(e); } catch (Exception e) { - String errMessage = - String.format("Encountered Unexpected Exception for Preparation %s", preparationId); - LOG.error(errMessage, e); + LOG.error("Encountered Unexpected Exception for Preparation {}", preparationId, e); responseObserver.onError(Status.INTERNAL.withCause(e).asException()); } } @@ -363,9 +361,7 @@ public void getState(GetJobStateRequest request, StreamObserver r } catch (StatusRuntimeException | StatusException e) { responseObserver.onError(e); } catch (Exception e) { - String errMessage = - String.format("Encountered Unexpected Exception for Invocation %s", invocationId); - LOG.error(errMessage, e); + LOG.error("Encountered Unexpected Exception for Invocation {}", invocationId, e); responseObserver.onError(Status.INTERNAL.withCause(e).asException()); } } @@ -385,9 +381,7 @@ public void getPipeline( } catch (StatusRuntimeException | StatusException e) { responseObserver.onError(e); } catch (Exception e) { - String errMessage = - String.format("Encountered Unexpected Exception for Invocation %s", invocationId); - LOG.error(errMessage, e); + LOG.error("Encountered Unexpected Exception for Invocation {}", invocationId, e); responseObserver.onError(Status.INTERNAL.withCause(e).asException()); } } @@ -406,9 +400,7 @@ public void cancel(CancelJobRequest request, StreamObserver r } catch (StatusRuntimeException | StatusException e) { responseObserver.onError(e); } catch (Exception e) { - String errMessage = - String.format("Encountered Unexpected Exception for Invocation %s", invocationId); - LOG.error(errMessage, e); + LOG.error("Encountered Unexpected Exception for Invocation {}", invocationId, e); responseObserver.onError(Status.INTERNAL.withCause(e).asException()); } } @@ -432,9 +424,7 @@ public void getStateStream( } catch (StatusRuntimeException | StatusException e) { responseObserver.onError(e); } catch (Exception e) { - String errMessage = - String.format("Encountered Unexpected Exception for Invocation %s", invocationId); - LOG.error(errMessage, e); + LOG.error("Encountered Unexpected Exception for Invocation {}", invocationId, e); responseObserver.onError(Status.INTERNAL.withCause(e).asException()); } } @@ -472,9 +462,7 @@ public void getMessageStream( } catch (StatusRuntimeException | StatusException e) { responseObserver.onError(e); } catch (Exception e) { - String errMessage = - String.format("Encountered Unexpected Exception for Invocation %s", invocationId); - LOG.error(errMessage, e); + LOG.error("Encountered Unexpected Exception for Invocation {}", invocationId, e); responseObserver.onError(Status.INTERNAL.withCause(e).asException()); } } @@ -498,7 +486,7 @@ public void getJobMetrics( } catch (StatusRuntimeException | StatusException e) { responseObserver.onError(e); } catch (Exception e) { - LOG.error(String.format("Encountered exception for job invocation %s", invocationId), e); + LOG.error("Encountered exception for job invocation {}", invocationId, e); responseObserver.onError(Status.INTERNAL.withCause(e).asException()); } LOG.info("Finished getting job metrics for {}", invocationId); diff --git a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobInvocation.java b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobInvocation.java index 8032549ab606..891aacaa5715 100644 --- a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobInvocation.java +++ b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobInvocation.java @@ -138,8 +138,7 @@ public void onFailure(@Nonnull Throwable throwable) { setState(JobState.Enum.CANCELLED); return; } - String message = String.format("Error during job invocation %s.", getId()); - LOG.error(message, throwable); + LOG.error("Error during job invocation {}.", getId(), throwable); sendMessage( JobMessage.newBuilder() .setMessageText(getStackTraceAsString(throwable)) diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetPipelineResult.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetPipelineResult.java index dd780c985b36..dccd52303987 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetPipelineResult.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetPipelineResult.java @@ -87,7 +87,7 @@ public State getState() { case SUSPENDED_EXPORTING_SNAPSHOT: return State.STOPPED; default: - LOG.warn("Unhandled " + JobStatus.class.getSimpleName() + ": " + status.name() + "!"); + LOG.warn("Unhandled {}: {}!", JobStatus.class.getSimpleName(), status.name()); return State.UNKNOWN; } } diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetRunner.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetRunner.java index c27f9a0f0611..d53de5e11a3d 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetRunner.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetRunner.java @@ -143,7 +143,7 @@ private void startClusterIfNeeded(JetPipelineOptions options) { for (int i = 0; i < noOfLocalMembers; i++) { jetInstances.add(Jet.newJetInstance()); } - LOG.info("Started " + jetInstances.size() + " Jet cluster members"); + LOG.info("Started {} Jet cluster members", jetInstances.size()); } } diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineResult.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineResult.java index ffcc949e4611..e84cf086edc9 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineResult.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineResult.java @@ -119,7 +119,7 @@ private StateInfo getStateInfo() { case SuccessfulFinish: return new StateInfo(State.DONE); case UnsuccessfulFinish: - LOG.error(status.getThrowable().getMessage(), status.getThrowable()); + LOG.error("Pipeline execution failed", status.getThrowable()); return new StateInfo( State.FAILED, new Pipeline.PipelineExecutionException(getUserCodeException(status.getThrowable()))); diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineRunner.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineRunner.java index fb974b79ab1b..897b78cf9e47 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineRunner.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineRunner.java @@ -62,7 +62,7 @@ public PortablePipelineResult run(final RunnerApi.Pipeline pipeline, JobInfo job : GreedyPipelineFuser.fuse(trimmedPipeline).toPipeline(); LOG.info("Portable pipeline to run:"); - LOG.info(PipelineDotRenderer.toDotString(fusedPipeline)); + LOG.info("{}", PipelineDotRenderer.toDotString(fusedPipeline)); // the pipeline option coming from sdk will set the sdk specific runner which will break // serialization // so we need to reset the runner here to a valid Java runner diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java index bc1ada6941b9..eb16faa41ac0 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java @@ -184,7 +184,7 @@ private Map getMetricsReporters() { final MetricsReporter reporter = options.getMetricsReporters().get(i); reporters.put(name, (MetricsReporterFactory) (nm, processorId, config) -> reporter); - LOG.info(name + ": " + reporter.getClass().getName()); + LOG.info("{}: {}", name, reporter.getClass().getName()); } return reporters; } else { diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystem.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystem.java index 260957e05ecc..92c9eea4293a 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystem.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystem.java @@ -422,7 +422,7 @@ public SystemConsumer getConsumer(String systemName, Config config, MetricsRegis @Override public SystemProducer getProducer(String systemName, Config config, MetricsRegistry registry) { - LOG.info("System " + systemName + " does not have producer."); + LOG.info("System {} does not have producer.", systemName); return null; } diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystem.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystem.java index 41214d7800e2..ffab2ff59ce5 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystem.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystem.java @@ -504,7 +504,7 @@ public SystemConsumer getConsumer(String systemName, Config config, MetricsRegis @Override public SystemProducer getProducer(String systemName, Config config, MetricsRegistry registry) { - LOG.info("System " + systemName + " does not have producer."); + LOG.info("System {} does not have producer.", systemName); return null; } diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/container/ContainerCfgLoader.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/container/ContainerCfgLoader.java index a79c04f24b0a..9437aea56561 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/container/ContainerCfgLoader.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/container/ContainerCfgLoader.java @@ -46,9 +46,9 @@ public Config getConfig() { synchronized (LOCK) { if (jobModel == null) { final String containerId = System.getenv(ShellCommandConfig.ENV_CONTAINER_ID); - LOG.info(String.format("Got container ID: %s", containerId)); + LOG.info("Got container ID: {}", containerId); final String coordinatorUrl = System.getenv(ShellCommandConfig.ENV_COORDINATOR_URL); - LOG.info(String.format("Got coordinator URL: %s", coordinatorUrl)); + LOG.info("Got coordinator URL: {}", coordinatorUrl); final int delay = RANDOM.nextInt(SamzaContainer.DEFAULT_READ_JOBMODEL_DELAY_MS()) + 1; jobModel = SamzaContainer.readJobModel(coordinatorUrl, delay); } diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/AsyncDoFnRunner.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/AsyncDoFnRunner.java index 2485ac2d5522..e1bc9251a304 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/AsyncDoFnRunner.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/AsyncDoFnRunner.java @@ -69,7 +69,7 @@ public static AsyncDoFnRunner create( boolean isStateful, SamzaPipelineOptions options) { - LOG.info("Run DoFn with " + AsyncDoFnRunner.class.getName()); + LOG.info("Run DoFn with {}", AsyncDoFnRunner.class.getName()); return new AsyncDoFnRunner<>(runner, emitter, futureCollector, isStateful, options); } diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ConfigBuilder.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ConfigBuilder.java index d3de60de6b4b..79b5bac238e1 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ConfigBuilder.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ConfigBuilder.java @@ -158,13 +158,13 @@ private static Map createUserConfig(SamzaPipelineOptions options // If user provides a config file, use it as base configs. if (StringUtils.isNoneEmpty(configFilePath)) { - LOG.info("configFilePath: " + configFilePath); + LOG.info("configFilePath: {}", configFilePath); final Config properties = new MapConfig(Collections.singletonMap("path", configFilePath)); final ConfigLoaderFactory configLoaderFactory = options.getConfigLoaderFactory().getDeclaredConstructor().newInstance(); - LOG.info("configLoaderFactory: " + configLoaderFactory.getClass().getName()); + LOG.info("configLoaderFactory: {}", configLoaderFactory.getClass().getName()); // Config file must exist for default properties config // TODO: add check to all non-empty files once we don't need to @@ -303,7 +303,7 @@ private static Map createSystemConfig( "org.apache.samza.storage.kv.inmemory.InMemoryKeyValueStorageEngineFactory"); } - LOG.info("Execution environment is " + options.getSamzaExecutionEnvironment()); + LOG.info("Execution environment is {}", options.getSamzaExecutionEnvironment()); switch (options.getSamzaExecutionEnvironment()) { case YARN: configBuilder.putAll(yarnRunConfig()); diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java index c5e984fbde07..4da5b8708f18 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java @@ -139,9 +139,10 @@ protected void registerInputMessageStreams( @SuppressWarnings("unchecked") MessageStream> messageStream = registeredInputStreams.get(streamId); LOG.info( - String.format( - "Stream id %s has already been mapped to %s stream. Mapping %s to the same message stream.", - streamId, messageStream, key)); + "Stream id {} has already been mapped to {} stream. Mapping {} to the same message stream.", + streamId, + messageStream, + key); streamsToMerge.add(messageStream); } else { final MessageStream> typedStream = diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java index 1e024a102601..91a94896b89b 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java @@ -138,7 +138,7 @@ public PortablePipelineResult run(RunnerApi.Pipeline pipeline, JobInfo jobInfo) // Register user-defined listeners. for (JavaStreamingListener listener : pipelineOptions.as(SparkContextOptions.class).getListeners()) { - LOG.info("Registered listener {}." + listener.getClass().getSimpleName()); + LOG.info("Registered listener {}.", listener.getClass().getSimpleName()); jssc.addStreamingListener(new JavaStreamingListenerWrapper(listener)); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java index 88db3b3f158b..e8cf6204eaf9 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java @@ -90,7 +90,7 @@ public static synchronized JavaSparkContext getSparkContext(SparkPipelineOptions throw new IllegalStateException( "No Spark context was provided. Use SparkContextFactor.setProvidedSparkContext to do so."); } else if (jsc.sc().isStopped()) { - LOG.error("The provided Spark context " + jsc + " was already stopped."); + LOG.error("The provided Spark context {} was already stopped.", jsc); throw new IllegalStateException("The provided Spark context was already stopped"); } LOG.info("Using a provided Spark Context"); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SideInputBroadcast.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SideInputBroadcast.java index a0e0dcaa29bc..4268191844e3 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SideInputBroadcast.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SideInputBroadcast.java @@ -82,7 +82,7 @@ private T deserialize() { val = coder.decode(new ByteArrayInputStream(bcast.value())); } catch (IOException ioe) { // this should not ever happen, log it if it does. - LOG.warn(ioe.getMessage()); + LOG.warn("Error reading broadcast data", ioe); val = null; } return val; diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/metrics/SparkMetricsPusherTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/metrics/SparkMetricsPusherTest.java index dbd569d89d15..906a4fcb6878 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/metrics/SparkMetricsPusherTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/metrics/SparkMetricsPusherTest.java @@ -109,7 +109,7 @@ public void processElement(ProcessContext context) { counter.inc(); context.output(context.element()); } catch (Exception e) { - LOG.warn("Exception caught" + e); + LOG.warn("Exception caught", e); } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/LoggingTransformProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/LoggingTransformProvider.java index 2908171f5c02..13f7de64700e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/LoggingTransformProvider.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/LoggingTransformProvider.java @@ -171,14 +171,14 @@ public void processElement(@Element Row row, OutputReceiver out) { // https://stackoverflow.com/questions/2621701/setting-log-level-of-message-at-runtime-in-slf4j switch (logLevel) { case DEBUG: - LOG.debug(msg); + LOG.debug("{}", msg); break; case INFO: - LOG.info(msg); + LOG.info("{}", msg); break; case ERROR: default: - LOG.error(msg); + LOG.error("{}", msg); } out.output(row); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowTracing.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowTracing.java index ab8a94c02120..539de5f4d0fd 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowTracing.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowTracing.java @@ -26,11 +26,12 @@ public final class WindowTracing { private static final Logger LOG = LoggerFactory.getLogger(WindowTracing.class); + @SuppressWarnings("Slf4jFormatShouldBeConst") public static void debug(final String format, Object... args) { LOG.debug(format, args); } - @SuppressWarnings("unused") + @SuppressWarnings({"unused", "Slf4jFormatShouldBeConst"}) public static void trace(final String format, Object... args) { LOG.trace(format, args); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PTransformTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PTransformTranslation.java index 3e38aad1ad4b..c7b42f59036d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PTransformTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PTransformTranslation.java @@ -554,10 +554,9 @@ public RunnerApi.PTransform translate( // Optional toConfigRow() has not been implemented. We can just ignore. } catch (Exception e) { LOG.warn( - "Could not attach the config row for transform " - + appliedPTransform.getTransform().getName() - + ": " - + e); + "Could not attach the config row for transform {}: {}", + appliedPTransform.getTransform().getName(), + e.toString()); // Ignoring the error and continuing with the translation since attaching config rows is // optional. } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TransformUpgrader.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TransformUpgrader.java index 4268c6c70671..253f5ce52756 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TransformUpgrader.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TransformUpgrader.java @@ -477,8 +477,8 @@ public static Object fromByteArray(byte[] bytes) throws InvalidClassException { } catch (InvalidClassException e) { LOG.info( "An object cannot be re-generated from the provided byte array. Caller may use the " - + "default value for the parameter when upgrading. Underlying error: " - + e); + + "default value for the parameter when upgrading. Underlying error: {}", + e.toString()); throw e; } catch (Exception e) { throw new RuntimeException(e); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogsTest.java index 896ca6958ec0..4919f799c0d4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogsTest.java @@ -49,36 +49,36 @@ public class ExpectedLogsTest { @Test public void testWhenNoExpectations() throws Throwable { - LOG.error(generateRandomString()); + LOG.error("{}", generateRandomString()); } @Test public void testVerifyWhenMatchedFully() throws Throwable { String expected = generateRandomString(); - LOG.error(expected); + LOG.error("{}", expected); expectedLogs.verifyError(expected); } @Test public void testVerifyWhenMatchedPartially() throws Throwable { String expected = generateRandomString(); - LOG.error("Extra stuff around expected " + expected + " blah"); + LOG.error("Extra stuff around expected {} blah", expected); expectedLogs.verifyError(expected); } @Test public void testVerifyWhenMatchedWithExceptionBeingLogged() throws Throwable { String expected = generateRandomString(); - LOG.error(expected, new IOException("Fake Exception")); + LOG.error("{}", expected, new IOException("Fake Exception")); expectedLogs.verifyError(expected); } @Test public void testVerifyLogRecords() throws Throwable { String expected = generateRandomString(); - LOG.error(expected); - LOG.error(expected); + LOG.error("{}", expected); + LOG.error("{}", expected); expectedLogs.verifyLogRecords( new TypeSafeMatcher>() { @Override @@ -108,21 +108,21 @@ public void testVerifyWhenNotMatched() throws Throwable { public void testVerifyNotLoggedWhenMatchedFully() throws Throwable { String expected = generateRandomString(); - LOG.error(expected); + LOG.error("{}", expected); expectedLogs.verifyNotLogged(expected); } @Test(expected = AssertionError.class) public void testVerifyNotLoggedWhenMatchedPartially() throws Throwable { String expected = generateRandomString(); - LOG.error("Extra stuff around expected " + expected + " blah"); + LOG.error("Extra stuff around expected {} blah", expected); expectedLogs.verifyNotLogged(expected); } @Test(expected = AssertionError.class) public void testVerifyNotLoggedWhenMatchedWithException() throws Throwable { String expected = generateRandomString(); - LOG.error(expected, new IOException("Fake Exception")); + LOG.error("{}", expected, new IOException("Fake Exception")); expectedLogs.verifyNotLogged(expected); } @@ -135,7 +135,7 @@ public void testVerifyNotLoggedWhenNotMatched() throws Throwable { @Test public void testLogCaptureOccursAtLowestLogLevel() throws Throwable { String expected = generateRandomString(); - LOG.trace(expected); + LOG.trace("{}", expected); expectedLogs.verifyTrace(expected); } @@ -158,7 +158,7 @@ public void testThreadSafetyOfLogSaver() throws Throwable { 1, scheduledLogTime - TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS))); - LOG.trace(expected); + LOG.trace("{}", expected); return null; }); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultTest.java index b30658fab081..f9fd4da32288 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultTest.java @@ -207,7 +207,7 @@ public void runCrazyIteration(int numIterations, int... tagSizes) { for (int trial = 0; trial < 10; trial++) { // Populate this with a constant to reproduce failures. int seed = (int) (Integer.MAX_VALUE * Math.random()); - LOG.info("Running " + Arrays.toString(tagSizes) + " with seed " + seed); + LOG.info("Running {} with seed {}", Arrays.toString(tagSizes), seed); Random random = new Random(seed); List tags = new ArrayList<>(); for (int tagNum = 0; tagNum < tagSizes.length; tagNum++) { diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java index c3c3ccfd3266..d0aa0f6c0e9b 100644 --- a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java +++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java @@ -172,9 +172,8 @@ public Map knownTransforms() { urn = translator.getUrn(); if (urn == null) { LOG.debug( - "Could not load the TransformPayloadTranslator " - + translator - + " to the Expansion Service since it did not produce a unique URN."); + "Could not load the TransformPayloadTranslator {} to the Expansion Service since it did not produce a unique URN.", + translator); continue; } else if (urn.equals(BeamUrns.getUrn(SCHEMA_TRANSFORM)) && translator instanceof SchemaTransformPayloadTranslator) { @@ -182,9 +181,8 @@ public Map knownTransforms() { } } catch (Exception e) { LOG.info( - "Could not load the TransformPayloadTranslator " - + translator - + " to the Expansion Service.", + "Could not load the TransformPayloadTranslator {} to the Expansion Service.", + translator, e); continue; } diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUserFactory.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUserFactory.java index 066b65c5d5e6..65552b705dff 100644 --- a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUserFactory.java +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUserFactory.java @@ -41,7 +41,7 @@ public static AvroGeneratedUser newInstance( return (AvroGeneratedUser) constructor.newInstance(name, favoriteNumber, favoriteColor); } catch (ReflectiveOperationException e) { - LOG.error(String.format("Fail to create a AvroGeneratedUser instance: %s", e.getMessage())); + LOG.error("Fail to create a AvroGeneratedUser instance", e); return new AvroGeneratedUser(); // return an empty instance to fail the tests } } diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/TestAvroFactory.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/TestAvroFactory.java index 0414be587bcc..4d52a35543e0 100644 --- a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/TestAvroFactory.java +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/TestAvroFactory.java @@ -118,7 +118,7 @@ public static TestAvro newInstance( map); } } catch (ReflectiveOperationException e) { - LOG.error(String.format("Fail to create a TestAvro instance: %s", e.getMessage())); + LOG.error("Fail to create a TestAvro instance", e); return new TestAvro(); // return an empty instance to fail the tests } } diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptions.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptions.java index c370606de37f..4f36195fabac 100644 --- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptions.java +++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptions.java @@ -395,14 +395,13 @@ class GcpTempLocationFactory implements DefaultValueFactory { if (isSoftDeletePolicyEnabled(options, tempLocation)) { LOG.warn( - String.format( - "The bucket of gcpTempLocation %s has soft delete policy enabled." - + " Dataflow jobs use Cloud Storage to store temporary files during pipeline" - + " execution. To avoid being billed for unnecessary storage costs, turn off the soft" - + " delete feature on buckets that your Dataflow jobs use for temporary storage." - + " For more information, see" - + " https://cloud.google.com/storage/docs/use-soft-delete#remove-soft-delete-policy.", - tempLocation)); + "The bucket of gcpTempLocation {} has soft delete policy enabled." + + " Dataflow jobs use Cloud Storage to store temporary files during pipeline" + + " execution. To avoid being billed for unnecessary storage costs, turn off the soft" + + " delete feature on buckets that your Dataflow jobs use for temporary storage." + + " For more information, see" + + " https://cloud.google.com/storage/docs/use-soft-delete#remove-soft-delete-policy.", + tempLocation); } return tempLocation; @@ -420,9 +419,7 @@ static boolean isSoftDeletePolicyEnabled(PipelineOptions options, String tempLoc return true; } } catch (Exception e) { - LOG.warn( - String.format( - "Failed to access bucket for gcpTempLocation: %s.%nCaused by %s", tempLocation, e)); + LOG.warn("Failed to access bucket for gcpTempLocation: {}", tempLocation, e); } return false; } diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/RetryHttpRequestInitializer.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/RetryHttpRequestInitializer.java index d7161564d739..e3d4eb6ee363 100644 --- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/RetryHttpRequestInitializer.java +++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/RetryHttpRequestInitializer.java @@ -104,14 +104,12 @@ public boolean handleIOException(HttpRequest request, boolean supportsRetry) ioExceptionRetries += 1; LOG.debug("Request failed with IOException, will retry: {}", request.getUrl()); } else { - String message = + LOG.warn( "Request failed with IOException, " + "performed {} retries due to IOExceptions, " + "performed {} retries due to unsuccessful status codes, " + "HTTP framework says request {} be retried, " - + "(caller responsible for retrying): {}"; - LOG.warn( - message, + + "(caller responsible for retrying): {}", ioExceptionRetries, unsuccessfulResponseRetries, supportsRetry ? "can" : "cannot", @@ -138,7 +136,7 @@ && retryOnStatusCode(response.getStatusCode()) request.getUrl()); } else { - String message = + final String message = "Request failed with code {}, " + "performed {} retries due to IOExceptions, " + "performed {} retries due to unsuccessful status codes, " diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java index 4c4989966076..3c6c72eb486b 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/GlobalSequencesProcessorDoFn.java @@ -146,7 +146,7 @@ public void processElement( long sequence = eventAndSequence.getValue().getKey(); if (LOG.isTraceEnabled()) { - LOG.trace(key + ": " + sequence + " lastRange: " + lastContiguousRange); + LOG.trace("{}: {} lastRange: {}", key, sequence, lastContiguousRange); } ProcessingState processingState = processingStateProxy.read(); @@ -214,12 +214,10 @@ private void setBatchEmissionTimerIfNeeded( if (LOG.isTraceEnabled()) { LOG.trace( - "Setting batch emission timer to: " - + timerTime - + ", max time of the range: " - + lastCompleteGlobalSequence.getTimestamp() - + ", element time: " - + elementTimestamp); + "Setting batch emission timer to: {}, max time of the range: {}, element time: {}", + timerTime, + lastCompleteGlobalSequence.getTimestamp(), + elementTimestamp); } batchEmissionTimer.set(timerTime); @@ -238,7 +236,7 @@ public void onBatchEmission( MultiOutputReceiver outputReceiver) { if (LOG.isTraceEnabled()) { - LOG.trace("Running batch processing for: " + key); + LOG.trace("Running batch processing for: {}", key); } // At this point everything in the buffered state is ready to be processed up to the latest diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java index 9713428be9bb..3e97f85cd596 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/ProcessorDoFn.java @@ -245,7 +245,7 @@ protected void emitProcessingStatus( MultiOutputReceiver outputReceiver, Instant statusTimestamp) { if (LOG.isTraceEnabled()) { - LOG.trace("Emitting status for: " + processingState.getKey() + ", " + processingState); + LOG.trace("Emitting status for: {}, {}", processingState.getKey(), processingState); } outputReceiver .get(statusTupleTag) @@ -274,9 +274,8 @@ protected boolean reachedMaxResultCountForBundle( if (exceeded) { if (LOG.isTraceEnabled()) { LOG.trace( - "Setting the timer to output next batch of events for key '" - + processingState.getKey() - + "'"); + "Setting the timer to output next batch of events for key '{}'", + processingState.getKey()); } // See GroupIntoBatches for examples on how to hold the timestamp. // TODO: test that on draining the pipeline all the results are still produced correctly. @@ -388,12 +387,12 @@ StateT processBufferedEventRange( try { if (state == null) { if (LOG.isTraceEnabled()) { - LOG.trace("Creating a new state: " + processingState.getKey() + " " + bufferedEvent); + LOG.trace("Creating a new state: {} {}", processingState.getKey(), bufferedEvent); } state = eventExaminer.createStateOnInitialEvent(bufferedEvent); } else { if (LOG.isTraceEnabled()) { - LOG.trace("Mutating " + processingState.getKey() + " " + bufferedEvent); + LOG.trace("Mutating {} {}", processingState.getKey(), bufferedEvent); } state.mutate(bufferedEvent); } diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java index 878a0664ac87..486622d2069a 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/SequencePerKeyProcessorDoFn.java @@ -189,7 +189,7 @@ public void processElement( private boolean checkIfProcessingIsCompleted(ProcessingState processingState) { boolean result = processingState.isProcessingCompleted(); if (result && LOG.isTraceEnabled()) { - LOG.trace("Processing for key '" + processingState.getKey() + "' is completed."); + LOG.trace("Processing for key '{}' is completed.", processingState.getKey()); } return result; } @@ -256,7 +256,7 @@ public void onBatchEmission( return; } - LOG.debug("Starting to process batch for key '" + processingState.getKey() + "'"); + LOG.debug("Starting to process batch for key '{}'", processingState.getKey()); this.numberOfResultsBeforeBundleStart = processingState.getResultCount(); diff --git a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java index 8e3a517c7e86..bfe3aedad00a 100644 --- a/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java +++ b/sdks/java/extensions/ordered/src/main/java/org/apache/beam/sdk/extensions/ordered/combiner/DefaultSequenceCombiner.java @@ -96,7 +96,7 @@ public SequenceRangeAccumulator mergeAccumulators( public ContiguousSequenceRange extractOutput(SequenceRangeAccumulator accum) { ContiguousSequenceRange result = accum.largestContinuousRange(); if (LOG.isDebugEnabled()) { - LOG.debug("Returning completed sequence range: " + result); + LOG.debug("Returning completed sequence range: {}", result); } return result; } diff --git a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonService.java b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonService.java index ab4d02ec838d..8658cd42cda6 100644 --- a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonService.java +++ b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonService.java @@ -101,7 +101,7 @@ public AutoCloseable start() throws IOException, InterruptedException { if (!extraPackages.isEmpty()) { bootstrapCommand.add("--extra_packages=" + String.join(";", extraPackages)); } - LOG.info("Running bootstrap command " + bootstrapCommand); + LOG.info("Running bootstrap command {}", bootstrapCommand); Process bootstrap = new ProcessBuilder(bootstrapCommand).redirectError(ProcessBuilder.Redirect.INHERIT).start(); bootstrap.getOutputStream().close(); @@ -111,7 +111,7 @@ public AutoCloseable start() throws IOException, InterruptedException { String lastLine = reader.readLine(); String lastNonEmptyLine = lastLine; while (lastLine != null) { - LOG.info(lastLine); + LOG.info("{}", lastLine); if (lastLine.length() > 0) { lastNonEmptyLine = lastLine; } @@ -129,7 +129,7 @@ public AutoCloseable start() throws IOException, InterruptedException { command.add("-m"); command.add(module); command.addAll(args); - LOG.info("Starting python service with arguments " + command); + LOG.info("Starting python service with arguments {}", command); Process p = new ProcessBuilder(command) .redirectError(ProcessBuilder.Redirect.INHERIT) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ExternalWorkerService.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ExternalWorkerService.java index 501d2c609cba..466d8ef3193b 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ExternalWorkerService.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ExternalWorkerService.java @@ -106,7 +106,7 @@ public void startWorker( request.getWorkerId(), options, capabilities, logEndpoint, ctrlEndpoint, null); LOG.info("Successfully started worker {}.", request.getWorkerId()); } catch (Exception exn) { - LOG.error(String.format("Failed to start worker %s.", request.getWorkerId()), exn); + LOG.error("Failed to start worker {}.", request.getWorkerId(), exn); } }); th.setName("SDK-worker-" + request.getWorkerId()); diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ExecutionStateSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ExecutionStateSampler.java index edc5e5255146..8261839c58cc 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ExecutionStateSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ExecutionStateSampler.java @@ -451,30 +451,27 @@ private Optional takeSample(long currentTimeMillis, long millisSinceLast Thread thread = trackedThread.get(); if (thread == null) { LOG.warn( - String.format( - "Operation ongoing in bundle %s for at least %s without outputting " - + "or completing (stack trace unable to be generated).", - processBundleId.get(), - DURATION_FORMATTER.print(Duration.millis(lullTimeMs).toPeriod()))); + "Operation ongoing in bundle {} for at least {} without outputting " + + "or completing (stack trace unable to be generated).", + processBundleId.get(), + DURATION_FORMATTER.print(Duration.millis(lullTimeMs).toPeriod())); } else if (currentExecutionState == null) { LOG.warn( - String.format( - "Operation ongoing in bundle %s for at least %s without outputting " - + "or completing:%n at %s", - processBundleId.get(), - DURATION_FORMATTER.print(Duration.millis(lullTimeMs).toPeriod()), - Joiner.on("\n at ").join(thread.getStackTrace()))); + "Operation ongoing in bundle {} for at least {} without outputting " + + "or completing:\n at {}", + processBundleId.get(), + DURATION_FORMATTER.print(Duration.millis(lullTimeMs).toPeriod()), + Joiner.on("\n at ").join(thread.getStackTrace())); } else { LOG.warn( - String.format( - "Operation ongoing in bundle %s for PTransform{id=%s, name=%s, state=%s} " - + "for at least %s without outputting or completing:%n at %s", - processBundleId.get(), - currentExecutionState.ptransformId, - currentExecutionState.ptransformUniqueName, - currentExecutionState.stateName, - DURATION_FORMATTER.print(Duration.millis(lullTimeMs).toPeriod()), - Joiner.on("\n at ").join(thread.getStackTrace()))); + "Operation ongoing in bundle {} for PTransform{{id={}, name={}, state={}}} " + + "for at least {} without outputting or completing:\n at {}", + processBundleId.get(), + currentExecutionState.ptransformId, + currentExecutionState.ptransformUniqueName, + currentExecutionState.stateName, + DURATION_FORMATTER.print(Duration.millis(lullTimeMs).toPeriod()), + Joiner.on("\n at ").join(thread.getStackTrace())); } } } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java index f773e7945797..5a57b137bf6b 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java @@ -1301,7 +1301,7 @@ public void addRunnerForPTransform(Context context) { String.format( "No factory registered for %s, known factories %s", context.getPTransform().getSpec().getUrn(), knownUrns); - LOG.error(message); + LOG.error("{}", message); throw new IllegalStateException(message); } } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java index 29011b82a4dc..4d42009c6773 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java @@ -173,7 +173,8 @@ public synchronized BeamFnApi.InstructionResponse.Builder handleDataSampleReques pcollectionId, ElementList.newBuilder().addAllElements(outputSampler.samples()).build()); } catch (IOException e) { - LOG.warn("Could not encode elements from \"" + pcollectionId + "\" to bytes: " + e); + LOG.warn( + "Could not encode elements from \"{}\" to bytes: {}", pcollectionId, e.toString()); } }); diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/common/AsyncBatchWriteHandler.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/common/AsyncBatchWriteHandler.java index 95a65fe11658..0faa932b30ac 100644 --- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/common/AsyncBatchWriteHandler.java +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/common/AsyncBatchWriteHandler.java @@ -344,7 +344,7 @@ public void accept(List results, Throwable throwable) { } else { try { if (BackOffUtils.next(Sleeper.DEFAULT, backoff)) { - LOG.info(summarizeErrors("Attempting retry", results)); + LOG.info("{}", summarizeErrors("Attempting retry", results)); records = failedRecords(records, results); run(); } else { diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/EFOShardSubscriber.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/EFOShardSubscriber.java index 67bc8472a7c5..96ee35e5d45f 100644 --- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/EFOShardSubscriber.java +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/EFOShardSubscriber.java @@ -180,7 +180,7 @@ private static Throwable unwrapCompletionException(Throwable completionException String.format( "Pool %s - unknown case which is likely a bug: state=%s seqnum=%s", pool.getPoolId(), state, lastContinuationSequenceNumber); - LOG.warn(msg); + LOG.warn("{}", msg); done.completeExceptionally(new IllegalStateException(msg)); }; } diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sns/SnsIO.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sns/SnsIO.java index ecbe87a5da44..3da8fbfbf144 100644 --- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sns/SnsIO.java +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sns/SnsIO.java @@ -183,7 +183,7 @@ private boolean checkTopicExists(AwsOptions options) { return true; } catch (NotFoundException | InvalidParameterException e) { LoggerFactory.getLogger(Write.class) - .warn("Configured topic ARN '" + getTopicArn() + "' does not exist.", e); + .warn("Configured topic ARN '{}' does not exist.", getTopicArn(), e); return false; } } diff --git a/sdks/java/io/amqp/src/test/java/org/apache/beam/sdk/io/amqp/AmqpIOTest.java b/sdks/java/io/amqp/src/test/java/org/apache/beam/sdk/io/amqp/AmqpIOTest.java index ebb2a5487cf4..0e8cbd508bab 100644 --- a/sdks/java/io/amqp/src/test/java/org/apache/beam/sdk/io/amqp/AmqpIOTest.java +++ b/sdks/java/io/amqp/src/test/java/org/apache/beam/sdk/io/amqp/AmqpIOTest.java @@ -93,7 +93,7 @@ public void testWrite() throws Exception { messenger.recv(); while (messenger.incoming() > 0) { Message message = messenger.get(); - LOG.info("Received: " + message.getBody().toString()); + LOG.info("Received: {}", message.getBody().toString()); received.add(message.getBody().toString()); } } 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 ba4ac2769949..a136030535cb 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 @@ -305,7 +305,7 @@ static List createWriteReport( } if (numErrors > 0) { - LOG.error(errorMessages.toString()); + LOG.error("{}", errorMessages.toString()); if (throwWriteErrors) { throw new IOException(errorMessages.toString()); } diff --git a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseSharedConnection.java b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseSharedConnection.java index 52195fed275a..f0ae171bb201 100644 --- a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseSharedConnection.java +++ b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseSharedConnection.java @@ -120,7 +120,7 @@ private static synchronized void close(String confString) throws IOException { // Warn if connection count is not 0 and reset connection count if (connectionPool.get(confString).getSecond() < 0) { - LOG.warn("Connection count for + " + confString + " at below 0, " + getDebugString()); + LOG.warn("Connection count for {} at below 0, {}", confString, getDebugString()); connectionPool.get(confString).setSecond(0); } diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcUtil.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcUtil.java index 6cc3e6f3b9a6..80e7dbedce5a 100644 --- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcUtil.java +++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcUtil.java @@ -127,10 +127,10 @@ static URL[] saveFilesLocally(String driverJars) { ResourceId destResourceId = FileSystems.matchNewResource(destFile.getAbsolutePath(), false); copy(sourceResourceId, destResourceId); - LOG.info("Localized jar: " + sourceResourceId + " to: " + destResourceId); + LOG.info("Localized jar: {} to: {}", sourceResourceId, destResourceId); driverJarUrls.add(destFile.toURI().toURL()); } catch (IOException e) { - LOG.warn("Unable to copy " + jarPath, e); + LOG.warn("Unable to copy {}", jarPath, e); } }); return driverJarUrls.stream().toArray(URL[]::new); diff --git a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java index 2a7cd62d33d2..5deb7a2b62c7 100644 --- a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java +++ b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java @@ -820,7 +820,7 @@ private synchronized void closeSession() { session = null; } } catch (Exception e) { - LOG.error("Error closing session" + e.getMessage(), e); + LOG.error("Error closing session", e); } } diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java index 866dfd487108..68f8b773193d 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java @@ -630,16 +630,17 @@ private void commitCheckpointMark() { // KafkaIO because it relies on the offsets stored in KafkaCheckpointMark. if (now.isAfter(nextAllowedCommitFailLogTime)) { LOG.warn( - String.format( - "%s: Did not successfully commit finalized checkpoint for > %s. Current checkpoint: %s", - this, MIN_COMMIT_FAIL_LOG_INTERVAL, checkpointMark), + "{}: Did not successfully commit finalized checkpoint for > {}. Current checkpoint: {}", + this, + MIN_COMMIT_FAIL_LOG_INTERVAL, + checkpointMark, e); nextAllowedCommitFailLogTime = now.plus(MIN_COMMIT_FAIL_LOG_INTERVAL); } else { LOG.info( - String.format( - "%s: Could not commit finalized checkpoint. Commit will be retried with subsequent reads. Current checkpoint: %s", - this, checkpointMark), + "{}: Could not commit finalized checkpoint. Commit will be retried with subsequent reads. Current checkpoint: {}", + this, + checkpointMark, e); } } @@ -681,15 +682,15 @@ private void nextBatch() throws IOException { } if (recordsDequeuePollTimeout.isLongerThan(RECORDS_DEQUEUE_POLL_TIMEOUT_MIN)) { recordsDequeuePollTimeout = recordsDequeuePollTimeout.minus(Duration.millis(1)); - LOG.debug("Reducing poll timeout for reader to " + recordsDequeuePollTimeout.getMillis()); + LOG.debug("Reducing poll timeout for reader to {}", recordsDequeuePollTimeout.getMillis()); } return; } if (recordsDequeuePollTimeout.isShorterThan(RECORDS_DEQUEUE_POLL_TIMEOUT_MAX)) { recordsDequeuePollTimeout = recordsDequeuePollTimeout.plus(Duration.millis(1)); - LOG.debug("Increasing poll timeout for reader to " + recordsDequeuePollTimeout.getMillis()); - LOG.debug("Record count: " + records.count()); + LOG.debug("Increasing poll timeout for reader to {}", recordsDequeuePollTimeout.getMillis()); + LOG.debug("Record count: {}", records.count()); } partitionStates.forEach(p -> p.recordIter = records.records(p.topicPartition).iterator()); diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriter.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriter.java index cad0f8a68d8c..eca71d80334d 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriter.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriter.java @@ -185,7 +185,7 @@ private synchronized void checkForFailures() throws IOException { sendException = null; numSendFailures = 0; - LOG.warn(msg); + LOG.warn("{}", msg); throw new IOException(msg, e); } diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java index b1133eadb1cb..6e9bf4ec141a 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java @@ -593,7 +593,7 @@ public void processElement(@Element String element, OutputReceiver outpu public static class LogFn extends DoFn { @ProcessElement public void processElement(@Element String element, OutputReceiver outputReceiver) { - LOG.error(element); + LOG.error("{}", element); outputReceiver.output(element); } } diff --git a/sdks/java/io/kudu/src/test/java/org/apache/beam/sdk/io/kudu/KuduIOTest.java b/sdks/java/io/kudu/src/test/java/org/apache/beam/sdk/io/kudu/KuduIOTest.java index 4e3e7b1b3fce..fb626fe1ff28 100644 --- a/sdks/java/io/kudu/src/test/java/org/apache/beam/sdk/io/kudu/KuduIOTest.java +++ b/sdks/java/io/kudu/src/test/java/org/apache/beam/sdk/io/kudu/KuduIOTest.java @@ -183,17 +183,20 @@ private static class FakeWriter implements KuduService.Writer { private transient int id = 0; // set on deserialization @Override + @SuppressWarnings("Slf4jFormatShouldBeConst") public void openSession() { LOG.debug(String.format(LOG_OPEN_SESSION, id)); } @Override + @SuppressWarnings("Slf4jFormatShouldBeConst") public void write(Long entity) { LOG.debug(String.format(LOG_WRITE, entity)); LOG.debug(String.format(LOG_WRITE_VALUE, entity)); } @Override + @SuppressWarnings("Slf4jFormatShouldBeConst") public void closeSession() { LOG.debug(String.format(LOG_CLOSE_SESSION, id)); } @@ -234,6 +237,7 @@ private static class FakeReader extends BoundedSource.BoundedReader { } @Override + @SuppressWarnings("Slf4jFormatShouldBeConst") public boolean start() { // simulate the deserialization of a tablet scanner if (source.serializedToken != null) { diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java index 1283e873f2b6..1bcc54c15667 100644 --- a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java +++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java @@ -553,7 +553,7 @@ public List> split( ? Filters.and(findQuery.filters(), filters) : filters); FindQuery queryWithFilter = findQuery.toBuilder().setFilters(allFilters).build(); - LOG.debug("using filters: " + allFilters.toJson()); + LOG.debug("using filters: {}", allFilters.toJson()); sources.add(new BoundedMongoDbSource(spec.withQueryFn(queryWithFilter))); } } else { diff --git a/sdks/java/io/neo4j/src/main/java/org/apache/beam/sdk/io/neo4j/Neo4jIO.java b/sdks/java/io/neo4j/src/main/java/org/apache/beam/sdk/io/neo4j/Neo4jIO.java index 1dd95b44c2fc..2060b83b42dc 100644 --- a/sdks/java/io/neo4j/src/main/java/org/apache/beam/sdk/io/neo4j/Neo4jIO.java +++ b/sdks/java/io/neo4j/src/main/java/org/apache/beam/sdk/io/neo4j/Neo4jIO.java @@ -834,12 +834,10 @@ private void executeReadCypherStatement( String readWrite = writeTransaction ? "write" : "read"; LOG.info( - "Starting a " - + readWrite - + " transaction for cypher: " - + cypher - + ", parameters: " - + parametersString); + "Starting a {} transaction for cypher: {}, parameters: {}", + readWrite, + cypher, + parametersString); } // There are 2 ways to do a transaction on Neo4j: read or write @@ -854,7 +852,7 @@ private void executeReadCypherStatement( } else { count = driverSession.session.readTransaction(transactionWork, transactionConfig); } - LOG.debug("Retrieved " + count + " elements from Neo4J"); + LOG.debug("Retrieved {} elements from Neo4J", count); } } } @@ -1173,10 +1171,9 @@ private void executeCypherUnwindStatement() { if (logCypher && !loggingDone) { String parametersString = getParametersString(parametersMap); LOG.info( - "Starting a write transaction for unwind statement cypher: " - + cypher - + ", parameters: " - + parametersString); + "Starting a write transaction for unwind statement cypher: {}, parameters: {}", + cypher, + parametersString); loggingDone = true; } diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeIO.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeIO.java index 5350e0e2c1fe..d245cf7210f6 100644 --- a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeIO.java +++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeIO.java @@ -1372,10 +1372,10 @@ public void finishBundle() throws Exception { trackedFilesNames.remove(responseFileName); if (entry.getErrorsSeen() > 0) { - LOG.error(String.format("Snowflake SnowPipe ERROR: %s", entry.toString())); + LOG.error("Snowflake SnowPipe ERROR: {}", entry); } else if (entry.getErrorsSeen() == 0 && debugMode.equals(StreamingLogLevel.INFO)) { - LOG.info(String.format("Snowflake SnowPipe INFO: %s", entry.toString())); + LOG.info("Snowflake SnowPipe INFO: {}", entry); } } } @@ -1383,7 +1383,7 @@ public void finishBundle() throws Exception { } } trackedFilesNames.forEach( - file -> LOG.info(String.format("File %s was not found in ingest history", file))); + file -> LOG.info("File {} was not found in ingest history", file)); } } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java index 5342f86c13e1..13aa2808abf0 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java @@ -225,10 +225,9 @@ private static JCSMPProperties overrideConnectorProperties( } if ((ackWindowSize != null && ackWindowSize != BATCHED_PUB_ACK_WINDOW)) { LOG.warn( - String.format( - "SolaceIO.Write: Overriding PUB_ACK_WINDOW_SIZE to %d since" - + " HIGHER_THROUGHPUT mode was selected", - BATCHED_PUB_ACK_WINDOW)); + "SolaceIO.Write: Overriding PUB_ACK_WINDOW_SIZE to {} since" + + " HIGHER_THROUGHPUT mode was selected", + BATCHED_PUB_ACK_WINDOW); } // Override the properties @@ -250,10 +249,9 @@ private static JCSMPProperties overrideConnectorProperties( if ((ackWindowSize != null && ackWindowSize != STREAMING_PUB_ACK_WINDOW)) { LOG.warn( - String.format( - "SolaceIO.Write: Overriding PUB_ACK_WINDOW_SIZE to %d since" - + " LOWER_LATENCY mode was selected", - STREAMING_PUB_ACK_WINDOW)); + "SolaceIO.Write: Overriding PUB_ACK_WINDOW_SIZE to {} since" + + " LOWER_LATENCY mode was selected", + STREAMING_PUB_ACK_WINDOW); } // Override the properties diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceContainerManager.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceContainerManager.java index 6d2b3a27ffd0..d83a63c743cf 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceContainerManager.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceContainerManager.java @@ -115,7 +115,7 @@ private void executeCommand(String... command) { if (execResult.getExitCode() != 0) { logCommandError(execResult.getStderr(), command); } else { - LOG.info(execResult.getStdout()); + LOG.info("{}", execResult.getStdout()); } } catch (IOException | InterruptedException e) { logCommandError(e.getMessage(), command); diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java index 50130a8fbd1c..5537b015be07 100644 --- a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java @@ -677,7 +677,7 @@ private void flushBatch() throws IOException, InterruptedException { } else { // Note: this used in test cases to verify behavior - LOG.warn(String.format(RETRY_ATTEMPT_LOG, attempt), exception); + LOG.warn("Error writing to Solr. Retry attempt {}", attempt, exception); } } } diff --git a/sdks/java/io/sparkreceiver/3/src/test/java/org/apache/beam/sdk/io/sparkreceiver/RabbitMqReceiverWithOffset.java b/sdks/java/io/sparkreceiver/3/src/test/java/org/apache/beam/sdk/io/sparkreceiver/RabbitMqReceiverWithOffset.java index 730001ffe459..28730e0ffc7c 100644 --- a/sdks/java/io/sparkreceiver/3/src/test/java/org/apache/beam/sdk/io/sparkreceiver/RabbitMqReceiverWithOffset.java +++ b/sdks/java/io/sparkreceiver/3/src/test/java/org/apache/beam/sdk/io/sparkreceiver/RabbitMqReceiverWithOffset.java @@ -117,7 +117,7 @@ private void receive() { @Override public void stop(String message) { - LOG.info(message); + LOG.info("{}", message); isStopped.set(true); super.stop(message); try { @@ -135,7 +135,7 @@ public void stop(String message) { @Override public void stop(String message, Throwable error) { - LOG.error(message, error); + LOG.error("{}", message, error); isStopped.set(true); super.stop(message, error); try { diff --git a/sdks/java/io/sparkreceiver/3/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOIT.java b/sdks/java/io/sparkreceiver/3/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOIT.java index 32258934d0d9..9f21d65c7fdd 100644 --- a/sdks/java/io/sparkreceiver/3/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOIT.java +++ b/sdks/java/io/sparkreceiver/3/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOIT.java @@ -318,7 +318,7 @@ public void testSparkReceiverIOReadsInStreamingWithOffset() throws IOException { LOG.error("Can not write to rabbit.", e); fail(); } - LOG.info(sourceOptions.numRecords + " records were successfully written to RabbitMQ"); + LOG.info("{} records were successfully written to RabbitMQ", sourceOptions.numRecords); // Use streaming pipeline to read RabbitMQ records. readPipeline.getOptions().as(Options.class).setStreaming(true); diff --git a/sdks/java/io/thrift/src/main/java/org/apache/beam/sdk/io/thrift/ThriftIO.java b/sdks/java/io/thrift/src/main/java/org/apache/beam/sdk/io/thrift/ThriftIO.java index f97492143629..4e0522412416 100644 --- a/sdks/java/io/thrift/src/main/java/org/apache/beam/sdk/io/thrift/ThriftIO.java +++ b/sdks/java/io/thrift/src/main/java/org/apache/beam/sdk/io/thrift/ThriftIO.java @@ -187,7 +187,7 @@ public void processElement(@Element FileIO.ReadableFile file, OutputReceiver } } catch (Exception ioe) { String filename = file.getMetadata().resourceId().toString(); - LOG.error(String.format("Error in reading file: %1$s%n%2$s", filename, ioe)); + LOG.error("Error in reading file: {}", filename, ioe); throw new RuntimeException(ioe); } } @@ -265,10 +265,10 @@ public void write(T element) throws IOException { TProtocol protocol = protocolFactory.getProtocol(new TIOStreamTransport(baos)); element.write(protocol); } catch (TTransportException tte) { - LOG.error("Error in transport to TIOStreamTransport: " + tte); + LOG.error("Error in transport to TIOStreamTransport", tte); throw new RuntimeException(tte); } catch (TException te) { - LOG.error("Error in writing element to TProtocol: " + te); + LOG.error("Error in writing element to TProtocol", te); throw new RuntimeException(te); } this.stream.write(baos.toByteArray()); diff --git a/sdks/java/ml/inference/openai/src/test/java/org/apache/beam/sdk/ml/inference/openai/OpenAIModelHandlerIT.java b/sdks/java/ml/inference/openai/src/test/java/org/apache/beam/sdk/ml/inference/openai/OpenAIModelHandlerIT.java index 5bd6ef4a2450..517c7a368a83 100644 --- a/sdks/java/ml/inference/openai/src/test/java/org/apache/beam/sdk/ml/inference/openai/OpenAIModelHandlerIT.java +++ b/sdks/java/ml/inference/openai/src/test/java/org/apache/beam/sdk/ml/inference/openai/OpenAIModelHandlerIT.java @@ -402,7 +402,7 @@ public void testWithJsonOutputFormat() { batches) { for (PredictionResult result : batch) { String output = result.getOutput().getModelResponse(); - LOG.info("Structured output: " + output); + LOG.info("Structured output: {}", output); // Verify output contains expected information assertTrue( diff --git a/sdks/java/testing/watermarks/src/main/java/org/apache/beam/sdk/testing/watermarks/WatermarkLatency.java b/sdks/java/testing/watermarks/src/main/java/org/apache/beam/sdk/testing/watermarks/WatermarkLatency.java index f1673b35a067..68600a44f494 100644 --- a/sdks/java/testing/watermarks/src/main/java/org/apache/beam/sdk/testing/watermarks/WatermarkLatency.java +++ b/sdks/java/testing/watermarks/src/main/java/org/apache/beam/sdk/testing/watermarks/WatermarkLatency.java @@ -185,7 +185,7 @@ public void process(ProcessContext c) { p95.getMillis(), p99.getMillis(), numElements); - LOG.info(out); + LOG.info("{}", out); } private void computePercentiles(Iterable vals) { diff --git a/sdks/java/transform-service/launcher/src/main/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncher.java b/sdks/java/transform-service/launcher/src/main/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncher.java index c0a9097a762f..c26729a66a4e 100644 --- a/sdks/java/transform-service/launcher/src/main/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncher.java +++ b/sdks/java/transform-service/launcher/src/main/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncher.java @@ -89,11 +89,11 @@ private TransformServiceLauncher( // Setting up the credentials directory. File credentialsDir = Paths.get(tmpDir, "credentials_dir").toFile(); if (credentialsDir.exists()) { - LOG.info("Reusing the existing credentials directory " + credentialsDir.getAbsolutePath()); + LOG.info("Reusing the existing credentials directory {}", credentialsDir.getAbsolutePath()); } else { LOG.info( - "Creating a temporary directory for storing credentials: " - + credentialsDir.getAbsolutePath()); + "Creating a temporary directory for storing credentials: {}", + credentialsDir.getAbsolutePath()); if (!credentialsDir.mkdir()) { throw new IOException( "Could not create a temporary directory for storing credentials: " @@ -131,11 +131,11 @@ private TransformServiceLauncher( File dependenciesDir = Paths.get(tmpDir, "dependencies_dir").toFile(); Path updatedRequirementsFilePath = Paths.get(dependenciesDir.toString(), "requirements.txt"); if (dependenciesDir.exists()) { - LOG.info("Reusing the existing dependencies directory " + dependenciesDir.getAbsolutePath()); + LOG.info("Reusing the existing dependencies directory {}", dependenciesDir.getAbsolutePath()); } else { LOG.info( - "Creating a temporary directory for storing dependencies: " - + dependenciesDir.getAbsolutePath()); + "Creating a temporary directory for storing dependencies: {}", + dependenciesDir.getAbsolutePath()); if (!dependenciesDir.mkdir()) { throw new IOException( "Could not create a temporary directory for storing dependencies: " From f1dfb6440191d8192a623f7cfc9eeee5a03350d8 Mon Sep 17 00:00:00 2001 From: Radek Stankiewicz Date: Mon, 9 Mar 2026 13:17:50 +0100 Subject: [PATCH 3/4] Resolving Slf4jFormatShouldBeConst violations --- .../beam/gradle/BeamModulePlugin.groovy | 1 - .../beam/examples/SchemaTransformExample.java | 2 +- .../beam/examples/SqlTransformExample.java | 2 +- .../examples/ApproximateQuantilesExample.java | 2 +- .../examples/CoCombineTransformExample.java | 2 +- .../beam/examples/CoGroupByKeyExample.java | 2 +- .../apache/beam/examples/CombineExample.java | 2 +- .../apache/beam/examples/CountExample.java | 2 +- .../beam/examples/CountPerKeyExample.java | 2 +- .../apache/beam/examples/CreateExample.java | 2 +- .../beam/examples/DebuggingWordCount.java | 4 +-- .../apache/beam/examples/DistinctExample.java | 2 +- .../beam/examples/FlatMapElementsExample.java | 2 +- .../examples/GroupIntoBatchesExample.java | 2 +- .../org/apache/beam/examples/KeysExample.java | 2 +- .../apache/beam/examples/KvSwapExample.java | 2 +- .../apache/beam/examples/LatestExample.java | 2 +- .../beam/examples/MapElementsExample.java | 2 +- .../org/apache/beam/examples/MaxExample.java | 2 +- .../beam/examples/MaxPerKeyExample.java | 2 +- .../org/apache/beam/examples/MeanExample.java | 2 +- .../beam/examples/MeanPerKeyExample.java | 2 +- .../org/apache/beam/examples/MinExample.java | 2 +- .../beam/examples/MinPerKeyExample.java | 2 +- .../beam/examples/PartitionExample.java | 2 +- .../beam/examples/RateLimiterSimple.java | 2 +- .../apache/beam/examples/RegexExample.java | 2 +- .../apache/beam/examples/SampleExample.java | 2 +- .../org/apache/beam/examples/SumExample.java | 2 +- .../beam/examples/SumPerKeyExample.java | 2 +- .../apache/beam/examples/ToStringExample.java | 2 +- .../org/apache/beam/examples/TopExample.java | 2 +- .../apache/beam/examples/ValuesExample.java | 2 +- .../org/apache/beam/examples/ViewExample.java | 2 +- .../apache/beam/examples/WindowExample.java | 2 +- .../datatokenization/utils/CsvConverters.java | 2 +- .../datatokenization/utils/SchemasUtils.java | 4 +-- .../examples/complete/game/GameStats.java | 10 +++----- .../examples/complete/game/UserScore.java | 2 +- .../complete/kafkatopubsub/KafkaToPubsub.java | 10 +++----- .../cookbook/BigQueryStreamingTornadoes.java | 2 +- .../examples/cookbook/BigQueryTornadoes.java | 2 +- .../cookbook/MinimalBigQueryTornadoes.java | 2 +- .../subprocess/kernel/SubProcessIOFiles.java | 9 ++++--- .../subprocess/kernel/SubProcessKernel.java | 10 ++++---- .../utils/CallingSubProcessUtils.java | 2 +- .../examples/subprocess/utils/FileUtils.java | 25 ++++++++----------- .../gcp/dataflow/DefaultPipelineLauncher.java | 2 +- .../runners/dataflow/DataflowPipelineJob.java | 1 + .../beam/runners/dataflow/DataflowRunner.java | 7 +++--- .../dataflow/internal/CustomSources.java | 2 +- .../runners/dataflow/util/MonitoringUtil.java | 1 + .../runners/dataflow/util/PackageUtil.java | 1 + .../runners/dataflow/worker/ConcatReader.java | 2 +- .../dataflow/worker/DoFnInstanceManagers.java | 10 ++++---- .../runners/dataflow/worker/WindmillSink.java | 18 +++++-------- .../dataflow/worker/WorkerCustomSources.java | 2 +- .../worker/WorkerPipelineOptionsFactory.java | 4 +-- .../client/AbstractWindmillStream.java | 2 +- .../grpc/observers/DirectStreamObserver.java | 2 +- .../processing/StreamingWorkScheduler.java | 4 +-- .../client/grpc/GrpcWindmillServerTest.java | 2 +- .../extensions/sql/impl/CatalogSchema.java | 2 +- .../extensions/sql/impl/rel/BeamCalcRel.java | 2 +- .../sql/impl/rel/BeamEnumerableConverter.java | 2 +- .../meta/provider/bigquery/BigQueryTable.java | 2 +- .../meta/provider/kafka/BeamKafkaTable.java | 2 +- .../sql/meta/provider/text/TextTable.java | 2 +- .../apache/beam/sdk/nexmark/NexmarkUtils.java | 2 +- .../beam/sdk/nexmark/queries/Query10.java | 25 ++++++++++--------- 70 files changed, 121 insertions(+), 129 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 028600f762ec..48aa84b7cbd7 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -1556,7 +1556,6 @@ class BeamModulePlugin implements Plugin { "MixedMutabilityReturnType", "PreferJavaTimeOverload", "NonCanonicalType", - "Slf4jFormatShouldBeConst", "Slf4jSignOnlyFormat", "StaticAssignmentInConstructor", "ThreadPriorityCheck", diff --git a/examples/java/sql/src/main/java/org/apache/beam/examples/SchemaTransformExample.java b/examples/java/sql/src/main/java/org/apache/beam/examples/SchemaTransformExample.java index 861b35b463e1..5b8e4a34cf2c 100644 --- a/examples/java/sql/src/main/java/org/apache/beam/examples/SchemaTransformExample.java +++ b/examples/java/sql/src/main/java/org/apache/beam/examples/SchemaTransformExample.java @@ -102,7 +102,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/sql/src/main/java/org/apache/beam/examples/SqlTransformExample.java b/examples/java/sql/src/main/java/org/apache/beam/examples/SqlTransformExample.java index 62839f7a485f..9c2302c3de2f 100644 --- a/examples/java/sql/src/main/java/org/apache/beam/examples/SqlTransformExample.java +++ b/examples/java/sql/src/main/java/org/apache/beam/examples/SqlTransformExample.java @@ -96,7 +96,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/ApproximateQuantilesExample.java b/examples/java/src/main/java/org/apache/beam/examples/ApproximateQuantilesExample.java index 52376e91d1d4..9e2a96b1eca9 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/ApproximateQuantilesExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/ApproximateQuantilesExample.java @@ -71,7 +71,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/CoCombineTransformExample.java b/examples/java/src/main/java/org/apache/beam/examples/CoCombineTransformExample.java index 3b91b29e9f28..0ee7d7bcac89 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/CoCombineTransformExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/CoCombineTransformExample.java @@ -211,7 +211,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/CoGroupByKeyExample.java b/examples/java/src/main/java/org/apache/beam/examples/CoGroupByKeyExample.java index 12d90b2f6bf7..c77708b5de20 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/CoGroupByKeyExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/CoGroupByKeyExample.java @@ -85,7 +85,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/CombineExample.java b/examples/java/src/main/java/org/apache/beam/examples/CombineExample.java index e63524989729..24bed27c2360 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/CombineExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/CombineExample.java @@ -69,7 +69,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/CountExample.java b/examples/java/src/main/java/org/apache/beam/examples/CountExample.java index e7586b6d5ea5..cb0bd0ecf943 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/CountExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/CountExample.java @@ -64,7 +64,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/CountPerKeyExample.java b/examples/java/src/main/java/org/apache/beam/examples/CountPerKeyExample.java index 9b84ae02c113..9bf9bf1ef00f 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/CountPerKeyExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/CountPerKeyExample.java @@ -68,7 +68,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/CreateExample.java b/examples/java/src/main/java/org/apache/beam/examples/CreateExample.java index 55cabd69fceb..5943ffa489d4 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/CreateExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/CreateExample.java @@ -80,7 +80,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java index f9b2e326127d..7c54e238da33 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java @@ -119,14 +119,14 @@ public void processElement(ProcessContext c) { if (filter.matcher(c.element().getKey()).matches()) { // Log at the "DEBUG" level each element that we match. When executing this pipeline // these log lines will appear only if the log level is set to "DEBUG" or lower. - LOG.debug("Matched: " + c.element().getKey()); + LOG.debug("Matched: {}", c.element().getKey()); matchedWords.inc(); c.output(c.element()); } else { // Log at the "TRACE" level each element that is not matched. Different log levels // can be used to control the verbosity of logging providing an effective mechanism // to filter less important information. - LOG.trace("Did not match: " + c.element().getKey()); + LOG.trace("Did not match: {}", c.element().getKey()); unmatchedWords.inc(); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/DistinctExample.java b/examples/java/src/main/java/org/apache/beam/examples/DistinctExample.java index d83399fcb832..d3ff9d663f44 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/DistinctExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/DistinctExample.java @@ -69,7 +69,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/FlatMapElementsExample.java b/examples/java/src/main/java/org/apache/beam/examples/FlatMapElementsExample.java index 6dcfad0ff6a0..71d05ac7ade3 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/FlatMapElementsExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/FlatMapElementsExample.java @@ -79,7 +79,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/GroupIntoBatchesExample.java b/examples/java/src/main/java/org/apache/beam/examples/GroupIntoBatchesExample.java index 7e3e4639d5fe..78e898b9c173 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/GroupIntoBatchesExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/GroupIntoBatchesExample.java @@ -75,7 +75,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/KeysExample.java b/examples/java/src/main/java/org/apache/beam/examples/KeysExample.java index d851b1e8524b..155834bc0a43 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/KeysExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/KeysExample.java @@ -71,7 +71,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/KvSwapExample.java b/examples/java/src/main/java/org/apache/beam/examples/KvSwapExample.java index 1b0fd2f7b520..090779de7c36 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/KvSwapExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/KvSwapExample.java @@ -70,7 +70,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/LatestExample.java b/examples/java/src/main/java/org/apache/beam/examples/LatestExample.java index 7b214ff92b39..5e9662a2f1ab 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/LatestExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/LatestExample.java @@ -85,7 +85,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/MapElementsExample.java b/examples/java/src/main/java/org/apache/beam/examples/MapElementsExample.java index ff5a52c1b10d..93d885750ae8 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/MapElementsExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/MapElementsExample.java @@ -77,7 +77,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/MaxExample.java b/examples/java/src/main/java/org/apache/beam/examples/MaxExample.java index 95c27592eeb5..9173d11754a3 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/MaxExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/MaxExample.java @@ -64,7 +64,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/MaxPerKeyExample.java b/examples/java/src/main/java/org/apache/beam/examples/MaxPerKeyExample.java index 8a0b599c7502..f5eda8179929 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/MaxPerKeyExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/MaxPerKeyExample.java @@ -68,7 +68,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/MeanExample.java b/examples/java/src/main/java/org/apache/beam/examples/MeanExample.java index 1084bf6fb809..a31907977dfb 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/MeanExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/MeanExample.java @@ -64,7 +64,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/MeanPerKeyExample.java b/examples/java/src/main/java/org/apache/beam/examples/MeanPerKeyExample.java index 6b853d599b5d..aecccee067e4 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/MeanPerKeyExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/MeanPerKeyExample.java @@ -68,7 +68,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/MinExample.java b/examples/java/src/main/java/org/apache/beam/examples/MinExample.java index f9fce309c0cb..a76bcdc5ee3f 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/MinExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/MinExample.java @@ -64,7 +64,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/MinPerKeyExample.java b/examples/java/src/main/java/org/apache/beam/examples/MinPerKeyExample.java index 6861e5e49465..d3c0312feaf3 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/MinPerKeyExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/MinPerKeyExample.java @@ -68,7 +68,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("LogOutput: {} {}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/PartitionExample.java b/examples/java/src/main/java/org/apache/beam/examples/PartitionExample.java index 51e44daf5ae7..b34f2bdd16bf 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/PartitionExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/PartitionExample.java @@ -193,7 +193,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/RateLimiterSimple.java b/examples/java/src/main/java/org/apache/beam/examples/RateLimiterSimple.java index a33e99e4b239..7bd35b101960 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/RateLimiterSimple.java +++ b/examples/java/src/main/java/org/apache/beam/examples/RateLimiterSimple.java @@ -106,7 +106,7 @@ public void processElement(ProcessContext c) throws Exception { } // Simulate external API call - LOG.info("Processing: " + element); + LOG.info("Processing: {}", element); Thread.sleep(100); c.output("Processed: " + element); } diff --git a/examples/java/src/main/java/org/apache/beam/examples/RegexExample.java b/examples/java/src/main/java/org/apache/beam/examples/RegexExample.java index e5ee02224697..a0d467718bed 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/RegexExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/RegexExample.java @@ -76,7 +76,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/SampleExample.java b/examples/java/src/main/java/org/apache/beam/examples/SampleExample.java index 5d920a3851d4..ed1d90606d3b 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/SampleExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/SampleExample.java @@ -77,7 +77,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/SumExample.java b/examples/java/src/main/java/org/apache/beam/examples/SumExample.java index b52557fddbda..00fcc8697926 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/SumExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/SumExample.java @@ -64,7 +64,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/SumPerKeyExample.java b/examples/java/src/main/java/org/apache/beam/examples/SumPerKeyExample.java index e3a312314017..45d4a9ffd852 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/SumPerKeyExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/SumPerKeyExample.java @@ -68,7 +68,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/ToStringExample.java b/examples/java/src/main/java/org/apache/beam/examples/ToStringExample.java index 276f1754fbb3..23e3db6cfd96 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/ToStringExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/ToStringExample.java @@ -75,7 +75,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/TopExample.java b/examples/java/src/main/java/org/apache/beam/examples/TopExample.java index 6be6bca68086..520af0f66550 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/TopExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/TopExample.java @@ -65,7 +65,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/ValuesExample.java b/examples/java/src/main/java/org/apache/beam/examples/ValuesExample.java index 2b5ecd3aa3be..3fc9e84fcb39 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/ValuesExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/ValuesExample.java @@ -71,7 +71,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/ViewExample.java b/examples/java/src/main/java/org/apache/beam/examples/ViewExample.java index 2c44eff3b4e3..01c1b1c3f31d 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/ViewExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/ViewExample.java @@ -113,7 +113,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/WindowExample.java b/examples/java/src/main/java/org/apache/beam/examples/WindowExample.java index b143d5871aa6..244a42bc9613 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/WindowExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/WindowExample.java @@ -88,7 +88,7 @@ public LogOutput(String prefix) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java index 76eb486e21b0..f270845c90d4 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java @@ -130,7 +130,7 @@ static String buildJsonString( break; default: - LOG.error("Invalid data type, got: " + type); + LOG.error("Invalid data type, got: {}", type); throw new RuntimeException("Invalid data type, got: " + type); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java index 9d908e8bd6ca..30db973649db 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java @@ -73,7 +73,7 @@ public SchemasUtils(String path, Charset encoding) throws IOException { byte[] encoded = Files.readAllBytes(Paths.get(path)); parseJson(new String(encoded, encoding)); } - LOG.info("Extracted schema: " + bigQuerySchema.toPrettyString()); + LOG.info("Extracted schema: {}", bigQuerySchema.toPrettyString()); } public TableSchema getBigQuerySchema() { @@ -156,7 +156,7 @@ public static String getGcsFileAsString(String filePath) { return CharStreams.toString(reader); } catch (IOException ioe) { - LOG.error("File system i/o error: " + ioe.getMessage()); + LOG.error("File system i/o error", ioe); throw new RuntimeException(ioe); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/game/GameStats.java b/examples/java/src/main/java/org/apache/beam/examples/complete/game/GameStats.java index b81634626246..16983af41bb6 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/game/GameStats.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/game/GameStats.java @@ -130,12 +130,10 @@ public void processElement(ProcessContext c) { Double gmc = c.sideInput(globalMeanScore); if (score > (gmc * SCORE_WEIGHT)) { LOG.info( - "user " - + c.element().getKey() - + " spammer score " - + score - + " with mean " - + gmc); + "user {} spammer score {} with mean {}", + c.element().getKey(), + score, + gmc); numSpammerUsers.inc(); c.output(c.element()); } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/game/UserScore.java b/examples/java/src/main/java/org/apache/beam/examples/complete/game/UserScore.java index 8baa82cad8d9..f4dfd4334bd4 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/game/UserScore.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/game/UserScore.java @@ -172,7 +172,7 @@ public void processElement(ProcessContext c) { c.output(gInfo); } catch (ArrayIndexOutOfBoundsException | NumberFormatException e) { numParseErrors.inc(); - LOG.info("Parse error on " + c.element() + ", " + e.getMessage()); + LOG.info("Parse error on {}", c.element(), e); } } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/KafkaToPubsub.java b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/KafkaToPubsub.java index 49aaeb5ac577..75b04879121a 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/KafkaToPubsub.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/KafkaToPubsub.java @@ -199,12 +199,10 @@ public static PipelineResult run(Pipeline pipeline, KafkaToPubsubOptions options "bootstrapServers cannot be an empty string."); LOG.info( - "Starting Kafka-To-PubSub pipeline with parameters bootstrap servers:" - + options.getBootstrapServers() - + " input topics: " - + options.getInputTopics() - + " output pubsub topic: " - + options.getOutputTopic()); + "Starting Kafka-To-PubSub pipeline with parameters bootstrap servers:{} input topics: {} output pubsub topic: {}", + options.getBootstrapServers(), + options.getInputTopics(), + options.getOutputTopic()); /* * Steps: diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryStreamingTornadoes.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryStreamingTornadoes.java index 395da115e0ca..3b2653b7601e 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryStreamingTornadoes.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryStreamingTornadoes.java @@ -196,7 +196,7 @@ public static void applyBigQueryStreamingTornadoes(Pipeline p, Options options) } public static void runBigQueryTornadoes(Options options) { - LOG.info("Running BigQuery Tornadoes with options " + options.toString()); + LOG.info("Running BigQuery Tornadoes with options {}", options.toString()); Pipeline p = Pipeline.create(options); applyBigQueryStreamingTornadoes(p, options); p.run().waitUntilFinish(); diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java index b7ef94338d74..43d720e35268 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java @@ -220,7 +220,7 @@ public static void applyBigQueryTornadoes(Pipeline p, Options options) { } public static void runBigQueryTornadoes(Options options) { - LOG.info("Running BigQuery Tornadoes with options " + options.toString()); + LOG.info("Running BigQuery Tornadoes with options {}", options.toString()); Pipeline p = Pipeline.create(options); applyBigQueryTornadoes(p, options); p.run().waitUntilFinish(); diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/MinimalBigQueryTornadoes.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/MinimalBigQueryTornadoes.java index 60b5c02a5f46..713af1d50953 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/MinimalBigQueryTornadoes.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/MinimalBigQueryTornadoes.java @@ -135,7 +135,7 @@ static class LogOutput extends DoFn { @ProcessElement public void processElement(ProcessContext c) { - LOG.info(prefix + c.element()); + LOG.info("{}{}", prefix, c.element()); c.output(c.element()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/subprocess/kernel/SubProcessIOFiles.java b/examples/java/src/main/java/org/apache/beam/examples/subprocess/kernel/SubProcessIOFiles.java index 5f47cb52c48e..f6131010a571 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/subprocess/kernel/SubProcessIOFiles.java +++ b/examples/java/src/main/java/org/apache/beam/examples/subprocess/kernel/SubProcessIOFiles.java @@ -120,11 +120,12 @@ public void copyOutPutFilesToBucket(SubProcessConfiguration configuration, Strin } LOG.info( - String.format( - "Log Files for process: %s outFile was: %s errFile was: %s", - params, outFileLocation, errFileLocation)); + "Log Files for process: {} outFile was: {} errFile was: {}", + params, + outFileLocation, + errFileLocation); } else { - LOG.error(String.format("There was no output file or err file for process %s", params)); + LOG.error("There was no output file or err file for process {}", params); } } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/subprocess/kernel/SubProcessKernel.java b/examples/java/src/main/java/org/apache/beam/examples/subprocess/kernel/SubProcessKernel.java index 6853e2aaf9e9..8a04f211a412 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/subprocess/kernel/SubProcessKernel.java +++ b/examples/java/src/main/java/org/apache/beam/examples/subprocess/kernel/SubProcessKernel.java @@ -167,9 +167,9 @@ private Process execBinary( } catch (Exception ex) { LOG.error( - String.format( - "Error running process with parameters %s error was %s ", - createLogEntryFromInputs(builder.command()), ex.getMessage())); + "Error running process with parameters {}", + createLogEntryFromInputs(builder.command()), + ex); throw new Exception(ex); } } @@ -190,7 +190,7 @@ private List collectProcessResults( try { - LOG.debug(String.format("Executing process %s", createLogEntryFromInputs(builder.command()))); + LOG.debug("Executing process {}", createLogEntryFromInputs(builder.command())); // If process exit value is not 0 then subprocess failed, record logs if (process.exitValue() != 0) { @@ -237,7 +237,7 @@ private byte[] collectProcessResultsBytes( try { - LOG.debug(String.format("Executing process %s", createLogEntryFromInputs(builder.command()))); + LOG.debug("Executing process {}", createLogEntryFromInputs(builder.command())); // If process exit value is not 0 then subprocess failed, record logs if (process.exitValue() != 0) { diff --git a/examples/java/src/main/java/org/apache/beam/examples/subprocess/utils/CallingSubProcessUtils.java b/examples/java/src/main/java/org/apache/beam/examples/subprocess/utils/CallingSubProcessUtils.java index 0d5f978fab9a..3abbb25d7c79 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/subprocess/utils/CallingSubProcessUtils.java +++ b/examples/java/src/main/java/org/apache/beam/examples/subprocess/utils/CallingSubProcessUtils.java @@ -64,7 +64,7 @@ public static void setUp(SubProcessConfiguration configuration, String binaryNam public static synchronized void initSemaphore(Integer permits, String binaryName) { if (!semaphores.containsKey(binaryName)) { - LOG.info(String.format(String.format("Initialized Semaphore for binary %s ", binaryName))); + LOG.info("Initialized Semaphore for binary {} ", binaryName); semaphores.put(binaryName, new Semaphore(permits)); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/subprocess/utils/FileUtils.java b/examples/java/src/main/java/org/apache/beam/examples/subprocess/utils/FileUtils.java index 398acb7c35f1..d0244d233722 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/subprocess/utils/FileUtils.java +++ b/examples/java/src/main/java/org/apache/beam/examples/subprocess/utils/FileUtils.java @@ -63,7 +63,7 @@ public static String copyFileFromWorkerToGCS( ResourceId sourceFile = getFileResourceId(configuration.getWorkerPath(), fileName.toString()); - LOG.info("Copying file from worker " + sourceFile); + LOG.info("Copying file from worker {}", sourceFile); ResourceId destinationFile = getFileResourceId(configuration.getSourcePath(), fileName.toString()); @@ -72,8 +72,7 @@ public static String copyFileFromWorkerToGCS( try { return copyFile(sourceFile, destinationFile); } catch (Exception ex) { - LOG.error( - String.format("Error copying file from %s to %s", sourceFile, destinationFile), ex); + LOG.error("Error copying file from {} to {}", sourceFile, destinationFile, ex); throw ex; } } @@ -86,23 +85,22 @@ public static String copyFileFromGCSToWorker(ExecutableFile execuableFile) throw FileSystems.matchNewResource(execuableFile.getDestinationLocation(), false); try { LOG.info( - String.format( - "Moving File %s to %s ", - execuableFile.getSourceGCSLocation(), execuableFile.getDestinationLocation())); + "Moving File {} to {} ", + execuableFile.getSourceGCSLocation(), + execuableFile.getDestinationLocation()); Path path = Paths.get(execuableFile.getDestinationLocation()); if (path.toFile().exists()) { LOG.warn( - String.format( - "Overwriting file %s, should only see this once per worker.", - execuableFile.getDestinationLocation())); + "Overwriting file {}, should only see this once per worker.", + execuableFile.getDestinationLocation()); } copyFile(sourceFile, destinationFile); path.toFile().setExecutable(true); return path.toString(); } catch (Exception ex) { - LOG.error(String.format("Error moving file : %s ", execuableFile.fileName), ex); + LOG.error("Error moving file : {} ", execuableFile.fileName, ex); throw ex; } } @@ -144,13 +142,12 @@ public static void createDirectoriesOnWorker(SubProcessConfiguration configurati if (!path.toFile().exists()) { Files.createDirectories(path); - LOG.info(String.format("Created Folder %s ", path.toFile())); + LOG.info("Created Folder {} ", path.toFile()); } } catch (FileAlreadyExistsException ex) { LOG.warn( - String.format( - " Tried to create folder %s which already existsed, this should not happen!", - configuration.getWorkerPath()), + " Tried to create folder {} which already existsed, this should not happen!", + configuration.getWorkerPath(), ex); } } diff --git a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dataflow/DefaultPipelineLauncher.java b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dataflow/DefaultPipelineLauncher.java index 7e496aa18938..11a09c4ba749 100644 --- a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dataflow/DefaultPipelineLauncher.java +++ b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dataflow/DefaultPipelineLauncher.java @@ -467,7 +467,7 @@ private String executeCommandAndParseResponse(String cmd) throws IOException { new ProcessBuilder().command("/bin/bash", "-c", cmd).redirectErrorStream(true).start(); String output = new String(ByteStreams.toByteArray(process.getInputStream()), StandardCharsets.UTF_8); - LOG.info(output); + LOG.info("Command output: {}", output); Matcher m = JOB_ID_PATTERN.matcher(output); if (!m.find()) { throw new RuntimeException( diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java index 2be4b569ca9b..400f161dee2f 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java @@ -433,6 +433,7 @@ private Exception processJobMessages( private AtomicReference> cancelState = new AtomicReference<>(); + @SuppressWarnings("Slf4jFormatShouldBeConst") @Override public State cancel() throws IOException { // Enforce that a cancel() call on the job is done at most once - as diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 9d963af0ecb5..928a06fc1fbc 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -1238,6 +1238,7 @@ private static boolean includesTransformUpgrades(Pipeline pipeline) { private static final Random RANDOM = new Random(); + @SuppressWarnings("Slf4jFormatShouldBeConst") @Override public DataflowPipelineJob run(Pipeline pipeline) { // Multi-language pipelines and pipelines that include upgrades should automatically be upgraded @@ -1890,9 +1891,9 @@ public void visitPrimitiveTransform(Node node) { }); for (String unconsumed : unconsumedDLQ.values()) { LOG.warn( - "No transform processes the failed-inserts output from BigQuery sink: " - + unconsumed - + "! Not processing failed inserts means that those rows will be lost."); + "No transform processes the failed-inserts output from BigQuery sink: {}" + + "! Not processing failed inserts means that those rows will be lost.", + unconsumed); } } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/CustomSources.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/CustomSources.java index 007402baf71b..1de8febebc3e 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/CustomSources.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/CustomSources.java @@ -85,7 +85,7 @@ public static com.google.api.services.dataflow.model.Source serializeToCloudSour try { metadata.setEstimatedSizeBytes(boundedSource.getEstimatedSizeBytes(options)); } catch (Exception e) { - LOG.warn("Size estimation of the source failed: " + source, e); + LOG.warn("Size estimation of the source failed: {}", source, e); } } else if (source instanceof UnboundedSource) { UnboundedSource unboundedSource = (UnboundedSource) source; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/MonitoringUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/MonitoringUtil.java index 0237923e4e47..d117cf786129 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/MonitoringUtil.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/MonitoringUtil.java @@ -69,6 +69,7 @@ public interface JobMessagesHandler { public static class LoggingHandler implements JobMessagesHandler { private static final Logger LOG = LoggerFactory.getLogger(LoggingHandler.class); + @SuppressWarnings("Slf4jFormatShouldBeConst") @Override public void process(List messages) { for (JobMessage message : messages) { diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java index 2f440dcca071..31c9a5f3ce0d 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java @@ -157,6 +157,7 @@ private StagingResult stagePackageSynchronously( } } + @SuppressWarnings("Slf4jFormatShouldBeConst") private StagingResult tryStagePackageWithRetry( PackageAttributes attributes, Sleeper retrySleeper, CreateOptions createOptions) throws IOException, InterruptedException { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ConcatReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ConcatReader.java index 1f49591d162e..d166bc33191a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ConcatReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ConcatReader.java @@ -267,7 +267,7 @@ public DynamicSplitResult requestDynamicSplit(DynamicSplitRequest splitRequest) return new DynamicSplitResultWithPosition( SourceTranslationUtils.cloudPositionToReaderPosition(positionToSplit)); } else { - LOG.debug("Could not perform the dynamic split request " + splitRequest); + LOG.debug("Could not perform the dynamic split request {}", splitRequest); return null; } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DoFnInstanceManagers.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DoFnInstanceManagers.java index 60a1b067cc52..a956f66aec52 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DoFnInstanceManagers.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DoFnInstanceManagers.java @@ -92,11 +92,11 @@ private ConcurrentQueueInstanceManager(DoFnInfo info, PipelineOptions opti if (elapsed > 180_000) { // 3 min // Work item could fail for long-running setup due to Dataflow worker lease timeout LOG.warn( - String.format( - "DoFn.setup for %s ran for %d seconds.\nThis could cause Dataflow worker " - + "lease expire and failing the job. DoFn.Setup should not contain long " - + "running operations.", - fn.getDoFn().getClass(), elapsed / 1_000)); + "DoFn.setup for {} ran for {} seconds.\nThis could cause Dataflow worker " + + "lease expire and failing the job. DoFn.Setup should not contain long " + + "running operations.", + fn.getDoFn().getClass(), + elapsed / 1_000); } return fn; } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java index 2ed29125bd40..12f21a6d49fe 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java @@ -248,12 +248,9 @@ public long add(WindowedValue data) throws IOException { throw new OutputTooLargeException("Key too large: " + key.size()); } else { LOG.error( - "Trying to output too large key with size " - + key.size() - + ". Limit is " - + context.getMaxOutputKeyBytes() - + ". See https://cloud.google.com/dataflow/docs/guides/common-errors#key-commit-too-large-exception." - + " Running with --experiments=throw_exceptions_on_large_output will instead throw an OutputTooLargeException which may be caught in user code."); + "Trying to output too large key with size {}. Limit is {}. See https://cloud.google.com/dataflow/docs/guides/common-errors#key-commit-too-large-exception. Running with --experiments=throw_exceptions_on_large_output will instead throw an OutputTooLargeException which may be caught in user code.", + key.size(), + context.getMaxOutputKeyBytes()); } } if (value.size() > context.getMaxOutputValueBytes()) { @@ -261,12 +258,9 @@ public long add(WindowedValue data) throws IOException { throw new OutputTooLargeException("Value too large: " + value.size()); } else { LOG.error( - "Trying to output too large value with size " - + value.size() - + ". Limit is " - + context.getMaxOutputValueBytes() - + ". See https://cloud.google.com/dataflow/docs/guides/common-errors#key-commit-too-large-exception." - + " Running with --experiments=throw_exceptions_on_large_output will instead throw an OutputTooLargeException which may be caught in user code."); + "Trying to output too large value with size {}. Limit is {}. See https://cloud.google.com/dataflow/docs/guides/common-errors#key-commit-too-large-exception. Running with --experiments=throw_exceptions_on_large_output will instead throw an OutputTooLargeException which may be caught in user code.", + value.size(), + context.getMaxOutputValueBytes()); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java index ece8e017fd42..c00edffeaf95 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java @@ -161,7 +161,7 @@ private static com.google.api.services.dataflow.model.Source serializeSplitToClo "Ignoring negative estimated size {} produced by source {}", estimatedSize, source); } } catch (Exception e) { - LOG.warn("Size estimation of the source failed: " + source, e); + LOG.warn("Size estimation of the source failed: {}", source, e); } cloudSource.setMetadata(metadata); return cloudSource; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerPipelineOptionsFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerPipelineOptionsFactory.java index c9df5d96eb6c..382e997414e6 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerPipelineOptionsFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerPipelineOptionsFactory.java @@ -58,10 +58,10 @@ public static T createFromSystemPropert objectMapper.readValue(serializedOptions, PipelineOptions.class).as(harnessOptionsClass); } else if (System.getProperties().containsKey("sdk_pipeline_options_file")) { String filePath = System.getProperty("sdk_pipeline_options_file"); - LOG.info("Loading pipeline options from " + filePath); + LOG.info("Loading pipeline options from {}", filePath); String serializedOptions = new String(Files.readAllBytes(Paths.get(filePath)), StandardCharsets.UTF_8); - LOG.info("Worker harness starting with: " + serializedOptions); + LOG.info("Worker harness starting with: {}", serializedOptions); options = objectMapper.readValue(serializedOptions, PipelineOptions.class).as(harnessOptionsClass); } else { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java index 7dec8d1ed6ce..db11e658634d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java @@ -625,7 +625,7 @@ private void recordStreamRestart(Status status) { String restartReason = "Stream completed successfully but did not complete requested operations, " + "recreating"; - logger.warn(restartReason); + logger.warn("{}", restartReason); debugMetrics.recordRestartReason(restartReason); } else { int currentErrorCount = debugMetrics.incrementAndGetErrors(); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java index bf060bd6acfe..08f2778e5aca 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java @@ -155,7 +155,7 @@ public void onNext(T value) { totalSecondsWaited += waitSeconds; if (totalSecondsWaited > deadlineSeconds) { String errorMessage = constructStreamCancelledErrorMessage(totalSecondsWaited); - LOG.error(errorMessage); + LOG.error("{}", errorMessage); throw new WindmillRpcException(errorMessage, e); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkScheduler.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkScheduler.java index bb936831d6ea..bfcbb33087a5 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkScheduler.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkScheduler.java @@ -332,7 +332,7 @@ private Windmill.WorkItemCommitRequest validateCommitRequestSize( KeyCommitTooLargeException e = KeyCommitTooLargeException.causedBy(computationId, byteLimit, commitRequest); failureTracker.trackFailure(computationId, workItem, e); - LOG.error(e.toString()); + LOG.error("{}", e.toString()); // Drop the current request in favor of a new, minimal one requesting truncation. // Messages, timers, counters, and other commit content will not be used by the service @@ -417,7 +417,7 @@ private ExecuteWorkResult executeWork( computationState.sourceBytesProcessCounterName()); outputBuilder.setSourceBytesProcessed(sourceBytesProcessed); } catch (Exception e) { - LOG.error(e.toString()); + LOG.error("{}", e.toString()); } commitFinalizer.cacheCommitFinalizers(computationWorkExecutor.context().flushState()); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java index d417d7d3417c..d68f9a1e2167 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java @@ -613,7 +613,7 @@ private void flushResponse() { responseObserver.onNext(responseBuilder.build()); } catch (Exception e) { // Stream is already closed. - LOG.warn(Arrays.toString(e.getStackTrace())); + LOG.warn("{}", Arrays.toString(e.getStackTrace())); } responseBuilder.clear(); } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java index e532355d8568..57d462670ece 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java @@ -108,7 +108,7 @@ public void createDatabase(SqlIdentifier databaseIdentifier, boolean ifNotExists if (alreadyExists) { String message = format("Database '%s' already exists.", name); if (ifNotExists || name.equals(DEFAULT)) { - LOG.info(message); + LOG.info("Database '{}' already exists.", name); } else { throw SqlUtil.newContextException( databaseIdentifier.getParserPosition(), RESOURCE.internal(message)); diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java index 044e75574391..3c1c2579dedf 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java @@ -326,7 +326,7 @@ public void processElement( Schema schema = BeamSqlRelUtils.getErrorRowSchema(row.getSchema()); Row errorRow = toBeamRow(Arrays.asList(row.getValues(), e.getCause().getMessage()), schema, true); - LOG.error("CalcFn failed to evaluate: " + processElementBlock, e.getCause()); + LOG.error("CalcFn failed to evaluate: {}", processElementBlock, e.getCause()); multiOutputReceiver.get(errors).output(errorRow); } else { throw new RuntimeException( diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java index c54ab14ba8d8..8826eef6d0d7 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java @@ -203,7 +203,7 @@ private static PipelineResult limitRun( break; } } catch (IOException e) { - LOG.warn(e.toString()); + LOG.warn("{}", e.toString()); break; } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java index bc10d6b99a95..202d46978b24 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java @@ -256,7 +256,7 @@ private static BeamTableStatistics getRowCountFromBQ(PipelineOptions o, String b return BeamTableStatistics.createBoundedTableStatistics(rowCount.doubleValue()); } catch (IOException | InterruptedException e) { - LOG.warn("Could not get the row count for the table " + bqLocation, e); + LOG.warn("Could not get the row count for the table {}", bqLocation, e); } return BeamTableStatistics.BOUNDED_UNKNOWN; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTable.java index 8752ec1affe4..a5904ca116e1 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTable.java @@ -201,7 +201,7 @@ public BeamTableStatistics getTableStatistics(PipelineOptions options) { BeamTableStatistics.createUnboundedTableStatistics( this.computeRate(numberOfRecordsForRate)); } catch (Exception e) { - LOG.warn("Could not get the row count for the topics " + getTopics(), e); + LOG.warn("Could not get the row count for the topics {}", getTopics(), e); rowCountStatistics = BeamTableStatistics.UNBOUNDED_UNKNOWN; } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTable.java index 701c9c4e6dbc..89a61f6cd426 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTable.java @@ -93,7 +93,7 @@ private static BeamTableStatistics getTextRowEstimate( Double rows = textRowCountEstimator.estimateRowCount(options); return BeamTableStatistics.createBoundedTableStatistics(rows); } catch (IOException | TextRowCountEstimator.NoEstimationException e) { - LOG.warn("Could not get the row count for the text table " + filePattern, e); + LOG.warn("Could not get the row count for the text table {}", filePattern, e); } return BeamTableStatistics.BOUNDED_UNKNOWN; } diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkUtils.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkUtils.java index 2a536c4c6c7c..9533f2585adf 100644 --- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkUtils.java +++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkUtils.java @@ -328,7 +328,7 @@ public int stepLengthSec(int ratePeriodSec) { @FormatMethod public static void info(@FormatString String format, Object... args) { if (LOG_INFO) { - LOG.info(String.format(format, args)); + LOG.info("{}", String.format(format, args)); if (LOG_TO_CONSOLE) { System.out.println(String.format(format, args)); } diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query10.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query10.java index c894e55a6c09..b99e16d25198 100644 --- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query10.java +++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query10.java @@ -243,15 +243,14 @@ public void processElement(ProcessContext c, BoundedWindow window) { } String shard = c.element().getKey(); LOG.debug( - String.format( - "%s with timestamp %s has %d actually late and %d on-time " - + "elements in pane %s for window %s", - shard, - c.timestamp(), - numLate, - numOnTime, - c.pane(), - window.maxTimestamp())); + "{} with timestamp {} has {} actually late and {} on-time " + + "elements in pane {} for window {}", + shard, + c.timestamp(), + numLate, + numOnTime, + c.pane(), + window.maxTimestamp()); if (c.pane().getTiming() == PaneInfo.Timing.LATE) { if (numLate == 0) { LOG.error("ERROR! No late events in late pane for {}", shard); @@ -292,9 +291,11 @@ public void processElement(ProcessContext c, BoundedWindow window) GcsOptions options = c.getPipelineOptions().as(GcsOptions.class); OutputFile outputFile = outputFileFor(window, shard, c.pane()); LOG.debug( - String.format( - "Writing %s with record timestamp %s, window timestamp %s, pane %s", - shard, c.timestamp(), window.maxTimestamp(), c.pane())); + "Writing {} with record timestamp {}, window timestamp {}, pane {}", + shard, + c.timestamp(), + window.maxTimestamp(), + c.pane()); if (outputFile.filename != null) { LOG.info("Beginning write to '{}'", outputFile.filename); int n = 0; From 4303b932fe852b9fec613e11802d50854e44debb Mon Sep 17 00:00:00 2001 From: Radek Stankiewicz Date: Tue, 10 Mar 2026 16:39:33 +0100 Subject: [PATCH 4/4] revert log format due to usage in tests --- .../solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java index 5537b015be07..715f0634c74a 100644 --- a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java @@ -642,6 +642,7 @@ public void finishBundle(FinishBundleContext context) throws Exception { } // Flushes the batch, implementing the retry mechanism as configured in the spec. + @SuppressWarnings("Slf4jFormatShouldBeConst") private void flushBatch() throws IOException, InterruptedException { if (batch.isEmpty()) { return; @@ -677,7 +678,7 @@ private void flushBatch() throws IOException, InterruptedException { } else { // Note: this used in test cases to verify behavior - LOG.warn("Error writing to Solr. Retry attempt {}", attempt, exception); + LOG.warn(String.format(RETRY_ATTEMPT_LOG, attempt), exception); } } }