Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 5 additions & 0 deletions core/trino-main/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -212,6 +212,11 @@
<artifactId>jsr305</artifactId>
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Don't you need to add @FormatString to airlift logger methods?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There's already one, and that's how we can find all these issues here:

    @FormatMethod
    public void debug(String format, Object... args)
    {
        ...
    }

</dependency>

<dependency>
<groupId>com.google.errorprone</groupId>
<artifactId>error_prone_annotations</artifactId>
</dependency>

<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
Expand Down
2 changes: 2 additions & 0 deletions core/trino-main/src/main/java/io/trino/operator/Driver.java
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.SettableFuture;
import com.google.errorprone.annotations.FormatMethod;
import io.airlift.log.Logger;
import io.airlift.units.Duration;
import io.trino.execution.ScheduledSplit;
Expand Down Expand Up @@ -609,6 +610,7 @@ private Optional<ListenableFuture<Void>> getBlockedFuture(Operator operator)
return Optional.empty();
}

@FormatMethod
private static Throwable addSuppressedException(Throwable inFlightException, Throwable newException, String message, Object... args)
{
if (newException instanceof Error) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
import com.google.common.collect.ImmutableList;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.SettableFuture;
import com.google.errorprone.annotations.FormatMethod;
import io.airlift.log.Logger;
import io.airlift.units.DataSize;
import io.airlift.units.Duration;
Expand Down Expand Up @@ -555,6 +556,7 @@ private void closeOperators(int lastOperatorIndex)
}
}

@FormatMethod
private static Throwable handleOperatorCloseError(Throwable inFlightException, Throwable newException, String message, Object... args)
{
if (newException instanceof Error) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
import com.google.common.collect.ObjectArrays;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.errorprone.annotations.FormatMethod;
import io.airlift.event.client.ServiceUnavailableException;
import io.airlift.log.Logger;
import io.airlift.units.Duration;
Expand Down Expand Up @@ -110,10 +111,10 @@ public void requestFailed(Throwable reason)
// log failure message
if (isExpectedError(reason)) {
// don't print a stack for a known errors
log.warn("Error " + jobDescription + " %s: %s: %s", taskId, reason.getMessage(), taskUri);
log.warn("Error %s %s: %s: %s", jobDescription, taskId, reason.getMessage(), taskUri);
}
else {
log.warn(reason, "Error " + jobDescription + " %s: %s", taskId, taskUri);
log.warn(reason, "Error %s %s: %s", jobDescription, taskId, taskUri);
}

// remember the first 10 errors
Expand All @@ -138,6 +139,8 @@ public void requestFailed(Throwable reason)
}
}

@FormatMethod
@SuppressWarnings("FormatStringAnnotation") // we manipulate the format string and there's no way to make Error Prone accept the result
static void logError(Throwable t, String format, Object... args)
{
if (isExpectedError(t)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -119,7 +119,7 @@ private Optional<Map<String, Object>> getAccessToken(ContainerRequestContext req
return service.convertTokenToClaims(accessToken.get());
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is nice. What validates (does the checkstyle for) this?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is Error Prone, though somewhat by accident. It sees that there's no placeholder for the exception in the format string.

}
catch (JwtException | IllegalArgumentException e) {
LOG.debug("Unable to parse JWT token: " + e.getMessage(), e);
LOG.debug(e, "Unable to parse JWT token");
}
}
return Optional.empty();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -364,7 +364,7 @@ private void setLocalityGroups(Map<String, Object> tableProperties, AccumuloTabl
}

Map<String, Set<Text>> localityGroups = localityGroupsBuilder.build();
LOG.debug("Setting locality groups: {}", localityGroups);
LOG.debug("Setting locality groups: %s", localityGroups);
tableManager.setLocalityGroups(table.getFullTableName(), localityGroups);
}

Expand Down Expand Up @@ -851,7 +851,7 @@ private Optional<String> getTabletLocation(String table, Key key)
// Swallow this exception so the query does not fail due to being unable
// to locate the tablet server for the provided Key.
// This is purely an optimization, but we will want to log the error.
LOG.error("Failed to get tablet location, returning dummy location", e);
LOG.error(e, "Failed to get tablet location, returning dummy location");
return Optional.empty();
}
}
Expand Down Expand Up @@ -882,7 +882,7 @@ private Optional<String> getDefaultTabletLocation(String fulltable)
catch (Exception e) {
// Swallow this exception so the query does not fail due to being unable to locate the tablet server for the default tablet.
// This is purely an optimization, but we will want to log the error.
LOG.error("Failed to get tablet location, returning dummy location", e);
LOG.error(e, "Failed to get tablet location, returning dummy location");
return Optional.empty();
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -270,7 +270,7 @@ private boolean getRangesWithMetrics(
// of rows
long numEntries = indexRanges.size();
double ratio = (double) numEntries / (double) numRows;
LOG.debug("Use of index would scan %d of %d rows, ratio %s. Threshold %2f, Using for table? %b", numEntries, numRows, ratio, threshold, ratio < threshold, table);
LOG.debug("Use of index would scan %d of %d rows, ratio %s. Threshold %2f, Using for table %s? %b", numEntries, numRows, ratio, threshold, table, ratio < threshold);

// If the percentage of scanned rows, the ratio, less than the configured threshold
if (ratio < threshold) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -128,8 +128,7 @@ private static void copyTable(
break;
}

LOG.info("Running import for %s", target, sql);
LOG.info("%s", sql);
LOG.info("Running import for %s%n%s", target, sql);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
LOG.info("Running import for %s%n%s", target, sql);
LOG.info("Running import for %s", target);
LOG.info("Query %s", sql);

Maybe

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That would be a more straightforward transformation, sure, but I think that if they are two separate logs then they may get separated in the output if there are concurrent threads. Perhaps this merging can be done in a separate commit, though.

long start = System.nanoTime();
long rows = queryRunner.execute(session, sql).getUpdateCount().getAsLong();
LOG.info("Imported %s rows for %s in %s", rows, target, nanosSince(start));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -316,7 +316,7 @@ public void close()
}
catch (Exception e) {
// ignore
LOG.debug("Error clearing scroll", e);
LOG.debug(e, "Error clearing scroll");
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we handle the logger based changes in a separate commit ? The one where exception should be first and followed by message

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I split it into four commits. Thanks!

}
}
}
Expand Down
5 changes: 5 additions & 0 deletions plugin/trino-hive/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -169,6 +169,11 @@
<optional>true</optional>
</dependency>

<dependency>
<groupId>com.google.errorprone</groupId>
<artifactId>error_prone_annotations</artifactId>
</dependency>

<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.errorprone.annotations.FormatMethod;
import io.airlift.log.Logger;
import io.airlift.units.Duration;
import io.trino.plugin.hive.HdfsEnvironment;
Expand Down Expand Up @@ -2308,6 +2309,7 @@ private static boolean isSameOrParent(Path parent, Path child)
return parent.equals(child);
}

@FormatMethod
private void logCleanupFailure(String format, Object... args)
{
if (throwOnCleanupFailure) {
Expand All @@ -2316,6 +2318,7 @@ private void logCleanupFailure(String format, Object... args)
log.warn(format, args);
}

@FormatMethod
private void logCleanupFailure(Throwable t, String format, Object... args)
{
if (throwOnCleanupFailure) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ public void write(ParquetHiveRecord record)
}
catch (RuntimeException e) {
String errorMessage = "Parquet record is malformed: " + e.getMessage();
log.error(errorMessage, e);
log.error(e, errorMessage);
throw new RuntimeException(errorMessage, e);
}
recordConsumer.endMessage();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -145,7 +145,7 @@ public void postInit(DistributedQueryRunner queryRunner)
long start = System.nanoTime();
log.info("Running import for %s", table.getTableName());
queryRunner.execute(format("INSERT INTO %1$s SELECT * FROM tpch.tiny.%1$s", table.getTableName()));
log.info("Imported %s in %s", 0, table.getTableName(), nanosSince(start).convertToMostSuccinctTimeUnit());
log.info("Imported %s in %s", table.getTableName(), nanosSince(start).convertToMostSuccinctTimeUnit());
}
log.info("Loading complete in %s", nanosSince(startTime).toString(SECONDS));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -198,7 +198,7 @@ private void updateTablesFromS3()
log.info("Put table description into the map from %s", summary.getKey());
}
catch (IOException iox) {
log.error("Problem reading input stream from object.", iox);
log.error(iox, "Problem reading input stream from object.");
throw new RuntimeException(iox);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -231,7 +231,7 @@ public KuduTable openTable(SchemaTableName schemaTableName)
return client.openTable(rawName);
}
catch (KuduException e) {
log.debug("Error on doOpenTable: " + e, e);
log.debug(e, "Error on doOpenTable");
if (!listSchemaNames().contains(schemaTableName.getSchemaName())) {
throw new SchemaNotFoundException(schemaTableName.getSchemaName());
}
Expand Down
1 change: 1 addition & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -1721,6 +1721,7 @@
-Xep:EqualsIncompatibleType:ERROR
-Xep:FallThrough:ERROR
-Xep:FormatString:ERROR
-Xep:FormatStringAnnotation:ERROR
-Xep:GetClassOnAnnotation:ERROR
-Xep:GetClassOnClass:ERROR
-Xep:IdentityBinaryExpression:ERROR
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,7 @@ private void queryWithWhere(QueryRunner trinoExecutor, Session session, List<Typ
checkResults(expectedTypes, expectedResults, filteredRows);
}
catch (RuntimeException e) {
log.error("Exception caught during query with merged WHERE clause, querying one column at a time", e);
log.error(e, "Exception caught during query with merged WHERE clause, querying one column at a time");
debugTypes(trinoExecutor, session, expectedTypes, expectedResults, testTable);
}
}
Expand Down