diff --git a/core/src/main/java/org/apache/druid/common/config/ConfigManager.java b/core/src/main/java/org/apache/druid/common/config/ConfigManager.java index 578a7f5ba974..2b4145496963 100644 --- a/core/src/main/java/org/apache/druid/common/config/ConfigManager.java +++ b/core/src/main/java/org/apache/druid/common/config/ConfigManager.java @@ -124,7 +124,7 @@ public AtomicReference watchConfig(final String key, final ConfigSerde ConfigHolder holder = watchedConfigs.get(key); if (holder == null) { try { - log.info("Creating watch for key[%s]", key); + log.debug("Creating watch for key[%s]", key); holder = exec.submit( new Callable>() diff --git a/core/src/main/java/org/apache/druid/common/guava/ThreadRenamingCallable.java b/core/src/main/java/org/apache/druid/common/guava/ThreadRenamingCallable.java index fabbd23f8314..1fcba2309ea7 100644 --- a/core/src/main/java/org/apache/druid/common/guava/ThreadRenamingCallable.java +++ b/core/src/main/java/org/apache/druid/common/guava/ThreadRenamingCallable.java @@ -21,8 +21,6 @@ import java.util.concurrent.Callable; -/** - */ public abstract class ThreadRenamingCallable implements Callable { private final String name; @@ -48,5 +46,5 @@ public final T call() throws Exception } } - public abstract T doCall() throws Exception; + public abstract T doCall(); } diff --git a/core/src/main/java/org/apache/druid/guice/Jerseys.java b/core/src/main/java/org/apache/druid/guice/Jerseys.java index 5a9ab5b0f0c6..2054f8a1df6b 100644 --- a/core/src/main/java/org/apache/druid/guice/Jerseys.java +++ b/core/src/main/java/org/apache/druid/guice/Jerseys.java @@ -26,8 +26,6 @@ import org.apache.druid.guice.annotations.PublicApi; import org.apache.druid.java.util.common.logger.Logger; -/** - */ @PublicApi public class Jerseys { @@ -35,8 +33,8 @@ public class Jerseys public static void addResource(Binder binder, Class resourceClazz) { - LOG.info("Adding Jersey resource: " + resourceClazz.getName()); - Multibinder.newSetBinder(binder, new TypeLiteral>(){}, JSR311Resource.class) + LOG.debug("Adding Jersey resource: " + resourceClazz.getName()); + Multibinder.newSetBinder(binder, new TypeLiteral>() {}, JSR311Resource.class) .addBinding() .toInstance(resourceClazz); } diff --git a/core/src/main/java/org/apache/druid/guice/JsonConfigurator.java b/core/src/main/java/org/apache/druid/guice/JsonConfigurator.java index a39b189c104e..d0db1e241ae2 100644 --- a/core/src/main/java/org/apache/druid/guice/JsonConfigurator.java +++ b/core/src/main/java/org/apache/druid/guice/JsonConfigurator.java @@ -187,7 +187,7 @@ public Message apply(String input) ); } - log.info("Loaded class[%s] from props[%s] as [%s]", clazz, propertyBase, config); + log.debug("Loaded class[%s] from props[%s] as [%s]", clazz, propertyBase, config); return config; } diff --git a/core/src/main/java/org/apache/druid/java/util/common/io/smoosh/FileSmoosher.java b/core/src/main/java/org/apache/druid/java/util/common/io/smoosh/FileSmoosher.java index f2ed8d383e37..f8e5d8374450 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/io/smoosh/FileSmoosher.java +++ b/core/src/main/java/org/apache/druid/java/util/common/io/smoosh/FileSmoosher.java @@ -369,7 +369,8 @@ public void close() throws IOException File metaFile = metaFile(baseDir); - try (Writer out = new BufferedWriter(new OutputStreamWriter(new FileOutputStream(metaFile), StandardCharsets.UTF_8))) { + try (Writer out = + new BufferedWriter(new OutputStreamWriter(new FileOutputStream(metaFile), StandardCharsets.UTF_8))) { out.write(StringUtils.format("v1,%d,%d", maxChunkSize, outFiles.size())); out.write("\n"); @@ -469,7 +470,11 @@ public boolean isOpen() public void close() throws IOException { closer.close(); - FileSmoosher.LOG.info("Created smoosh file [%s] of size [%s] bytes.", outFile.getAbsolutePath(), outFile.length()); + FileSmoosher.LOG.debug( + "Created smoosh file [%s] of size [%s] bytes.", + outFile.getAbsolutePath(), + outFile.length() + ); } } } diff --git a/core/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java b/core/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java index 413f4ba13bbe..8082ebbec0d0 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java +++ b/core/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java @@ -438,7 +438,7 @@ public void start() throws Exception } } if (doStart) { - log.info("Invoking start method[%s] on object[%s].", method, o); + log.debug("Invoking start method[%s] on object[%s].", method, o); method.invoke(o); } } @@ -456,7 +456,7 @@ public void stop() } } if (doStop) { - log.info("Invoking stop method[%s] on object[%s].", method, o); + log.debug("Invoking stop method[%s] on object[%s].", method, o); try { method.invoke(o); } diff --git a/core/src/main/java/org/apache/druid/java/util/common/logger/Logger.java b/core/src/main/java/org/apache/druid/java/util/common/logger/Logger.java index 569537a18a88..8aa0e1cdf7a7 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/logger/Logger.java +++ b/core/src/main/java/org/apache/druid/java/util/common/logger/Logger.java @@ -22,20 +22,32 @@ import org.apache.druid.java.util.common.StringUtils; import org.slf4j.LoggerFactory; -/** - */ +import java.util.function.BiConsumer; + public class Logger { private final org.slf4j.Logger log; + private final boolean stackTraces; public Logger(String name) { - log = LoggerFactory.getLogger(name); + this(LoggerFactory.getLogger(name), true); } public Logger(Class clazz) { - log = LoggerFactory.getLogger(clazz); + this(LoggerFactory.getLogger(clazz), true); + } + + protected Logger(org.slf4j.Logger log, boolean stackTraces) + { + this.log = log; + this.stackTraces = stackTraces; + } + + protected org.slf4j.Logger getSlf4jLogger() + { + return log; } @Override @@ -44,6 +56,15 @@ public String toString() return StringUtils.format("Logger{name=[%s], class[%s]}", log.getName(), log.getClass()); } + /** + * Create a copy of this Logger that does not log exception stack traces, unless the log level is DEBUG or lower. + * Useful for writing code like: {@code log.noStackTrace().warn(e, "Something happened.");} + */ + public Logger noStackTrace() + { + return new Logger(log, false); + } + public void trace(String message, Object... formatArgs) { if (log.isTraceEnabled()) { @@ -61,7 +82,7 @@ public void debug(String message, Object... formatArgs) public void debug(Throwable t, String message, Object... formatArgs) { if (log.isDebugEnabled()) { - log.debug(StringUtils.nonStrictFormat(message, formatArgs), t); + logException(log::debug, t, StringUtils.nonStrictFormat(message, formatArgs)); } } @@ -75,7 +96,7 @@ public void info(String message, Object... formatArgs) public void info(Throwable t, String message, Object... formatArgs) { if (log.isInfoEnabled()) { - log.info(StringUtils.nonStrictFormat(message, formatArgs), t); + logException(log::info, t, StringUtils.nonStrictFormat(message, formatArgs)); } } @@ -88,7 +109,7 @@ public void info(Throwable t, String message, Object... formatArgs) @Deprecated public void warn(String message, Throwable t) { - log.warn(message, t); + warn(t, message); } public void warn(String message, Object... formatArgs) @@ -98,7 +119,7 @@ public void warn(String message, Object... formatArgs) public void warn(Throwable t, String message, Object... formatArgs) { - log.warn(StringUtils.nonStrictFormat(message, formatArgs), t); + logException(log::warn, t, StringUtils.nonStrictFormat(message, formatArgs)); } public void error(String message, Object... formatArgs) @@ -115,12 +136,12 @@ public void error(String message, Object... formatArgs) @Deprecated public void error(String message, Throwable t) { - log.error(message, t); + error(t, message); } public void error(Throwable t, String message, Object... formatArgs) { - log.error(StringUtils.nonStrictFormat(message, formatArgs), t); + logException(log::error, t, StringUtils.nonStrictFormat(message, formatArgs)); } public void assertionError(String message, Object... formatArgs) @@ -152,4 +173,17 @@ public boolean isInfoEnabled() { return log.isInfoEnabled(); } + + private void logException(BiConsumer fn, Throwable t, String message) + { + if (stackTraces || log.isDebugEnabled()) { + fn.accept(message, t); + } else { + if (message.isEmpty()) { + fn.accept(t.toString(), null); + } else { + fn.accept(StringUtils.nonStrictFormat("%s (%s)", message, t.toString()), null); + } + } + } } diff --git a/core/src/main/java/org/apache/druid/java/util/emitter/EmittingLogger.java b/core/src/main/java/org/apache/druid/java/util/emitter/EmittingLogger.java index ef5d47a1f4b5..0f6bb8bf3482 100644 --- a/core/src/main/java/org/apache/druid/java/util/emitter/EmittingLogger.java +++ b/core/src/main/java/org/apache/druid/java/util/emitter/EmittingLogger.java @@ -53,10 +53,21 @@ public static void registerEmitter(ServiceEmitter emitter) public EmittingLogger(Class clazz) { super(clazz); - this.className = clazz.getName(); } + private EmittingLogger(org.slf4j.Logger log, boolean stackTraces) + { + super(log, stackTraces); + this.className = log.getName(); + } + + @Override + public EmittingLogger noStackTrace() + { + return new EmittingLogger(getSlf4jLogger(), false); + } + public AlertBuilder makeAlert(String message, Object... objects) { return makeAlert(null, message, objects); diff --git a/core/src/main/java/org/apache/druid/java/util/emitter/core/NoopEmitter.java b/core/src/main/java/org/apache/druid/java/util/emitter/core/NoopEmitter.java index 1564eb2b0588..072f168e970b 100644 --- a/core/src/main/java/org/apache/druid/java/util/emitter/core/NoopEmitter.java +++ b/core/src/main/java/org/apache/druid/java/util/emitter/core/NoopEmitter.java @@ -46,4 +46,10 @@ public void close() { // Do nothing } + + @Override + public String toString() + { + return "NoopEmitter{}"; + } } diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/pool/ResourcePool.java b/core/src/main/java/org/apache/druid/java/util/http/client/pool/ResourcePool.java index 00dfab758487..d4354d82e46b 100644 --- a/core/src/main/java/org/apache/druid/java/util/http/client/pool/ResourcePool.java +++ b/core/src/main/java/org/apache/druid/java/util/http/client/pool/ResourcePool.java @@ -248,22 +248,22 @@ void giveBack(V object) if (resourceHolderList.size() >= maxSize) { if (holderListContains(object)) { log.warn( + new Exception("Exception for stacktrace"), StringUtils.format( "Returning object[%s] at key[%s] that has already been returned!? Skipping", object, key - ), - new Exception("Exception for stacktrace") + ) ); } else { log.warn( + new Exception("Exception for stacktrace"), StringUtils.format( "Returning object[%s] at key[%s] even though we already have all that we can hold[%s]!? Skipping", object, key, resourceHolderList - ), - new Exception("Exception for stacktrace") + ) ); } return; diff --git a/core/src/main/java/org/apache/druid/segment/SegmentUtils.java b/core/src/main/java/org/apache/druid/segment/SegmentUtils.java index 10d673002811..a6d98a47566f 100644 --- a/core/src/main/java/org/apache/druid/segment/SegmentUtils.java +++ b/core/src/main/java/org/apache/druid/segment/SegmentUtils.java @@ -34,8 +34,10 @@ import java.io.IOException; import java.io.InputStream; import java.nio.charset.StandardCharsets; +import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.stream.Collectors; /** * Utility methods useful for implementing deep storage extensions. @@ -75,6 +77,19 @@ public static int getVersionFromDir(File inDir) throws IOException throw new IOE("Invalid segment dir [%s]. Can't find either of version.bin or index.drd.", inDir); } + /** + * Returns a String with identifiers of "segments" comma-separated. Useful for log messages. Not useful for anything + * else, because this doesn't take special effort to escape commas that occur in identifiers (not common, but could + * potentially occur in a datasource name). + */ + public static String commaSeparateIdentifiers(final Collection segments) + { + return segments + .stream() + .map(segment -> segment.getId().toString()) + .collect(Collectors.joining(", ")); + } + private SegmentUtils() { } diff --git a/core/src/main/java/org/apache/druid/timeline/DataSegment.java b/core/src/main/java/org/apache/druid/timeline/DataSegment.java index eec03a26e589..07c6ae7f689c 100644 --- a/core/src/main/java/org/apache/druid/timeline/DataSegment.java +++ b/core/src/main/java/org/apache/druid/timeline/DataSegment.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.annotation.JsonDeserialize; import com.fasterxml.jackson.databind.annotation.JsonSerialize; @@ -70,8 +71,13 @@ public static class PruneSpecsHolder @VisibleForTesting public static final PruneSpecsHolder DEFAULT = new PruneSpecsHolder(); - @Inject(optional = true) @PruneLoadSpec boolean pruneLoadSpec = false; - @Inject(optional = true) @PruneLastCompactionState boolean pruneLastCompactionState = false; + @Inject(optional = true) + @PruneLoadSpec + boolean pruneLoadSpec = false; + + @Inject(optional = true) + @PruneLastCompactionState + boolean pruneLastCompactionState = false; } private static final Interner STRING_INTERNER = Interners.newWeakInterner(); @@ -312,6 +318,7 @@ public ShardSpec getShardSpec() @Nullable @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) public CompactionState getLastCompactionState() { return lastCompactionState; diff --git a/core/src/main/java/org/apache/druid/utils/CompressionUtils.java b/core/src/main/java/org/apache/druid/utils/CompressionUtils.java index 5ea6ff526e04..76b75a82f913 100644 --- a/core/src/main/java/org/apache/druid/utils/CompressionUtils.java +++ b/core/src/main/java/org/apache/druid/utils/CompressionUtils.java @@ -139,7 +139,7 @@ public static long zip(File directory, OutputStream out) throws IOException long totalSize = 0; for (File file : directory.listFiles()) { - log.info("Adding file[%s] with size[%,d]. Total size so far[%,d]", file, file.length(), totalSize); + log.debug("Adding file[%s] with size[%,d]. Total size so far[%,d]", file, file.length(), totalSize); if (file.length() > Integer.MAX_VALUE) { zipOut.finish(); throw new IOE("file[%s] too large [%,d]", file, file.length()); diff --git a/core/src/test/java/org/apache/druid/java/util/common/logger/LoggerTest.java b/core/src/test/java/org/apache/druid/java/util/common/logger/LoggerTest.java index d5d55d98a420..086df91281c6 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/logger/LoggerTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/logger/LoggerTest.java @@ -23,12 +23,13 @@ public class LoggerTest { + private final Logger log = new Logger(LoggerTest.class); + @SuppressWarnings("MalformedFormatString") @Test public void testLogWithCrazyMessages() { final String message = "this % might %d kill %*.s the %s parser"; - final Logger log = new Logger(LoggerTest.class); log.warn(message); } @@ -36,10 +37,17 @@ public void testLogWithCrazyMessages() @Test public void testLegacyLogging() { - final Logger log = new Logger(LoggerTest.class); final Throwable throwable = new Throwable(); // These should show up in an IDE as deprecated, but shouldn't actually fail. log.error("foo", throwable); log.warn("foo", throwable); } + + @Test + public void testErrorExceptions() + { + log.noStackTrace().error(new RuntimeException("beep"), "Feel the hatred of %d years", 10000); + log.noStackTrace().error(new RuntimeException("beep"), ""); + log.error(new RuntimeException("beep"), "An exception"); + } } diff --git a/core/src/test/java/org/apache/druid/timeline/SegmentWithOvershadowedStatusTest.java b/core/src/test/java/org/apache/druid/timeline/SegmentWithOvershadowedStatusTest.java index e5cb4faafcfb..cc1898f8acdf 100644 --- a/core/src/test/java/org/apache/druid/timeline/SegmentWithOvershadowedStatusTest.java +++ b/core/src/test/java/org/apache/druid/timeline/SegmentWithOvershadowedStatusTest.java @@ -81,7 +81,7 @@ public void testUnwrappedSegmentWithOvershadowedStatusDeserialization() throws E JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - Assert.assertEquals(12, objectMap.size()); + Assert.assertEquals(11, objectMap.size()); Assert.assertEquals("something", objectMap.get("dataSource")); Assert.assertEquals(interval.toString(), objectMap.get("interval")); Assert.assertEquals("1", objectMap.get("version")); diff --git a/examples/conf/druid/cluster/_common/log4j2.xml b/examples/conf/druid/cluster/_common/log4j2.xml index bd2b3a687cc0..2f88d04b56af 100644 --- a/examples/conf/druid/cluster/_common/log4j2.xml +++ b/examples/conf/druid/cluster/_common/log4j2.xml @@ -19,15 +19,42 @@ --> - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/examples/conf/druid/single-server/large/_common/log4j2.xml b/examples/conf/druid/single-server/large/_common/log4j2.xml index bd2b3a687cc0..2f88d04b56af 100644 --- a/examples/conf/druid/single-server/large/_common/log4j2.xml +++ b/examples/conf/druid/single-server/large/_common/log4j2.xml @@ -19,15 +19,42 @@ --> - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/examples/conf/druid/single-server/medium/_common/log4j2.xml b/examples/conf/druid/single-server/medium/_common/log4j2.xml index bd2b3a687cc0..2f88d04b56af 100644 --- a/examples/conf/druid/single-server/medium/_common/log4j2.xml +++ b/examples/conf/druid/single-server/medium/_common/log4j2.xml @@ -19,15 +19,42 @@ --> - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/examples/conf/druid/single-server/micro-quickstart/_common/log4j2.xml b/examples/conf/druid/single-server/micro-quickstart/_common/log4j2.xml index bd2b3a687cc0..2f88d04b56af 100644 --- a/examples/conf/druid/single-server/micro-quickstart/_common/log4j2.xml +++ b/examples/conf/druid/single-server/micro-quickstart/_common/log4j2.xml @@ -19,15 +19,42 @@ --> - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/examples/conf/druid/single-server/nano-quickstart/_common/log4j2.xml b/examples/conf/druid/single-server/nano-quickstart/_common/log4j2.xml index bd2b3a687cc0..2f88d04b56af 100644 --- a/examples/conf/druid/single-server/nano-quickstart/_common/log4j2.xml +++ b/examples/conf/druid/single-server/nano-quickstart/_common/log4j2.xml @@ -19,15 +19,42 @@ --> - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/examples/conf/druid/single-server/small/_common/log4j2.xml b/examples/conf/druid/single-server/small/_common/log4j2.xml index bd2b3a687cc0..2f88d04b56af 100644 --- a/examples/conf/druid/single-server/small/_common/log4j2.xml +++ b/examples/conf/druid/single-server/small/_common/log4j2.xml @@ -19,15 +19,42 @@ --> - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/examples/conf/druid/single-server/xlarge/_common/log4j2.xml b/examples/conf/druid/single-server/xlarge/_common/log4j2.xml index bd2b3a687cc0..2f88d04b56af 100644 --- a/examples/conf/druid/single-server/xlarge/_common/log4j2.xml +++ b/examples/conf/druid/single-server/xlarge/_common/log4j2.xml @@ -19,15 +19,42 @@ --> - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPusher.java b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPusher.java index 50d0927d213c..42fe23f94318 100644 --- a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPusher.java +++ b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPusher.java @@ -53,8 +53,6 @@ public CloudFilesDataSegmentPusher( String container = this.config.getContainer(); this.objectApi = new CloudFilesObjectApiProxy(cloudFilesApi, region, container); this.jsonMapper = jsonMapper; - - log.info("Configured CloudFiles as deep storage"); } @Override diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleDataSegmentPusher.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleDataSegmentPusher.java index 896209adc289..35c763f2be3a 100644 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleDataSegmentPusher.java +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleDataSegmentPusher.java @@ -42,7 +42,7 @@ public class GoogleDataSegmentPusher implements DataSegmentPusher { - private static final Logger LOG = new Logger(GoogleDataSegmentPusher.class); + private static final Logger log = new Logger(GoogleDataSegmentPusher.class); private final GoogleStorage storage; private final GoogleAccountConfig config; @@ -55,8 +55,6 @@ public GoogleDataSegmentPusher( { this.storage = storage; this.config = config; - - LOG.info("Configured Google as deep storage"); } @Deprecated @@ -81,7 +79,7 @@ public List getAllowedPropertyPrefixesForHadoop() public void insert(final File file, final String contentType, final String path) throws IOException { - LOG.info("Inserting [%s] to [%s]", file, path); + log.debug("Inserting [%s] to [%s]", file, path); try { RetryUtils.retry( (RetryUtils.Task) () -> { @@ -105,7 +103,7 @@ public void insert(final File file, final String contentType, final String path) public DataSegment push(final File indexFilesDir, final DataSegment segment, final boolean useUniquePath) throws IOException { - LOG.info("Uploading [%s] to Google.", indexFilesDir); + log.debug("Uploading [%s] to Google.", indexFilesDir); final int version = SegmentUtils.getVersionFromDir(indexFilesDir); File indexFile = null; @@ -130,7 +128,7 @@ public DataSegment push(final File indexFilesDir, final DataSegment segment, fin } finally { if (indexFile != null) { - LOG.info("Deleting file [%s]", indexFile); + log.debug("Deleting file [%s]", indexFile); indexFile.delete(); } } diff --git a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentPusher.java b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentPusher.java index fd479768530c..3fe756d13fbb 100644 --- a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentPusher.java +++ b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentPusher.java @@ -46,6 +46,7 @@ import java.util.Map; /** + * */ public class HdfsDataSegmentPusher implements DataSegmentPusher { @@ -77,8 +78,6 @@ public HdfsDataSegmentPusher(HdfsDataSegmentPusherConfig config, Configuration h } } ); - - log.info("Configured HDFS as deep storage"); } @Deprecated @@ -101,7 +100,7 @@ public DataSegment push(final File inDir, final DataSegment segment, final boole // '{partitionNum}_index.zip' without unique paths and '{partitionNum}_{UUID}_index.zip' with unique paths. final String storageDir = this.getStorageDir(segment, false); - log.info( + log.debug( "Copying segment[%s] to HDFS at location[%s/%s]", segment.getId(), fullyQualifiedStorageDirectory.get(), @@ -118,7 +117,7 @@ public DataSegment push(final File inDir, final DataSegment segment, final boole FileSystem fs = tmpIndexFile.getFileSystem(hadoopConfig); fs.mkdirs(tmpIndexFile.getParent()); - log.info("Compressing files from[%s] to [%s]", inDir, tmpIndexFile); + log.debug("Compressing files from[%s] to [%s]", inDir, tmpIndexFile); final long size; final DataSegment dataSegment; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java index 449878715908..6bf24d269f2f 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java @@ -240,8 +240,8 @@ protected TreeMap> getCheckPointsFromContext( ) throws IOException { if (checkpointsString != null) { - log.info("Checkpoints [%s]", checkpointsString); - return toolbox.getObjectMapper().readValue( + log.debug("Got checkpoints from task context[%s].", checkpointsString); + return toolbox.getJsonMapper().readValue( checkpointsString, new TypeReference>>() { diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java index 59df3058e200..3ffe7fa83ec0 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java @@ -53,8 +53,6 @@ public class KafkaRecordSupplier implements RecordSupplier { private final KafkaConsumer consumer; - private final Map consumerProperties; - private final ObjectMapper sortingMapper; private boolean closed; public KafkaRecordSupplier( @@ -62,18 +60,14 @@ public KafkaRecordSupplier( ObjectMapper sortingMapper ) { - this(consumerProperties, sortingMapper, getKafkaConsumer(sortingMapper, consumerProperties)); + this(getKafkaConsumer(sortingMapper, consumerProperties)); } @VisibleForTesting public KafkaRecordSupplier( - Map consumerProperties, - ObjectMapper sortingMapper, KafkaConsumer consumer ) { - this.consumerProperties = consumerProperties; - this.sortingMapper = sortingMapper; this.consumer = consumer; } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 0662e6e4ef46..c379f7050fa0 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -3852,8 +3852,6 @@ protected RecordSupplier setupRecordSupplier() Deserializer keyDeserializerObject = new ByteArrayDeserializer(); Deserializer valueDeserializerObject = new ByteArrayDeserializer(); return new KafkaRecordSupplier( - consumerProperties, - sortingMapper, new KafkaConsumer<>(props, keyDeserializerObject, valueDeserializerObject) ); } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java index c46d223e5cf3..fa43d22bd379 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java @@ -195,8 +195,8 @@ protected TreeMap> getCheckPointsFromContext( ) throws IOException { if (checkpointsString != null) { - log.info("Checkpoints [%s]", checkpointsString); - return toolbox.getObjectMapper().readValue( + log.debug("Got checkpoints from task context[%s]", checkpointsString); + return toolbox.getJsonMapper().readValue( checkpointsString, new TypeReference>>() { diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentPusher.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentPusher.java index e393f42f04a6..17dac631a117 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentPusher.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentPusher.java @@ -51,8 +51,6 @@ public S3DataSegmentPusher( { this.s3Client = s3Client; this.config = config; - - log.info("Configured S3 as deep storage"); } @Override @@ -83,7 +81,7 @@ public DataSegment push(final File indexFilesDir, final DataSegment inSegment, f { final String s3Path = S3Utils.constructSegmentPath(config.getBaseKey(), getStorageDir(inSegment, useUniquePath)); - log.info("Copying segment[%s] to S3 at location[%s]", inSegment.getId(), s3Path); + log.debug("Copying segment[%s] to S3 at location[%s]", inSegment.getId(), s3Path); final File zipOutFile = File.createTempFile("druid", "index.zip"); final long indexSize = CompressionUtils.zip(indexFilesDir, zipOutFile); @@ -108,7 +106,7 @@ public DataSegment push(final File indexFilesDir, final DataSegment inSegment, f throw new RuntimeException(e); } finally { - log.info("Deleting temporary cached index.zip"); + log.debug("Deleting temporary cached index.zip"); zipOutFile.delete(); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java index d1e04598ef1a..812679df9c9e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java @@ -87,7 +87,7 @@ public class TaskToolbox private final MonitorScheduler monitorScheduler; private final ExecutorService queryExecutorService; private final SegmentLoader segmentLoader; - private final ObjectMapper objectMapper; + private final ObjectMapper jsonMapper; private final File taskWorkDir; private final IndexIO indexIO; private final Cache cache; @@ -118,7 +118,7 @@ public TaskToolbox( ExecutorService queryExecutorService, MonitorScheduler monitorScheduler, SegmentLoader segmentLoader, - ObjectMapper objectMapper, + ObjectMapper jsonMapper, File taskWorkDir, IndexIO indexIO, Cache cache, @@ -148,7 +148,7 @@ public TaskToolbox( this.queryExecutorService = queryExecutorService; this.monitorScheduler = monitorScheduler; this.segmentLoader = segmentLoader; - this.objectMapper = objectMapper; + this.jsonMapper = jsonMapper; this.taskWorkDir = taskWorkDir; this.indexIO = Preconditions.checkNotNull(indexIO, "Null IndexIO"); this.cache = cache; @@ -160,7 +160,7 @@ public TaskToolbox( this.lookupNodeService = lookupNodeService; this.dataNodeService = dataNodeService; this.taskReportFileWriter = taskReportFileWriter; - this.taskReportFileWriter.setObjectMapper(this.objectMapper); + this.taskReportFileWriter.setObjectMapper(this.jsonMapper); this.intermediaryDataManager = intermediaryDataManager; } @@ -234,9 +234,9 @@ public MonitorScheduler getMonitorScheduler() return monitorScheduler; } - public ObjectMapper getObjectMapper() + public ObjectMapper getJsonMapper() { - return objectMapper; + return jsonMapper; } public Map fetchSegments(List segments) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java index 830243b625bd..e1c04a93c1bc 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java @@ -29,6 +29,7 @@ import org.apache.druid.discovery.DataNodeService; import org.apache.druid.discovery.DruidNodeAnnouncer; import org.apache.druid.discovery.LookupNodeService; +import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Parent; import org.apache.druid.guice.annotations.Processing; import org.apache.druid.guice.annotations.RemoteChatHandler; @@ -73,7 +74,7 @@ public class TaskToolboxFactory private final ExecutorService queryExecutorService; private final MonitorScheduler monitorScheduler; private final SegmentLoaderFactory segmentLoaderFactory; - private final ObjectMapper objectMapper; + private final ObjectMapper jsonMapper; private final IndexIO indexIO; private final Cache cache; private final CacheConfig cacheConfig; @@ -103,7 +104,7 @@ public TaskToolboxFactory( @Processing ExecutorService queryExecutorService, MonitorScheduler monitorScheduler, SegmentLoaderFactory segmentLoaderFactory, - ObjectMapper objectMapper, + @Json ObjectMapper jsonMapper, IndexIO indexIO, Cache cache, CacheConfig cacheConfig, @@ -132,7 +133,7 @@ public TaskToolboxFactory( this.queryExecutorService = queryExecutorService; this.monitorScheduler = monitorScheduler; this.segmentLoaderFactory = segmentLoaderFactory; - this.objectMapper = objectMapper; + this.jsonMapper = jsonMapper; this.indexIO = Preconditions.checkNotNull(indexIO, "Null IndexIO"); this.cache = cache; this.cacheConfig = cacheConfig; @@ -165,7 +166,7 @@ public TaskToolbox build(Task task) queryExecutorService, monitorScheduler, segmentLoaderFactory.manufacturate(taskWorkDir), - objectMapper, + jsonMapper, taskWorkDir, indexIO, cache, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClient.java index 14892a4c8499..eb1d62da8dfd 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClient.java @@ -62,7 +62,7 @@ public RemoteTaskActionClient( @Override public RetType submit(TaskAction taskAction) throws IOException { - log.info("Performing action for task[%s]: %s", task.getId(), taskAction); + log.debug("Performing action for task[%s]: %s", task.getId(), taskAction); byte[] dataToSend = jsonMapper.writeValueAsBytes(new TaskActionHolder(task, taskAction)); @@ -73,7 +73,11 @@ public RetType submit(TaskAction taskAction) throws IOExcepti final StringFullResponseHolder fullResponseHolder; - log.info("Submitting action for task[%s] to overlord: [%s].", task.getId(), taskAction); + log.debug( + "Submitting action for task[%s] to Overlord: %s", + task.getId(), + jsonMapper.writeValueAsString(taskAction) + ); fullResponseHolder = druidLeaderClient.go( druidLeaderClient.makeRequest(HttpMethod.POST, "/druid/indexer/v1/action") @@ -96,7 +100,12 @@ public RetType submit(TaskAction taskAction) throws IOExcepti } } catch (IOException | ChannelException e) { - log.warn(e, "Exception submitting action for task[%s]", task.getId()); + log.noStackTrace().warn( + e, + "Exception submitting action for task[%s]: %s", + task.getId(), + jsonMapper.writeValueAsString(taskAction) + ); final Duration delay = retryPolicy.getAndIncrementRetryDelay(); if (delay == null) { @@ -104,7 +113,7 @@ public RetType submit(TaskAction taskAction) throws IOExcepti } else { try { final long sleepTime = jitter(delay.getMillis()); - log.info("Will try again in [%s].", new Duration(sleepTime).toString()); + log.warn("Will try again in [%s].", new Duration(sleepTime).toString()); Thread.sleep(sleepTime); } catch (InterruptedException e2) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java index 5cbede11e561..cb9a9df37702 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java @@ -765,7 +765,7 @@ private Appenderator newAppenderator( tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()), metrics, toolbox.getSegmentPusher(), - toolbox.getObjectMapper(), + toolbox.getJsonMapper(), toolbox.getIndexIO(), toolbox.getIndexMergerV9(), toolbox.getQueryRunnerFactoryConglomerate(), @@ -805,7 +805,7 @@ private static StreamAppenderatorDriver newDriver( toolbox.getSegmentHandoffNotifierFactory(), new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), toolbox.getDataSegmentKiller(), - toolbox.getObjectMapper(), + toolbox.getJsonMapper(), metrics ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java index 827053e8fb70..e2a8aa2f43cd 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java @@ -72,7 +72,7 @@ public static Appenderator newAppenderator( storeCompactionState, metrics, segmentPusher, - toolbox.getObjectMapper(), + toolbox.getJsonMapper(), toolbox.getIndexIO(), toolbox.getIndexMergerV9() ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java index 80082ddea21b..33e082b06761 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java @@ -111,7 +111,7 @@ private Map> getIntervalToSegmentIds() th getDataSource(), interval, findVersion(intervalToVersion, interval), - shardSpecFactory.create(getToolbox().getObjectMapper(), i) + shardSpecFactory.create(getToolbox().getJsonMapper(), i) )) .collect(Collectors.toList()) ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java index 57081f52afb6..32bdf0248c34 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java @@ -332,7 +332,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception determinePartitionsStatsGetter = new InnerProcessingStatsGetter(determinePartitionsInnerProcessingRunner); String[] determinePartitionsInput = new String[]{ - toolbox.getObjectMapper().writeValueAsString(spec), + toolbox.getJsonMapper().writeValueAsString(spec), toolbox.getConfig().getHadoopWorkingPath(), toolbox.getSegmentPusher().getPathForHadoop(), hadoopJobIdFile @@ -357,7 +357,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception determineConfigStatus = toolbox - .getObjectMapper() + .getJsonMapper() .readValue(determineConfigStatusString, HadoopDetermineConfigInnerProcessingStatus.class); indexerSchema = determineConfigStatus.getSchema(); @@ -424,7 +424,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception buildSegmentsStatsGetter = new InnerProcessingStatsGetter(innerProcessingRunner); String[] buildSegmentsInput = new String[]{ - toolbox.getObjectMapper().writeValueAsString(indexerSchema), + toolbox.getJsonMapper().writeValueAsString(indexerSchema), version, hadoopJobIdFile }; @@ -441,7 +441,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception new Object[]{buildSegmentsInput} ); - buildSegmentsStatus = toolbox.getObjectMapper().readValue( + buildSegmentsStatus = toolbox.getJsonMapper().readValue( jobStatusString, HadoopIndexGeneratorInnerProcessingStatus.class ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index b3fecb19c979..5afa08bd8de5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -458,7 +458,7 @@ public TaskStatus runTask(final TaskToolbox toolbox) { try { if (chatHandlerProvider.isPresent()) { - log.info("Found chat handler of class[%s]", chatHandlerProvider.get().getClass().getName()); + log.debug("Found chat handler of class[%s]", chatHandlerProvider.get().getClass().getName()); if (chatHandlerProvider.get().get(getId()).isPresent()) { // This is a workaround for ParallelIndexSupervisorTask to avoid double registering when it runs in the @@ -609,7 +609,7 @@ private Map> determineShardSpecs( final PartitionsSpec nonNullPartitionsSpec ) throws IOException { - final ObjectMapper jsonMapper = toolbox.getObjectMapper(); + final ObjectMapper jsonMapper = toolbox.getJsonMapper(); final IndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig(); final IndexIOConfig ioConfig = ingestionSchema.getIOConfig(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java index 65c4291438ae..6d2dc0d75d1e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java @@ -35,6 +35,7 @@ import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.SegmentUtils; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; @@ -155,7 +156,7 @@ public SegmentsAndMetadata process( // If those segments are not pushed here, the remaining available space in appenderator will be kept // small which could lead to smaller segments. final SegmentsAndMetadata pushed = driver.pushAllAndClear(pushTimeout); - LOG.info("Pushed segments[%s]", pushed.getSegments()); + LOG.debug("Pushed segments: %s", SegmentUtils.commaSeparateIdentifiers(pushed.getSegments())); } } } else { @@ -174,7 +175,9 @@ public SegmentsAndMetadata process( } final SegmentsAndMetadata pushed = driver.pushAllAndClear(pushTimeout); - LOG.info("Pushed segments[%s]", pushed.getSegments()); + + LOG.debug("Pushed segments: %s", SegmentUtils.commaSeparateIdentifiers(pushed.getSegments())); + return pushed; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java index dc07d2b22b18..c85d4683310b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java @@ -346,7 +346,7 @@ public String getVersion(final Interval interval) toolbox.getCache(), toolbox.getCacheConfig(), toolbox.getCachePopulatorStats(), - toolbox.getObjectMapper() + toolbox.getJsonMapper() ); this.plumber = plumberSchool.findPlumber(dataSchema, tuningConfig, metrics); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index f821c2f09b3b..b2e3c25c566d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -687,7 +687,7 @@ public Response allocateSegment(DateTime timestamp, @Context final HttpServletRe try { final SegmentIdWithShardSpec segmentIdentifier = allocateNewSegment(timestamp); - return Response.ok(toolbox.getObjectMapper().writeValueAsBytes(segmentIdentifier)).build(); + return Response.ok(toolbox.getJsonMapper().writeValueAsBytes(segmentIdentifier)).build(); } catch (IOException | IllegalStateException e) { return Response.serverError().entity(Throwables.getStackTraceAsString(e)).build(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java index 58dba1310f41..4a10957c7e19 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java @@ -391,7 +391,7 @@ private Set mergeAndPushSegments( partitionId, Preconditions.checkNotNull(partitionsSpec.getNumShards(), "numShards"), partitionsSpec.getPartitionDimensions(), - toolbox.getObjectMapper() + toolbox.getJsonMapper() ), null, // will be filled in the segmentPusher 0 // will be filled in the segmentPusher diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java index 4c9910f07974..30dcec286bb2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.io.Files; @@ -54,7 +55,9 @@ public abstract class BaseRestorableTaskRunner> listeners = new CopyOnWriteArrayList<>(); - /** Writes must be synchronized. This is only a ConcurrentMap so "informational" reads can occur without waiting. */ + /** + * Writes must be synchronized. This is only a ConcurrentMap so "informational" reads can occur without waiting. + */ protected final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); protected final ObjectMapper jsonMapper; protected final TaskConfig taskConfig; @@ -105,7 +108,9 @@ public List>> restore() } } - LOG.info("Restored %,d tasks.", retVal.size()); + if (!retVal.isEmpty()) { + LOG.info("Restored %,d tasks: %s", retVal.size(), Joiner.on(", ").join(retVal)); + } return retVal; } @@ -127,7 +132,7 @@ public void registerListener(TaskRunnerListener listener, Executor executor) } listeners.add(listenerPair); - LOG.info("Registered listener [%s]", listener.getListenerId()); + LOG.debug("Registered listener [%s]", listener.getListenerId()); } } @@ -137,7 +142,7 @@ public void unregisterListener(String listenerId) for (Pair pair : listeners) { if (pair.lhs.getListenerId().equals(listenerId)) { listeners.remove(pair); - LOG.info("Unregistered listener [%s]", listenerId); + LOG.debug("Unregistered listener [%s]", listenerId); return; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java index 0ab0317db7c1..849c39990e90 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java @@ -421,8 +421,8 @@ public TaskStatus call() try { if (!stopping && taskDir.exists()) { - LOGGER.info("Removing task directory: %s", taskDir); FileUtils.deleteDirectory(taskDir); + LOGGER.info("Removing task directory: %s", taskDir); } } catch (Exception e) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunnerUtils.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunnerUtils.java index 49c44c698fb8..d39ccaa81eaa 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunnerUtils.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunnerUtils.java @@ -44,19 +44,10 @@ public static void notifyLocationChanged( final TaskLocation location ) { - log.info("Task [%s] location changed to [%s].", taskId, location); + log.debug("Task [%s] location changed to [%s].", taskId, location); for (final Pair listener : listeners) { try { - listener.rhs.execute( - new Runnable() - { - @Override - public void run() - { - listener.lhs.locationChanged(taskId, location); - } - } - ); + listener.rhs.execute(() -> listener.lhs.locationChanged(taskId, location)); } catch (Exception e) { log.makeAlert(e, "Unable to notify task listener") @@ -74,19 +65,10 @@ public static void notifyStatusChanged( final TaskStatus status ) { - log.info("Task [%s] status changed to [%s].", taskId, status.getStatusCode()); + log.debug("Task [%s] status changed to [%s].", taskId, status.getStatusCode()); for (final Pair listener : listeners) { try { - listener.rhs.execute( - new Runnable() - { - @Override - public void run() - { - listener.lhs.statusChanged(taskId, status); - } - } - ); + listener.rhs.execute(() -> listener.lhs.statusChanged(taskId, status)); } catch (Exception e) { log.makeAlert(e, "Unable to notify task listener") diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java index e92b9da51292..8451e14a33ed 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java @@ -193,7 +193,7 @@ public TaskStatus call() // This will block for a while. So we append the thread information with more details final String priorThreadName = Thread.currentThread().getName(); Thread.currentThread() - .setName(StringUtils.format("%s-[%s]", priorThreadName, task.getId())); + .setName(StringUtils.format("[%s]-%s", task.getId(), priorThreadName)); TaskStatus taskStatus = null; final TaskToolbox toolbox = toolboxFactory.build(task); @@ -243,8 +243,8 @@ public TaskStatus call() try { if (!stopping && taskDir.exists()) { - LOGGER.info("Removing task directory: %s", taskDir); FileUtils.deleteDirectory(taskDir); + LOGGER.info("Removed task directory: %s", taskDir); } } catch (Exception e) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index 2f8118786b31..dedba07d2238 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -199,7 +199,7 @@ public Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox t tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()), metrics, toolbox.getSegmentPusher(), - toolbox.getObjectMapper(), + toolbox.getJsonMapper(), toolbox.getIndexIO(), toolbox.getIndexMergerV9(), toolbox.getQueryRunnerFactoryConglomerate(), @@ -238,7 +238,7 @@ public StreamAppenderatorDriver newDriver( toolbox.getSegmentHandoffNotifierFactory(), new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), toolbox.getDataSegmentKiller(), - toolbox.getObjectMapper(), + toolbox.getJsonMapper(), metrics ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 3ac7139cc4c3..a2618dddf65b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -30,6 +30,7 @@ import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.google.common.util.concurrent.FutureCallback; @@ -343,12 +344,11 @@ public void initializeSequences() throws IOException } } - log.info("Starting with sequences: %s", sequences); + log.info("Starting with sequences: %s", sequences); } private TaskStatus runInternal(TaskToolbox toolbox) throws Exception { - log.info("SeekableStream indexing task starting up!"); startTime = DateTimes.nowUtc(); status = Status.STARTING; @@ -356,10 +356,10 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception initializeSequences(); if (chatHandlerProvider.isPresent()) { - log.info("Found chat handler of class[%s]", chatHandlerProvider.get().getClass().getName()); + log.debug("Found chat handler of class[%s]", chatHandlerProvider.get().getClass().getName()); chatHandlerProvider.get().register(task.getId(), this, false); } else { - log.warn("No chat handler detected"); + log.warn("No chat handler detected."); } runThread = Thread.currentThread(); @@ -444,7 +444,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception final Map restoredMetadataMap = (Map) restoredMetadata; final SeekableStreamEndSequenceNumbers restoredNextPartitions = deserializePartitionsFromMetadata( - toolbox.getObjectMapper(), + toolbox.getJsonMapper(), restoredMetadataMap.get(METADATA_NEXT_PARTITIONS) ); @@ -472,16 +472,20 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception this.endOffsets.putAll(sequences.size() == 0 ? currOffsets : getLastSequenceMetadata().getEndOffsets()); - log.info("End sequences changed to [%s]", endOffsets); } } + log.info( + "Initialized sequences: %s", + sequences.stream().map(SequenceMetadata::toString).collect(Collectors.joining(", ")) + ); + // Filter out partitions with END_OF_SHARD markers since these partitions have already been fully read. This // should have been done by the supervisor already so this is defensive. int numPreFilterPartitions = currOffsets.size(); if (currOffsets.entrySet().removeIf(x -> isEndOfShard(x.getValue()))) { log.info( - "Removed [%d] partitions from assignment which have already been closed", + "Removed [%d] partitions from assignment which have already been closed.", numPreFilterPartitions - currOffsets.size() ); } @@ -548,7 +552,7 @@ public void run() possiblyResetDataSourceMetadata(toolbox, recordSupplier, assignment); if (assignment.isEmpty()) { - log.info("All partitions have been fully read"); + log.debug("All partitions have been fully read."); publishOnStop.set(true); stopRequested.set(true); } @@ -639,7 +643,8 @@ public void run() // move the segment out from the active segments of BaseAppenderatorDriver to make a new segment. final boolean isPushRequired = addResult.isPushRequired( tuningConfig.getPartitionsSpec().getMaxRowsPerSegment(), - tuningConfig.getPartitionsSpec().getMaxTotalRowsOr(DynamicPartitionsSpec.DEFAULT_MAX_TOTAL_ROWS) + tuningConfig.getPartitionsSpec() + .getMaxTotalRowsOr(DynamicPartitionsSpec.DEFAULT_MAX_TOTAL_ROWS) ); if (isPushRequired && !sequenceToUse.isCheckpointed()) { sequenceToCheckpoint = sequenceToUse; @@ -669,7 +674,7 @@ public void run() @Override public void onSuccess(@Nullable Object result) { - log.info("Persist completed with metadata [%s]", result); + log.debug("Persist completed with metadata: %s", result); } @Override @@ -747,7 +752,6 @@ public void onFailure(Throwable t) throw e; } finally { - log.info("Persisting all pending data"); try { driver.persist(committerSupplier.get()); // persist pending data } @@ -813,17 +817,17 @@ public void onFailure(Throwable t) catch (TimeoutException e) { // Handoff timeout is not an indexing failure, but coordination failure. We simply ignore timeout exception // here. - log.makeAlert("Timed out after [%d] millis waiting for handoffs", tuningConfig.getHandoffConditionTimeout()) - .addData("TaskId", task.getId()) + log.makeAlert("Timeout waiting for handoff") + .addData("taskId", task.getId()) + .addData("handoffConditionTimeout", tuningConfig.getHandoffConditionTimeout()) .emit(); } } for (SegmentsAndMetadata handedOff : handedOffList) { log.info( - "Handoff completed for segments %s with metadata[%s].", - Lists.transform(handedOff.getSegments(), DataSegment::getId), - Preconditions.checkNotNull(handedOff.getCommitMetadata(), "commitMetadata") + "Handoff complete for segments: %s", + String.join(", ", Lists.transform(handedOff.getSegments(), DataSegment::toString)) ); } @@ -855,8 +859,6 @@ public void onFailure(Throwable t) Thread.currentThread().interrupt(); throw e; } - - log.info("The task was asked to stop before completing"); } catch (Exception e) { // (3) catch all other exceptions thrown for the whole ingestion steps including the final publishing. @@ -932,7 +934,7 @@ private void checkPublishAndHandoffFailure() throws ExecutionException, Interrup private void publishAndRegisterHandoff(SequenceMetadata sequenceMetadata) { - log.info("Publishing segments for sequence [%s]", sequenceMetadata); + log.debug("Publishing segments for sequence [%s].", sequenceMetadata); final ListenableFuture publishFuture = Futures.transform( driver.publish( @@ -965,13 +967,15 @@ private void publishAndRegisterHandoff(SequenceMetadata( - toolbox.getObjectMapper().>>readValue( + toolbox.getJsonMapper().>>readValue( sequencesPersistFile, getSequenceMetadataTypeReference() ) @@ -1035,10 +1042,11 @@ private boolean restoreSequences() throws IOException private synchronized void persistSequences() throws IOException { - log.info("Persisting Sequences Metadata [%s]", sequences); - toolbox.getObjectMapper().writerWithType( + toolbox.getJsonMapper().writerFor( getSequenceMetadataTypeReference() ).writeValue(getSequencesPersistFile(toolbox), sequences); + + log.info("Saved sequence metadata to disk: %s", sequences); } private Map getTaskCompletionReports(@Nullable String errorMsg) @@ -1087,16 +1095,16 @@ private void maybePersistAndPublishSequences(Supplier committerSuppli if (!sequenceMetadata.isOpen() && !publishingSequences.contains(sequenceMetadata.getSequenceName())) { publishingSequences.add(sequenceMetadata.getSequenceName()); try { - Object result = driver.persist(committerSupplier.get()); - log.info( - "Persist completed with results: [%s], adding sequence [%s] to publish queue", + final Object result = driver.persist(committerSupplier.get()); + log.debug( + "Persist completed with metadata [%s], adding sequence [%s] to publish queue.", result, - sequenceMetadata + sequenceMetadata.getSequenceName() ); publishAndRegisterHandoff(sequenceMetadata); } catch (InterruptedException e) { - log.warn("Interrupted while persisting sequence [%s]", sequenceMetadata); + log.warn("Interrupted while persisting metadata for sequence [%s].", sequenceMetadata.getSequenceName()); throw e; } } @@ -1123,7 +1131,7 @@ private Set> assignPartitions( assignment.add(StreamPartition.of(stream, partition)); } else { - log.info("Finished reading partition[%s].", partition); + log.info("Finished reading partition[%s], up to[%s].", partition, currOffset); } } @@ -1228,7 +1236,7 @@ private void seekToStartingSequence( { for (final StreamPartition partition : partitions) { final SequenceOffsetType sequence = currOffsets.get(partition.getPartitionId()); - log.info("Seeking partition[%s] to sequenceNumber[%s].", partition.getPartitionId(), sequence); + log.info("Seeking partition[%s] to[%s].", partition.getPartitionId(), sequence); recordSupplier.seek(partition, sequence); } } @@ -1258,13 +1266,13 @@ private boolean possiblyPause() throws InterruptedException hasPaused.signalAll(); while (pauseRequested) { - log.info("Pausing ingestion until resumed"); + log.debug("Received pause command, pausing ingestion until resumed."); shouldResume.await(); } status = Status.READING; shouldResume.signalAll(); - log.info("Ingestion loop resumed"); + log.debug("Received resume command, resuming ingestion."); return true; } } @@ -1276,31 +1284,30 @@ private boolean possiblyPause() throws InterruptedException } - private void handleParseException(ParseException pe, OrderedPartitionableRecord record) + private void handleParseException(ParseException e, OrderedPartitionableRecord record) { - if (pe.isFromPartiallyValidRow()) { + if (e.isFromPartiallyValidRow()) { rowIngestionMeters.incrementProcessedWithError(); } else { rowIngestionMeters.incrementUnparseable(); } if (tuningConfig.isLogParseExceptions()) { - log.error( - pe, - "Encountered parse exception on row from partition[%s] sequenceNumber[%s]", + log.info( + e, + "Row at partition[%s] offset[%s] was unparseable.", record.getPartitionId(), record.getSequenceNumber() ); } if (savedParseExceptions != null) { - savedParseExceptions.add(pe); + savedParseExceptions.add(e); } if (rowIngestionMeters.getUnparseable() + rowIngestionMeters.getProcessedWithError() > tuningConfig.getMaxParseExceptions()) { - log.error("Max parse exceptions exceeded, terminating task..."); - throw new RuntimeException("Max parse exceptions exceeded, terminating task..."); + throw new RuntimeException("Max parse exceptions exceeded"); } } @@ -1318,8 +1325,7 @@ private void requestPause() protected void sendResetRequestAndWait( Map, SequenceOffsetType> outOfRangePartitions, TaskToolbox taskToolbox - ) - throws IOException + ) throws IOException { Map partitionOffsetMap = CollectionUtils.mapKeys( outOfRangePartitions, @@ -1341,13 +1347,19 @@ protected void sendResetRequestAndWait( ); if (result) { - log.makeAlert("Resetting sequences for datasource [%s]", task.getDataSource()) + log.makeAlert("Offsets were reset automatically, potential data duplication or loss") + .addData("task", task.getId()) + .addData("dataSource", task.getDataSource()) .addData("partitions", partitionOffsetMap.keySet()) .emit(); requestPause(); } else { - log.makeAlert("Failed to send reset request for partitions [%s]", partitionOffsetMap.keySet()).emit(); + log.makeAlert("Failed to send offset reset request") + .addData("task", task.getId()) + .addData("dataSource", task.getDataSource()) + .addData("partitions", ImmutableSet.copyOf(partitionOffsetMap.keySet())) + .emit(); } } @@ -1609,7 +1621,7 @@ public Response setEndOffsets( && latestSequence.getExclusiveStartPartitions().equals(exclusiveStartPartitions) && !finish) || (latestSequence.getEndOffsets().equals(sequenceNumbers) && finish)) { - log.warn("Ignoring duplicate request, end sequences already set for sequences [%s]", sequenceNumbers); + log.warn("Ignoring duplicate request, end offsets already set for sequences [%s]", sequenceNumbers); resume(); return Response.ok(sequenceNumbers).build(); } else if (latestSequence.isCheckpointed()) { @@ -1621,7 +1633,7 @@ public Response setEndOffsets( )).build(); } else if (!isPaused()) { return Response.status(Response.Status.BAD_REQUEST) - .entity("Task must be paused before changing the end sequences") + .entity("Task must be paused before changing the end offsets") .build(); } @@ -1644,11 +1656,15 @@ public Response setEndOffsets( latestSequence.setEndOffsets(sequenceNumbers); if (finish) { - log.info("Updating endOffsets from [%s] to [%s]", endOffsets, sequenceNumbers); + log.info( + "Sequence[%s] end offsets updated from [%s] to [%s].", + latestSequence.getSequenceName(), + endOffsets, + sequenceNumbers + ); endOffsets.putAll(sequenceNumbers); } else { // create new sequence - log.info("Creating new sequence with startOffsets [%s] and endOffsets [%s]", sequenceNumbers, endOffsets); final SequenceMetadata newSequence = new SequenceMetadata<>( latestSequence.getSequenceId() + 1, StringUtils.format("%s_%d", ioConfig.getBaseSequenceName(), latestSequence.getSequenceId() + 1), @@ -1657,12 +1673,20 @@ public Response setEndOffsets( false, exclusiveStartPartitions ); + + log.info( + "Sequence[%s] created with start offsets [%s] and end offsets [%s].", + newSequence.getSequenceName(), + sequenceNumbers, + endOffsets + ); + addSequence(newSequence); } persistSequences(); } catch (Exception e) { - log.error(e, "Unable to set end sequences, dying"); + log.error(e, "Failed to set end offsets."); backgroundThreadException = e; // should resume to immediately finish kafka index task as failed resume(); @@ -1769,7 +1793,7 @@ public Response pause() throws InterruptedException } try { - return Response.ok().entity(toolbox.getObjectMapper().writeValueAsString(getCurrentOffsets())).build(); + return Response.ok().entity(toolbox.getJsonMapper().writeValueAsString(getCurrentOffsets())).build(); } catch (JsonProcessingException e) { throw new RuntimeException(e); @@ -1833,7 +1857,7 @@ private boolean verifyRecordInRange( // Verify that the record is at least as high as its currOffset. final SequenceOffsetType currOffset = Preconditions.checkNotNull( currOffsets.get(partition), - "Current offset is null for sequenceNumber[%s] and partition[%s]", + "Current offset is null for partition[%s]", recordOffset, partition ); @@ -1870,7 +1894,9 @@ private boolean verifyRecordInRange( * * @param toolbox task toolbox * @param checkpointsString the json-serialized checkpoint string + * * @return checkpoint + * * @throws IOException jsonProcessingException */ @Nullable @@ -1884,6 +1910,7 @@ protected abstract TreeMap> ge * This is what would become the start offsets of the next reader, if we stopped reading now. * * @param sequenceNumber the sequence number that has already been processed + * * @return next sequence number to be stored */ protected abstract SequenceOffsetType getNextStartOffset(SequenceOffsetType sequenceNumber); @@ -1893,6 +1920,7 @@ protected abstract TreeMap> ge * * @param mapper json objectMapper * @param object metadata + * * @return SeekableStreamEndSequenceNumbers */ protected abstract SeekableStreamEndSequenceNumbers deserializePartitionsFromMetadata( @@ -1906,7 +1934,9 @@ protected abstract SeekableStreamEndSequenceNumbers createDataSourceMetadata( @@ -1929,6 +1960,7 @@ protected abstract SeekableStreamDataSourceMetadata createSequenceNumber(SequenceOffsetType sequenceNumber); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java index 17ea3c9415b4..2b2370694713 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java @@ -308,7 +308,7 @@ TransactionalSegmentPublisher createPublisher( final Map commitMetaMap = (Map) Preconditions.checkNotNull(commitMetadata, "commitMetadata"); final SeekableStreamEndSequenceNumbers finalPartitions = runner.deserializePartitionsFromMetadata( - toolbox.getObjectMapper(), + toolbox.getJsonMapper(), commitMetaMap.get(SeekableStreamIndexTaskRunner.METADATA_PUBLISH_PARTITIONS) ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 17b75c887ff8..5a3db116dc58 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -2701,7 +2701,8 @@ private void createNewTasks() .filter(x -> x.getValue().get() != null) .collect( Collectors.collectingAndThen( - Collectors.toMap(Entry::getKey, x -> x.getValue().get()), ImmutableMap::copyOf + Collectors.toMap(Entry::getKey, x -> x.getValue().get()), + ImmutableMap::copyOf ) ); @@ -2713,7 +2714,8 @@ private void createNewTasks() .filter(x -> x.getValue().get() != null) .collect( Collectors.collectingAndThen( - Collectors.toMap(Entry::getKey, x -> x.getValue().get()), ImmutableMap::copyOf + Collectors.toMap(Entry::getKey, x -> x.getValue().get()), + ImmutableMap::copyOf ) ); } else { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/IntermediaryDataManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/IntermediaryDataManager.java index 5b6078f74f9b..b35b1918d339 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/IntermediaryDataManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/IntermediaryDataManager.java @@ -205,11 +205,14 @@ private void discoverSupervisorTaskPartitions() ); } } - LOG.info( - "Discovered partitions for [%s] new supervisor tasks under location[%s]", - numDiscovered.getValue(), - location.getPath() - ); + + if (numDiscovered.getValue() > 0) { + LOG.info( + "Discovered partitions for [%s] new supervisor tasks under location[%s]", + numDiscovered.getValue(), + location.getPath() + ); + } } } @@ -232,7 +235,9 @@ private void deleteExpiredSuprevisorTaskPartitionsIfNotRunning() throws Interrup } } - LOG.info("Found [%s] expired supervisor tasks", expiredSupervisorTasks.size()); + if (!expiredSupervisorTasks.isEmpty()) { + LOG.info("Found [%s] expired supervisor tasks", expiredSupervisorTasks.size()); + } if (!expiredSupervisorTasks.isEmpty()) { final Map taskStatuses = indexingServiceClient.getTaskStatuses(expiredSupervisorTasks); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java index 575524a1960d..bd84c8c5da0b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java @@ -42,6 +42,7 @@ import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; @@ -66,6 +67,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; /** * This class manages the list of tasks assigned to this worker. @@ -129,7 +131,7 @@ public void start() throws Exception synchronized (lock) { try { - log.info("Starting..."); + log.debug("Starting..."); cleanupAndMakeTmpTaskDir(); registerLocationListener(); restoreRestorableTasks(); @@ -137,7 +139,7 @@ public void start() throws Exception initCompletedTasks(); scheduleCompletedTasksCleanup(); lifecycleLock.started(); - log.info("Started."); + log.debug("Started."); } catch (Exception e) { log.makeAlert(e, "Exception starting WorkerTaskManager.").emit(); @@ -163,7 +165,7 @@ public void stop() throws Exception taskRunner.unregisterListener("WorkerTaskManager"); exec.shutdownNow(); taskRunner.stop(); - log.info("Stopped WorkerTaskManager."); + log.debug("Stopped WorkerTaskManager."); } catch (Exception e) { log.makeAlert(e, "Exception stopping WorkerTaskManager") @@ -263,16 +265,18 @@ public void assignTask(Task task) if (assignedTasks.containsKey(task.getId()) || runningTasks.containsKey(task.getId()) || completedTasks.containsKey(task.getId())) { - log.info("Assign task[%s] request ignored because it exists already.", task.getId()); + log.warn("Request to assign task[%s] ignored because it exists already.", task.getId()); return; } try { - FileUtils.writeAtomically(new File(getAssignedTaskDir(), task.getId()), getTmpTaskDir(), + FileUtils.writeAtomically( + new File(getAssignedTaskDir(), task.getId()), + getTmpTaskDir(), os -> { - jsonMapper.writeValue(os, task); - return null; - } + jsonMapper.writeValue(os, task); + return null; + } ); assignedTasks.put(task.getId(), task); } @@ -326,7 +330,7 @@ private void initAssignedTasks() { File assignedTaskDir = getAssignedTaskDir(); - log.info("Looking for any previously assigned tasks on disk[%s].", assignedTaskDir); + log.debug("Looking for any previously assigned tasks on disk[%s].", assignedTaskDir); assignedTaskDir.mkdirs(); @@ -340,16 +344,24 @@ private void initAssignedTasks() Task task = jsonMapper.readValue(taskFile, Task.class); if (taskId.equals(task.getId())) { assignedTasks.put(taskId, task); - log.info("Found assigned task[%s].", taskId); } else { throw new ISE("WTF! Corrupted assigned task on disk[%s].", taskFile.getAbsoluteFile()); } } catch (IOException ex) { - log.error(ex, "Failed to read assigned task from disk at [%s]. Ignored.", taskFile.getAbsoluteFile()); + log.noStackTrace() + .error(ex, "Failed to read assigned task from disk at [%s]. Ignored.", taskFile.getAbsoluteFile()); } } + if (!assignedTasks.isEmpty()) { + log.info( + "Found %,d running tasks from previous run: %s", + assignedTasks.size(), + assignedTasks.values().stream().map(Task::getId).collect(Collectors.joining(", ")) + ); + } + for (Task task : assignedTasks.values()) { submitNoticeToExec(new RunNotice(task)); } @@ -426,11 +438,12 @@ private void moveFromRunningToCompleted(String taskId, TaskAnnouncement taskAnno completedTasks.put(taskId, taskAnnouncement); try { - FileUtils.writeAtomically(new File(getCompletedTaskDir(), taskId), getTmpTaskDir(), + FileUtils.writeAtomically( + new File(getCompletedTaskDir(), taskId), getTmpTaskDir(), os -> { - jsonMapper.writeValue(os, taskAnnouncement); - return null; - } + jsonMapper.writeValue(os, taskAnnouncement); + return null; + } ); } catch (IOException ex) { @@ -443,7 +456,7 @@ private void moveFromRunningToCompleted(String taskId, TaskAnnouncement taskAnno private void initCompletedTasks() { File completedTaskDir = getCompletedTaskDir(); - log.info("Looking for any previously completed tasks on disk[%s].", completedTaskDir); + log.debug("Looking for any previously completed tasks on disk[%s].", completedTaskDir); completedTaskDir.mkdirs(); @@ -457,7 +470,6 @@ private void initCompletedTasks() TaskAnnouncement taskAnnouncement = jsonMapper.readValue(taskFile, TaskAnnouncement.class); if (taskId.equals(taskAnnouncement.getTaskId())) { completedTasks.put(taskId, taskAnnouncement); - log.info("Found completed task[%s] with status[%s].", taskId, taskAnnouncement.getStatus()); } else { throw new ISE("WTF! Corrupted completed task on disk[%s].", taskFile.getAbsoluteFile()); } @@ -466,6 +478,17 @@ private void initCompletedTasks() log.error(ex, "Failed to read completed task from disk at [%s]. Ignored.", taskFile.getAbsoluteFile()); } } + + if (!completedTasks.isEmpty()) { + log.info( + "Found %,d complete tasks from previous run: %s", + completedTasks.size(), + completedTasks.values().stream().map( + taskAnnouncement -> + StringUtils.format("%s (%s)", taskAnnouncement.getTaskId(), taskAnnouncement.getStatus()) + ).collect(Collectors.joining(", ")) + ); + } } private void scheduleCompletedTasksCleanup() @@ -526,7 +549,7 @@ private void scheduleCompletedTasksCleanup() TaskStatus status = taskStatusesFromOverlord.get(taskId); if (status == null || status.isComplete()) { - log.info( + log.debug( "Deleting completed task[%s] information, overlord task status[%s].", taskId, status == null ? "unknown" : status.getStatusCode() @@ -696,7 +719,7 @@ public void handle() changeHistory.addChangeRequest(new WorkerHistoryItem.TaskUpdate(latest)); taskAnnouncementChanged(latest); log.info( - "Job's finished. Completed [%s] with status [%s]", + "Task [%s] completed with status [%s].", task.getId(), status.getStatusCode() ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskMonitor.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskMonitor.java index d577233521d9..23b19683acbb 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskMonitor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskMonitor.java @@ -102,7 +102,7 @@ public void start() throws Exception registerRunListener(); pathChildrenCache.start(); - log.info("Started WorkerTaskMonitor."); + log.debug("Started WorkerTaskMonitor."); started = true; } catch (InterruptedException e) { @@ -131,10 +131,11 @@ private void cleanupStaleAnnouncements() throws Exception } if (completionStatus != null) { - log.info("Cleaning up stale announcement for task [%s]. New status is [%s].", - announcement.getTaskStatus().getId(), - completionStatus.getStatusCode() - ); + log.info( + "Cleaning up stale announcement for task [%s]. New status is [%s].", + announcement.getTaskStatus().getId(), + completionStatus.getStatusCode() + ); workerCuratorCoordinator.updateTaskStatusAnnouncement( TaskAnnouncement.create( announcement.getTaskStatus().getId(), @@ -186,7 +187,7 @@ public void stop() throws Exception started = false; pathChildrenCache.close(); - log.info("Stopped WorkerTaskMonitor."); + log.debug("Stopped WorkerTaskMonitor."); } catch (Exception e) { log.makeAlert(e, "Exception stopping WorkerTaskMonitor") diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java index 9330396d3249..aed8c0fb596c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java @@ -159,7 +159,7 @@ public void testGetMonitorScheduler() @Test public void testGetObjectMapper() { - Assert.assertEquals(ObjectMapper, taskToolbox.build(task).getObjectMapper()); + Assert.assertEquals(ObjectMapper, taskToolbox.build(task).getJsonMapper()); } @Test diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index f0594724d325..f267d76b3c6b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -86,6 +86,7 @@ import org.apache.druid.timeline.partition.NumberedOverwriteShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.PartitionIds; +import org.hamcrest.CoreMatchers; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; @@ -1636,7 +1637,10 @@ public void testOverwriteWithDifferentSegmentGranularity() throws Exception public static void checkTaskStatusErrorMsgForParseExceptionsExceeded(TaskStatus status) { // full stacktrace will be too long and make tests brittle (e.g. if line # changes), just match the main message - Assert.assertTrue(status.getErrorMsg().contains("Max parse exceptions exceeded, terminating task...")); + Assert.assertThat( + status.getErrorMsg(), + CoreMatchers.containsString("Max parse exceptions exceeded") + ); } private Pair> runTask(IndexTask task) throws Exception diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java index 9ab63139859b..1bffbd86a4df 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java @@ -66,6 +66,7 @@ public Appenderator createRealtimeAppenderatorForTask( ) { realtimeAppenderator = Appenderators.createRealtime( + taskId, schema, config, metrics, @@ -98,6 +99,7 @@ public Appenderator createOfflineAppenderatorForTask( ) { return Appenderators.createOffline( + taskId, schema, config, storeCompactionState, diff --git a/integration-tests/src/test/resources/results/auth_test_sys_schema_segments.json b/integration-tests/src/test/resources/results/auth_test_sys_schema_segments.json index a0efc0f85666..4437e725e28b 100644 --- a/integration-tests/src/test/resources/results/auth_test_sys_schema_segments.json +++ b/integration-tests/src/test/resources/results/auth_test_sys_schema_segments.json @@ -13,6 +13,6 @@ "is_available": 1, "is_realtime": 0, "is_overshadowed": 0, - "payload": "{\"dataSource\":\"auth_test\",\"interval\":\"2012-12-29T00:00:00.000Z/2013-01-10T08:00:00.000Z\",\"version\":\"2013-01-10T08:13:47.830Z_v9\",\"loadSpec\":{\"load spec is pruned, because it's not needed on Brokers, but eats a lot of heap space\":\"\"},\"dimensions\":\"anonymous,area_code,city,continent_code,country_name,dma_code,geo,language,namespace,network,newpage,page,postal_code,region_lookup,robot,unpatrolled,user\",\"metrics\":\"added,count,deleted,delta,delta_hist,unique_users,variation\",\"shardSpec\":{\"type\":\"none\"},\"lastCompactionState\":null,\"binaryVersion\":9,\"size\":446027801,\"identifier\":\"auth_test_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\",\"overshadowed\":false}" + "payload": "{\"dataSource\":\"auth_test\",\"interval\":\"2012-12-29T00:00:00.000Z/2013-01-10T08:00:00.000Z\",\"version\":\"2013-01-10T08:13:47.830Z_v9\",\"loadSpec\":{\"load spec is pruned, because it's not needed on Brokers, but eats a lot of heap space\":\"\"},\"dimensions\":\"anonymous,area_code,city,continent_code,country_name,dma_code,geo,language,namespace,network,newpage,page,postal_code,region_lookup,robot,unpatrolled,user\",\"metrics\":\"added,count,deleted,delta,delta_hist,unique_users,variation\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":446027801,\"identifier\":\"auth_test_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\",\"overshadowed\":false}" } ] diff --git a/processing/src/main/java/org/apache/druid/guice/PropertiesModule.java b/processing/src/main/java/org/apache/druid/guice/PropertiesModule.java index 9fe6fec5dd13..4dd7b7045885 100644 --- a/processing/src/main/java/org/apache/druid/guice/PropertiesModule.java +++ b/processing/src/main/java/org/apache/druid/guice/PropertiesModule.java @@ -35,8 +35,6 @@ import java.util.List; import java.util.Properties; -/** - */ public class PropertiesModule implements Module { private static final Logger log = new Logger(PropertiesModule.class); @@ -68,7 +66,7 @@ public void configure(Binder binder) } if (stream != null) { - log.info("Loading properties from %s", propertiesFile); + log.debug("Loading properties from %s", propertiesFile); try (final InputStreamReader in = new InputStreamReader(stream, StandardCharsets.UTF_8)) { fileProps.load(in); } @@ -78,7 +76,7 @@ public void configure(Binder binder) } } catch (FileNotFoundException e) { - log.wtf(e, "This can only happen if the .exists() call lied. That's f'd up."); + log.wtf(e, "This can only happen if the .exists() call lied."); } finally { CloseQuietly.close(stream); diff --git a/processing/src/main/java/org/apache/druid/query/ChainedExecutionQueryRunner.java b/processing/src/main/java/org/apache/druid/query/ChainedExecutionQueryRunner.java index 0990c9825179..01cd0d9f7a6d 100644 --- a/processing/src/main/java/org/apache/druid/query/ChainedExecutionQueryRunner.java +++ b/processing/src/main/java/org/apache/druid/query/ChainedExecutionQueryRunner.java @@ -132,7 +132,7 @@ public Iterable call() throw new RuntimeException(e); } catch (Exception e) { - log.error(e, "Exception with one of the sequences!"); + log.noStackTrace().error(e, "Exception with one of the sequences!"); Throwables.propagateIfPossible(e); throw new RuntimeException(e); } @@ -155,7 +155,7 @@ public Iterable call() ).iterator(); } catch (InterruptedException e) { - log.warn(e, "Query interrupted, cancelling pending results, query id [%s]", query.getId()); + log.noStackTrace().warn(e, "Query interrupted, cancelling pending results, query id [%s]", query.getId()); futures.cancel(true); throw new QueryInterruptedException(e); } diff --git a/processing/src/main/java/org/apache/druid/segment/CompressedPools.java b/processing/src/main/java/org/apache/druid/segment/CompressedPools.java index 7137a1c3be98..d62443270bb5 100644 --- a/processing/src/main/java/org/apache/druid/segment/CompressedPools.java +++ b/processing/src/main/java/org/apache/druid/segment/CompressedPools.java @@ -30,8 +30,6 @@ import java.nio.ByteOrder; import java.util.concurrent.atomic.AtomicLong; -/** - */ public class CompressedPools { private static final Logger log = new Logger(CompressedPools.class); @@ -46,7 +44,7 @@ public class CompressedPools @Override public BufferRecycler get() { - log.info("Allocating new bufferRecycler[%,d]", counter.incrementAndGet()); + log.debug("Allocating new bufferRecycler[%,d]", counter.incrementAndGet()); return new BufferRecycler(); } } @@ -66,7 +64,7 @@ public static ResourceHolder getBufferRecycler() @Override public byte[] get() { - log.info("Allocating new outputBytesPool[%,d]", counter.incrementAndGet()); + log.debug("Allocating new outputBytesPool[%,d]", counter.incrementAndGet()); return new byte[BUFFER_SIZE]; } } @@ -86,7 +84,7 @@ public static ResourceHolder getOutputBytes() @Override public ByteBuffer get() { - log.info("Allocating new bigEndByteBuf[%,d]", counter.incrementAndGet()); + log.debug("Allocating new bigEndByteBuf[%,d]", counter.incrementAndGet()); return ByteBuffer.allocateDirect(BUFFER_SIZE).order(ByteOrder.BIG_ENDIAN); } } @@ -101,7 +99,7 @@ public ByteBuffer get() @Override public ByteBuffer get() { - log.info("Allocating new littleEndByteBuf[%,d]", counter.incrementAndGet()); + log.debug("Allocating new littleEndByteBuf[%,d]", counter.incrementAndGet()); return ByteBuffer.allocateDirect(BUFFER_SIZE).order(ByteOrder.LITTLE_ENDIAN); } } diff --git a/processing/src/main/java/org/apache/druid/segment/IndexMerger.java b/processing/src/main/java/org/apache/druid/segment/IndexMerger.java index c1d42eac1596..1b0079089403 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexMerger.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexMerger.java @@ -399,7 +399,7 @@ class DictionaryMergeIterator implements CloseableIterator Indexed indexed = dimValueLookups[i]; if (useDirect) { int allocationSize = indexed.size() * Integer.BYTES; - log.debug("Allocating dictionary merging direct buffer with size[%,d]", allocationSize); + log.trace("Allocating dictionary merging direct buffer with size[%,d]", allocationSize); mergeBufferTotalSize += allocationSize; final ByteBuffer conversionDirectBuffer = ByteBuffer.allocateDirect(allocationSize); conversions[i] = conversionDirectBuffer.asIntBuffer(); @@ -419,7 +419,7 @@ class DictionaryMergeIterator implements CloseableIterator pQueue.add(Pair.of(i, iter)); } } - log.info("Allocated [%,d] bytes of dictionary merging direct buffers", mergeBufferTotalSize); + log.debug("Allocated [%,d] bytes of dictionary merging direct buffers", mergeBufferTotalSize); } @Override @@ -482,11 +482,11 @@ public void close() { long mergeBufferTotalSize = 0; for (Pair bufferAllocation : directBufferAllocations) { - log.debug("Freeing dictionary merging direct buffer with size[%,d]", bufferAllocation.rhs); + log.trace("Freeing dictionary merging direct buffer with size[%,d]", bufferAllocation.rhs); mergeBufferTotalSize += bufferAllocation.rhs; ByteBufferUtils.free(bufferAllocation.lhs); } - log.info("Freed [%,d] bytes of dictionary merging direct buffers", mergeBufferTotalSize); + log.debug("Freed [%,d] bytes of dictionary merging direct buffers", mergeBufferTotalSize); } } } diff --git a/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java b/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java index aeb8fbb7eca6..f25a275a0e93 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java @@ -142,19 +142,19 @@ private File makeIndexFiles( SegmentWriteOutMediumFactory omf = segmentWriteOutMediumFactory != null ? segmentWriteOutMediumFactory : defaultSegmentWriteOutMediumFactory; - log.info("Using SegmentWriteOutMediumFactory[%s]", omf.getClass().getSimpleName()); + log.debug("Using SegmentWriteOutMediumFactory[%s]", omf.getClass().getSimpleName()); SegmentWriteOutMedium segmentWriteOutMedium = omf.makeSegmentWriteOutMedium(outDir); closer.register(segmentWriteOutMedium); long startTime = System.currentTimeMillis(); Files.asByteSink(new File(outDir, "version.bin")).write(Ints.toByteArray(IndexIO.V9_VERSION)); - log.info("Completed version.bin in %,d millis.", System.currentTimeMillis() - startTime); + log.debug("Completed version.bin in %,d millis.", System.currentTimeMillis() - startTime); progress.progress(); startTime = System.currentTimeMillis(); try (FileOutputStream fos = new FileOutputStream(new File(outDir, "factory.json"))) { mapper.writeValue(fos, new MMappedQueryableSegmentizerFactory(indexIO)); } - log.info("Completed factory.json in %,d millis", System.currentTimeMillis() - startTime); + log.debug("Completed factory.json in %,d millis", System.currentTimeMillis() - startTime); progress.progress(); final Map metricsValueTypes = new TreeMap<>(Comparators.naturalNullsFirst()); @@ -173,7 +173,7 @@ private File makeIndexFiles( progress.progress(); startTime = System.currentTimeMillis(); writeDimValuesAndSetupDimConversion(adapters, progress, mergedDimensions, mergers); - log.info("Completed dim conversions in %,d millis.", System.currentTimeMillis() - startTime); + log.debug("Completed dim conversions in %,d millis.", System.currentTimeMillis() - startTime); /************* Walk through data sets, merge them, and write merged columns *************/ progress.progress(); @@ -311,7 +311,7 @@ private void makeIndexBinary( writer.close(); IndexIO.checkFileSize(new File(outDir, "index.drd")); - log.info("Completed index.drd in %,d millis.", System.currentTimeMillis() - startTime); + log.debug("Completed index.drd in %,d millis.", System.currentTimeMillis() - startTime); progress.stopSection(section); } @@ -365,9 +365,9 @@ private void makeMetricsColumns( throw new ISE("Unknown type[%s]", type); } makeColumn(v9Smoosher, metric, builder.build()); - log.info("Completed metric column[%s] in %,d millis.", metric, System.currentTimeMillis() - metricStartTime); + log.debug("Completed metric column[%s] in %,d millis.", metric, System.currentTimeMillis() - metricStartTime); } - log.info("Completed metric columns in %,d millis.", System.currentTimeMillis() - startTime); + log.debug("Completed metric columns in %,d millis.", System.currentTimeMillis() - startTime); progress.stopSection(section); } @@ -439,7 +439,7 @@ private void makeTimeColumn( .addSerde(createLongColumnPartSerde(timeWriter, indexSpec)) .build(); makeColumn(v9Smoosher, ColumnHolder.TIME_COLUMN_NAME, serdeficator); - log.info("Completed time column in %,d millis.", System.currentTimeMillis() - startTime); + log.debug("Completed time column in %,d millis.", System.currentTimeMillis() - startTime); progress.stopSection(section); } @@ -545,7 +545,7 @@ private List mergeIndexesAndWriteColumns( } if ((++rowCount % 500000) == 0) { - log.info("walked 500,000/%d rows in %,d millis.", rowCount, System.currentTimeMillis() - time); + log.debug("walked 500,000/%d rows in %,d millis.", rowCount, System.currentTimeMillis() - time); time = System.currentTimeMillis(); } } @@ -554,7 +554,7 @@ private List mergeIndexesAndWriteColumns( rowNumConversion.rewind(); } } - log.info("completed walk through of %,d rows in %,d millis.", rowCount, System.currentTimeMillis() - startTime); + log.debug("completed walk through of %,d rows in %,d millis.", rowCount, System.currentTimeMillis() - startTime); progress.stopSection(section); return rowNumConversions; } @@ -776,7 +776,7 @@ public File persist( FileUtils.forceMkdir(outDir); - log.info("Starting persist for interval[%s], rows[%,d]", dataInterval, index.size()); + log.debug("Starting persist for interval[%s], rows[%,d]", dataInterval, index.size()); return merge( Collections.singletonList( new IncrementalIndexAdapter( diff --git a/processing/src/main/java/org/apache/druid/segment/StringDimensionMergerV9.java b/processing/src/main/java/org/apache/druid/segment/StringDimensionMergerV9.java index 63405020498a..2ccc0dd40857 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionMergerV9.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionMergerV9.java @@ -147,7 +147,7 @@ public void writeMergedValueDictionary(List adapters) throws I Indexed[] dimValueLookups = new Indexed[adapters.size() + 1]; for (int i = 0; i < adapters.size(); i++) { @SuppressWarnings("MustBeClosedChecker") // we register dimValues in the closer - Indexed dimValues = closer.register(adapters.get(i).getDimValueLookup(dimensionName)); + Indexed dimValues = closer.register(adapters.get(i).getDimValueLookup(dimensionName)); if (dimValues != null && !allNull(dimValues)) { dimHasValues = true; hasNull |= dimValues.indexOf(null) >= 0; @@ -194,7 +194,7 @@ public void writeMergedValueDictionary(List adapters) throws I cardinality = dimValueLookup.size(); } - log.info( + log.debug( "Completed dim[%s] conversions with cardinality[%,d] in %,d millis.", dimensionName, cardinality, @@ -248,7 +248,10 @@ protected void setupEncodedValueWriter() throws IOException } @Override - public ColumnValueSelector convertSortedSegmentRowValuesToMergedRowValues(int segmentIndex, ColumnValueSelector source) + public ColumnValueSelector convertSortedSegmentRowValuesToMergedRowValues( + int segmentIndex, + ColumnValueSelector source + ) { IntBuffer converter = dimConversions.get(segmentIndex); if (converter == null) { @@ -440,7 +443,7 @@ public void writeIndexes(@Nullable List segmentRowNumConversions) thr spatialWriter.write(ImmutableRTree.newImmutableFromMutable(tree)); } - log.info( + log.debug( "Completed dim[%s] inverted with cardinality[%,d] in %,d millis.", dimensionName, dictionarySize, @@ -547,8 +550,6 @@ public ColumnDescriptor makeColumnDescriptor() .withSpatialIndex(spatialWriter) .withByteOrder(IndexIO.BYTE_ORDER); - //log.info("Completed dimension column[%s] in %,d millis.", dimensionName, System.currentTimeMillis() - dimStartTime); - return builder .addSerde(partBuilder.build()) .build(); diff --git a/processing/src/main/java/org/apache/druid/segment/data/CompressionStrategy.java b/processing/src/main/java/org/apache/druid/segment/data/CompressionStrategy.java index 70c72e5bed98..1d229b51c8ac 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/CompressionStrategy.java +++ b/processing/src/main/java/org/apache/druid/segment/data/CompressionStrategy.java @@ -349,12 +349,12 @@ public ByteBuffer compress(ByteBuffer in, ByteBuffer out) */ private static void logLZ4State() { - LOG.info("java.library.path: " + System.getProperty("java.library.path")); + LOG.debug("java.library.path: " + System.getProperty("java.library.path")); LZ4Factory fastestInstance = LZ4Factory.fastestInstance(); try { //noinspection ObjectEquality if (fastestInstance == LZ4Factory.nativeInstance()) { - LOG.info("LZ4 compression is using native instance."); + LOG.debug("LZ4 compression is using native instance."); } } catch (Throwable t) { @@ -363,7 +363,7 @@ private static void logLZ4State() try { //noinspection ObjectEquality if (fastestInstance == LZ4Factory.unsafeInstance()) { - LOG.info("LZ4 compression is using unsafe instance."); + LOG.debug("LZ4 compression is using unsafe instance."); } } catch (Throwable t) { @@ -372,7 +372,7 @@ private static void logLZ4State() //noinspection ObjectEquality if (fastestInstance == LZ4Factory.safeInstance()) { - LOG.info("LZ4 compression is using safe instance."); + LOG.debug("LZ4 compression is using safe instance."); } } } diff --git a/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java b/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java index ce34da044879..f658cd8aa2c2 100644 --- a/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java +++ b/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java @@ -222,5 +222,13 @@ public int compareTo(final Row o) { return row.compareTo(o); } + + @Override + public String toString() + { + return "TransformedInputRow{" + + "row=" + row + + '}'; + } } } diff --git a/server/src/main/java/org/apache/druid/client/BrokerServerView.java b/server/src/main/java/org/apache/druid/client/BrokerServerView.java index 05b182b214c7..d3f0a6672308 100644 --- a/server/src/main/java/org/apache/druid/client/BrokerServerView.java +++ b/server/src/main/java/org/apache/druid/client/BrokerServerView.java @@ -164,10 +164,10 @@ public CallbackAction segmentViewInitialized() public void start() throws InterruptedException { if (segmentWatcherConfig.isAwaitInitializationOnStart()) { - final long startMillis = System.currentTimeMillis(); - log.info("%s waiting for initialization.", getClass().getSimpleName()); + final long startNanos = System.nanoTime(); + log.debug("%s waiting for initialization.", getClass().getSimpleName()); awaitInitialization(); - log.info("%s initialized in [%,d] ms.", getClass().getSimpleName(), System.currentTimeMillis() - startMillis); + log.info("%s initialized in [%,d] ms.", getClass().getSimpleName(), (System.nanoTime() - startNanos) / 1000000); } } diff --git a/server/src/main/java/org/apache/druid/curator/CuratorModule.java b/server/src/main/java/org/apache/druid/curator/CuratorModule.java index 3095991d8bc2..d5f516b898b3 100644 --- a/server/src/main/java/org/apache/druid/curator/CuratorModule.java +++ b/server/src/main/java/org/apache/druid/curator/CuratorModule.java @@ -56,7 +56,7 @@ public class CuratorModule implements Module private static final int MAX_SLEEP_TIME_MS = 45000; - private static final int MAX_RETRIES = 30; + private static final int MAX_RETRIES = 29; private static final Logger log = new Logger(CuratorModule.class); @@ -92,7 +92,7 @@ public CuratorFramework makeCurator(CuratorConfig config, EnsembleProvider ensem .build(); framework.getUnhandledErrorListenable().addListener((message, e) -> { - log.error(e, "Unhandled error in Curator Framework"); + log.error(e, "Unhandled error in Curator, stopping server."); shutdown(lifecycle); }); @@ -102,14 +102,14 @@ public CuratorFramework makeCurator(CuratorConfig config, EnsembleProvider ensem @Override public void start() { - log.info("Starting Curator"); + log.debug("Starting Curator"); framework.start(); } @Override public void stop() { - log.info("Stopping Curator"); + log.debug("Stopping Curator"); framework.close(); } } @@ -143,7 +143,7 @@ public EnsembleProvider makeEnsembleProvider(CuratorConfig config, ExhibitorConf @Override public void start() throws Exception { - log.info("Poll the list of zookeeper servers for initial ensemble"); + log.debug("Polling the list of ZooKeeper servers for the initial ensemble"); this.pollForInitialEnsemble(); super.start(); } @@ -177,7 +177,7 @@ private void shutdown(Lifecycle lifecycle) lifecycle.stop(); } catch (Throwable t) { - log.error(t, "Exception when stopping druid lifecycle"); + log.error(t, "Exception when stopping server after unhandled Curator error."); } finally { System.exit(1); diff --git a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java index 80e29c13fc3d..533389a02c3e 100644 --- a/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java +++ b/server/src/main/java/org/apache/druid/curator/announcement/Announcer.java @@ -104,7 +104,7 @@ Set getAddedChildren() @LifecycleStart public void start() { - log.info("Starting announcer"); + log.debug("Starting Announcer."); synchronized (toAnnounce) { if (started) { return; @@ -127,7 +127,7 @@ public void start() @LifecycleStop public void stop() { - log.info("Stopping announcer"); + log.debug("Stopping Announcer."); synchronized (toAnnounce) { if (!started) { return; @@ -395,13 +395,13 @@ public void unannounce(String path) log.debug("Path[%s] not announced, cannot unannounce.", path); return; } - log.info("unannouncing [%s]", path); + log.info("Unannouncing [%s]", path); try { curator.inTransaction().delete().forPath(path).and().commit(); } catch (KeeperException.NoNodeException e) { - log.info("node[%s] didn't exist anyway...", path); + log.info("Node[%s] didn't exist anyway...", path); } catch (Exception e) { throw new RuntimeException(e); diff --git a/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncer.java index ab27916907d3..5529d90b20d3 100644 --- a/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncer.java +++ b/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncer.java @@ -27,11 +27,10 @@ import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.DruidNodeAnnouncer; import org.apache.druid.guice.annotations.Json; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.server.initialization.ZkPathsConfig; -/** - */ public class CuratorDruidNodeAnnouncer implements DruidNodeAnnouncer { private static final Logger log = new Logger(CuratorDruidNodeAnnouncer.class); @@ -52,16 +51,18 @@ public CuratorDruidNodeAnnouncer(Announcer announcer, ZkPathsConfig config, @Jso public void announce(DiscoveryDruidNode discoveryDruidNode) { try { - log.info("Announcing [%s].", discoveryDruidNode); + final String asString = jsonMapper.writeValueAsString(discoveryDruidNode); + + log.debug("Announcing self [%s].", asString); String path = ZKPaths.makePath( config.getInternalDiscoveryPath(), discoveryDruidNode.getNodeType().toString(), discoveryDruidNode.getDruidNode().getHostAndPortToUse() ); - announcer.announce(path, jsonMapper.writeValueAsBytes(discoveryDruidNode)); + announcer.announce(path, StringUtils.toUtf8(asString)); - log.info("Announced [%s].", discoveryDruidNode); + log.info("Announced self [%s].", asString); } catch (JsonProcessingException e) { throw new RuntimeException(e); @@ -71,15 +72,22 @@ public void announce(DiscoveryDruidNode discoveryDruidNode) @Override public void unannounce(DiscoveryDruidNode discoveryDruidNode) { - log.info("Unannouncing [%s].", discoveryDruidNode); + try { + final String asString = jsonMapper.writeValueAsString(discoveryDruidNode); - String path = ZKPaths.makePath( - config.getInternalDiscoveryPath(), - discoveryDruidNode.getNodeType().toString(), - discoveryDruidNode.getDruidNode().getHostAndPortToUse() - ); - announcer.unannounce(path); + log.debug("Unannouncing self [%s].", asString); - log.info("Unannounced [%s].", discoveryDruidNode); + String path = ZKPaths.makePath( + config.getInternalDiscoveryPath(), + discoveryDruidNode.getNodeType().toString(), + discoveryDruidNode.getDruidNode().getHostAndPortToUse() + ); + announcer.unannounce(path); + + log.info("Unannounced self [%s].", asString); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } } } diff --git a/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java b/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java index 93903de6c96e..f4235a288444 100644 --- a/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java +++ b/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java @@ -54,6 +54,7 @@ import java.util.concurrent.TimeUnit; /** + * */ @ManageLifecycle public class CuratorDruidNodeDiscoveryProvider extends DruidNodeDiscoveryProvider @@ -90,7 +91,7 @@ public DruidNodeDiscovery getForNodeType(NodeType nodeType) return nodeTypeWatchers.computeIfAbsent( nodeType, nType -> { - log.info("Creating NodeTypeWatcher for nodeType [%s].", nType); + log.debug("Creating NodeTypeWatcher for nodeType [%s].", nType); NodeTypeWatcher nodeTypeWatcher = new NodeTypeWatcher( listenerExecutor, curatorFramework, @@ -99,7 +100,7 @@ public DruidNodeDiscovery getForNodeType(NodeType nodeType) nType ); nodeTypeWatcher.start(); - log.info("Created NodeTypeWatcher for nodeType [%s].", nType); + log.debug("Created NodeTypeWatcher for nodeType [%s].", nType); return nodeTypeWatcher; } ); @@ -113,13 +114,11 @@ public void start() } try { - log.info("starting"); - - // This is single-threaded to ensure that all listener calls are executed precisely in the oder of add/remove - // event occurences. + // This is single-threaded to ensure that all listener calls are executed precisely in the order of add/remove + // event occurrences. listenerExecutor = Execs.singleThreaded("CuratorDruidNodeDiscoveryProvider-ListenerExecutor"); - log.info("started"); + log.debug("Started."); lifecycleLock.started(); } @@ -135,14 +134,12 @@ public void stop() throw new ISE("can't stop."); } - log.info("stopping"); + log.debug("Stopping."); for (NodeTypeWatcher watcher : nodeTypeWatchers.values()) { watcher.stop(); } listenerExecutor.shutdownNow(); - - log.info("stopped"); } private static class NodeTypeWatcher implements DruidNodeDiscovery @@ -154,7 +151,9 @@ private static class NodeTypeWatcher implements DruidNodeDiscovery private final NodeType nodeType; private final ObjectMapper jsonMapper; - /** hostAndPort -> DiscoveryDruidNode */ + /** + * hostAndPort -> DiscoveryDruidNode + */ private final ConcurrentMap nodes = new ConcurrentHashMap<>(); private final Collection unmodifiableNodes = Collections.unmodifiableCollection(nodes.values()); @@ -205,7 +204,10 @@ public Collection getAllNodes() nodeViewInitialized = false; } if (!nodeViewInitialized) { - log.info("cache is not initialized yet. getAllNodes() might not return full information."); + log.info( + "Cache for process type [%s] not initialized yet; getAllNodes() might not return full information.", + nodeType.getJsonName() + ); } return unmodifiableNodes; } @@ -239,7 +241,7 @@ void handleChildEvent(PathChildrenCacheEvent event) data = curatorFramework.getData().decompressed().forPath(event.getData().getPath()); } catch (Exception ex) { - log.error( + log.noStackTrace().error( ex, "Failed to get data for path [%s]. Ignoring event [%s].", event.getData().getPath(), @@ -252,14 +254,19 @@ void handleChildEvent(PathChildrenCacheEvent event) if (!nodeType.equals(druidNode.getNodeType())) { log.warn( - "Node[%s:%s] add is discovered by node watcher of different node type. Ignored.", - druidNode.getDruidNode().getHostAndPortToUse(), - druidNode + "Node[%s] of type[%s] addition ignored due to mismatched type (expected type[%s]).", + druidNode.getDruidNode().getUriToUse(), + druidNode.getNodeType().getJsonName(), + nodeType.getJsonName() ); return; } - log.info("Node[%s:%s] appeared.", druidNode.getDruidNode().getHostAndPortToUse(), druidNode); + log.info( + "Node[%s] of type[%s] detected.", + druidNode.getDruidNode().getUriToUse(), + nodeType.getJsonName() + ); addNode(druidNode); @@ -270,17 +277,18 @@ void handleChildEvent(PathChildrenCacheEvent event) if (!nodeType.equals(druidNode.getNodeType())) { log.warn( - "Node[%s:%s] removal is discovered by node watcher of different type. Ignored.", - druidNode.getDruidNode().getHostAndPortToUse(), - druidNode + "Node[%s] of type[%s] removal ignored due to mismatched type (expected type[%s]).", + druidNode.getDruidNode().getUriToUse(), + druidNode.getNodeType().getJsonName(), + nodeType.getJsonName() ); return; } log.info( - "Node[%s:%s] disappeared.", - druidNode.getDruidNode().getHostAndPortToUse(), - druidNode + "Node[%s] of type[%s] went offline.", + druidNode.getDruidNode().getUriToUse(), + nodeType.getJsonName() ); removeNode(druidNode); @@ -295,7 +303,7 @@ void handleChildEvent(PathChildrenCacheEvent event) return; } - log.info("Received INITIALIZED in node watcher."); + log.info("Node watcher of type[%s] is now initialized.", nodeType.getJsonName()); for (Listener listener : nodeListeners) { safeSchedule( @@ -313,12 +321,12 @@ void handleChildEvent(PathChildrenCacheEvent event) break; } default: { - log.info("Ignored event type [%s] for nodeType watcher.", event.getType()); + log.warn("Ignored event type[%s] for node watcher of type[%s].", event.getType(), nodeType.getJsonName()); } } } catch (Exception ex) { - log.error(ex, "unknown error in node watcher."); + log.error(ex, "Unknown error in node watcher of type[%s].", nodeType.getJsonName()); } } } diff --git a/server/src/main/java/org/apache/druid/curator/inventory/CuratorInventoryManager.java b/server/src/main/java/org/apache/druid/curator/inventory/CuratorInventoryManager.java index bf6c7c670006..9032bca4fe98 100644 --- a/server/src/main/java/org/apache/druid/curator/inventory/CuratorInventoryManager.java +++ b/server/src/main/java/org/apache/druid/curator/inventory/CuratorInventoryManager.java @@ -228,7 +228,7 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) th byte[] data = getZkDataForNode(child.getPath()); if (data == null) { - log.info("Ignoring event: Type - %s , Path - %s , Version - %s", + log.warn("Ignoring event: Type - %s , Path - %s , Version - %s", event.getType(), child.getPath(), child.getStat().getVersion()); @@ -287,7 +287,7 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) th byte[] data = getZkDataForNode(child.getPath()); if (data == null) { - log.info( + log.warn( "Ignoring event: Type - %s , Path - %s , Version - %s", event.getType(), child.getPath(), @@ -363,7 +363,7 @@ public InventoryCacheListener(String containerKey, String inventoryPath) this.containerKey = containerKey; this.inventoryPath = inventoryPath; - log.info("Created new InventoryCacheListener for %s", inventoryPath); + log.debug("Created new InventoryCacheListener for %s", inventoryPath); } @Override @@ -380,7 +380,7 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) byte[] data = getZkDataForNode(child.getPath()); if (data == null) { - log.info("Ignoring event: Type - %s , Path - %s , Version - %s", + log.warn("Ignoring event: Type - %s , Path - %s , Version - %s", event.getType(), child.getPath(), child.getStat().getVersion()); @@ -403,7 +403,7 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) byte[] data = getZkDataForNode(child.getPath()); if (data == null) { - log.info("Ignoring event: Type - %s , Path - %s , Version - %s", + log.warn("Ignoring event: Type - %s , Path - %s , Version - %s", event.getType(), child.getPath(), child.getStat().getVersion()); diff --git a/server/src/main/java/org/apache/druid/discovery/DruidLeaderClient.java b/server/src/main/java/org/apache/druid/discovery/DruidLeaderClient.java index efda96c8fc48..7fb42f55998b 100644 --- a/server/src/main/java/org/apache/druid/discovery/DruidLeaderClient.java +++ b/server/src/main/java/org/apache/druid/discovery/DruidLeaderClient.java @@ -104,7 +104,7 @@ public void start() try { druidNodeDiscovery = druidNodeDiscoveryProvider.getForNodeType(nodeTypeToWatch); lifecycleLock.started(); - log.info("Started."); + log.debug("Started."); } finally { lifecycleLock.exitStart(); @@ -118,7 +118,7 @@ public void stop() throw new ISE("can't stop."); } - log.info("Stopped."); + log.debug("Stopped."); } /** diff --git a/server/src/main/java/org/apache/druid/initialization/Initialization.java b/server/src/main/java/org/apache/druid/initialization/Initialization.java index 760d9ad78369..8b417db016f2 100644 --- a/server/src/main/java/org/apache/druid/initialization/Initialization.java +++ b/server/src/main/java/org/apache/druid/initialization/Initialization.java @@ -75,6 +75,7 @@ import java.net.URL; import java.net.URLClassLoader; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashSet; @@ -84,8 +85,10 @@ import java.util.ServiceLoader; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; /** + * */ public class Initialization { @@ -173,12 +176,21 @@ private void addAllFromCurrentClassLoader() private void addAllFromFileSystem() { for (File extension : getExtensionFilesToLoad(extensionsConfig)) { - log.info("Loading extension [%s] for class [%s]", extension.getName(), serviceClass); + log.debug("Loading extension [%s] for class [%s]", extension.getName(), serviceClass); try { final URLClassLoader loader = getClassLoaderForExtension( extension, extensionsConfig.isUseExtensionClassloaderFirst() ); + + log.info( + "Loading extension [%s], jars: %s", + extension.getName(), + Arrays.stream(loader.getURLs()) + .map(u -> new File(u.getPath()).getName()) + .collect(Collectors.joining(", ")) + ); + ServiceLoader.load(serviceClass, loader).forEach(impl -> tryAdd(impl, "local file system")); } catch (Exception e) { @@ -197,7 +209,7 @@ private void tryAdd(T serviceImpl, String extensionType) serviceImpl.getClass().getName() ); } else if (!implClassNamesToLoad.contains(serviceImplName)) { - log.info( + log.debug( "Adding implementation [%s] for class [%s] from %s extension", serviceImplName, serviceClass, @@ -310,7 +322,7 @@ private static URLClassLoader makeClassLoaderForExtension( int i = 0; for (File jar : jars) { final URL url = jar.toURI().toURL(); - log.info("added URL[%s] for extension[%s]", url, extension.getName()); + log.debug("added URL[%s] for extension[%s]", url, extension.getName()); urls[i++] = url; } } diff --git a/server/src/main/java/org/apache/druid/query/lookup/LookupReferencesManager.java b/server/src/main/java/org/apache/druid/query/lookup/LookupReferencesManager.java index 5e50bd37b3bb..39b3613bc9b6 100644 --- a/server/src/main/java/org/apache/druid/query/lookup/LookupReferencesManager.java +++ b/server/src/main/java/org/apache/druid/query/lookup/LookupReferencesManager.java @@ -152,7 +152,7 @@ public void start() throw new ISE("can't start."); } try { - LOG.info("LookupExtractorFactoryContainerProvider is starting."); + LOG.debug("LookupExtractorFactoryContainerProvider starting."); loadAllLookupsAndInitStateRef(); if (!testMode) { mainThread = Execs.makeThread( @@ -178,7 +178,7 @@ public void start() LOG.error(t, "Error while waiting for lifecycle start. lookup updates notices will not be handled"); } finally { - LOG.info("Lookup Management loop exited, Lookup notices are not handled anymore."); + LOG.info("Lookup Management loop exited. Lookup notices are not handled anymore."); } }, true @@ -187,7 +187,7 @@ public void start() mainThread.start(); } - LOG.info("LookupExtractorFactoryContainerProvider is started."); + LOG.debug("LookupExtractorFactoryContainerProvider started."); lifecycleLock.started(); } finally { @@ -234,7 +234,7 @@ public void stop() throw new ISE("can't stop."); } - LOG.info("LookupExtractorFactoryContainerProvider is stopping."); + LOG.debug("LookupExtractorFactoryContainerProvider is stopping."); if (!testMode) { mainThread.interrupt(); @@ -249,8 +249,9 @@ public void stop() for (Map.Entry e : stateRef.get().lookupMap.entrySet()) { try { - LOG.info("Closing lookup [%s]", e.getKey()); - if (!e.getValue().getLookupExtractorFactory().close()) { + if (e.getValue().getLookupExtractorFactory().close()) { + LOG.info("Closed lookup [%s].", e.getKey()); + } else { LOG.error("Failed to close lookup [%s].", e.getKey()); } } @@ -259,7 +260,7 @@ public void stop() } } - LOG.info("LookupExtractorFactoryContainerProvider is stopped."); + LOG.debug("LookupExtractorFactoryContainerProvider is stopped."); } @@ -351,7 +352,7 @@ private void loadAllLookupsAndInitStateRef() if (lookupBeanList != null) { startLookups(lookupBeanList); } else { - LOG.info("No lookups to be loaded at this point"); + LOG.debug("No lookups to be loaded at this point."); stateRef.set(new LookupUpdateState(ImmutableMap.of(), ImmutableList.of(), ImmutableList.of())); } } @@ -481,7 +482,7 @@ private void startLookups(final List lookupBeanList) new ExecutorCompletionService<>(executorService); final List remainingLookups = new ArrayList<>(lookupBeanList); try { - LOG.info("Starting lookup loading process"); + LOG.info("Starting lookup loading process."); for (int i = 0; i < lookupConfig.getLookupStartRetries() && !remainingLookups.isEmpty(); i++) { LOG.info("Round of attempts #%d, [%d] lookups", i + 1, remainingLookups.size()); final Map successfulLookups = diff --git a/server/src/main/java/org/apache/druid/segment/loading/LocalDataSegmentKiller.java b/server/src/main/java/org/apache/druid/segment/loading/LocalDataSegmentKiller.java index 0c021094bf3f..da7068177f62 100644 --- a/server/src/main/java/org/apache/druid/segment/loading/LocalDataSegmentKiller.java +++ b/server/src/main/java/org/apache/druid/segment/loading/LocalDataSegmentKiller.java @@ -48,7 +48,7 @@ public LocalDataSegmentKiller(LocalDataSegmentPusherConfig config) public void kill(DataSegment segment) throws SegmentLoadingException { final File path = getPath(segment); - log.info("killing segment[%s] mapped to path[%s]", segment.getId(), path); + log.info("Deleting segment[%s] from directory[%s].", segment.getId(), path); try { if (path.getName().endsWith(".zip")) { @@ -80,7 +80,7 @@ public void kill(DataSegment segment) throws SegmentLoadingException @Override public void killAll() throws IOException { - log.info("Deleting all segment files from local dir [%s].", storageDirectory.getAbsolutePath()); + log.info("Deleting all segments from directory[%s].", storageDirectory.getAbsolutePath()); FileUtils.deleteDirectory(storageDirectory); } diff --git a/server/src/main/java/org/apache/druid/segment/loading/LocalDataSegmentPusher.java b/server/src/main/java/org/apache/druid/segment/loading/LocalDataSegmentPusher.java index 615f931094e8..455ddf7e2ad9 100644 --- a/server/src/main/java/org/apache/druid/segment/loading/LocalDataSegmentPusher.java +++ b/server/src/main/java/org/apache/druid/segment/loading/LocalDataSegmentPusher.java @@ -46,8 +46,6 @@ public class LocalDataSegmentPusher implements DataSegmentPusher public LocalDataSegmentPusher(LocalDataSegmentPusherConfig config) { this.config = config; - - log.info("Configured local filesystem as deep storage"); } @Override @@ -70,7 +68,7 @@ public DataSegment push(final File dataSegmentFile, final DataSegment segment, f final File baseStorageDir = config.getStorageDirectory(); final File outDir = new File(baseStorageDir, this.getStorageDir(segment, useUniquePath)); - log.info("Copying segment[%s] to local filesystem at location[%s]", segment.getId(), outDir.toString()); + log.debug("Copying segment[%s] to local filesystem at location[%s]", segment.getId(), outDir.toString()); if (dataSegmentFile.equals(outDir)) { long size = 0; @@ -84,7 +82,7 @@ public DataSegment push(final File dataSegmentFile, final DataSegment segment, f } final File tmpOutDir = new File(baseStorageDir, makeIntermediateDir()); - log.info("Creating intermediate directory[%s] for segment[%s]", tmpOutDir.toString(), segment.getId()); + log.debug("Creating intermediate directory[%s] for segment[%s].", tmpOutDir.toString(), segment.getId()); FileUtils.forceMkdir(tmpOutDir); try { @@ -122,7 +120,7 @@ private String makeIntermediateDir() private long compressSegment(File dataSegmentFile, File dest) throws IOException { - log.info("Compressing files from[%s] to [%s]", dataSegmentFile, dest); + log.debug("Compressing files from[%s] to [%s]", dataSegmentFile, dest); return CompressionUtils.zip(dataSegmentFile, dest, true); } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/FireHydrant.java b/server/src/main/java/org/apache/druid/segment/realtime/FireHydrant.java index 097d412a6c0e..fbd6e8d39885 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/FireHydrant.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/FireHydrant.java @@ -145,7 +145,7 @@ public String toString() // Do not include IncrementalIndex in toString as AbstractIndex.toString() actually prints // all the rows in the index return "FireHydrant{" + - ", queryable=" + adapter.get().getId() + + "queryable=" + adapter.get().getId() + ", count=" + count + '}'; } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java index ce5ac8f89d81..7f5c595d81a3 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java @@ -47,6 +47,11 @@ */ public interface Appenderator extends QuerySegmentWalker { + /** + * Return the identifier of this Appenderator; useful for log messages and such. + */ + String getId(); + /** * Return the name of the dataSource associated with this Appenderator. */ @@ -181,8 +186,8 @@ AppenderatorAddResult add( *

* If committer is not provided, no metadata is persisted. * - * @param identifiers list of segments to push - * @param committer a committer associated with all data that has been added so far + * @param identifiers list of segments to push + * @param committer a committer associated with all data that has been added so far * @param useUniquePath true if the segment should be written to a path with a unique identifier * * @return future that resolves when all segments have been pushed. The segment list will be the list of segments diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java index 44d5bf347d1b..f3a75c520848 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -39,7 +39,6 @@ import com.google.common.util.concurrent.MoreExecutors; import org.apache.commons.io.FileUtils; import org.apache.druid.client.cache.Cache; -import org.apache.druid.common.guava.ThreadRenamingCallable; import org.apache.druid.data.input.Committer; import org.apache.druid.data.input.InputRow; import org.apache.druid.java.util.common.DateTimes; @@ -89,6 +88,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; @@ -100,14 +100,13 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; -/** - */ public class AppenderatorImpl implements Appenderator { private static final EmittingLogger log = new EmittingLogger(AppenderatorImpl.class); private static final int WARN_DELAY = 1000; private static final String IDENTIFIER_FILE_NAME = "identifier.json"; + private final String myId; private final DataSchema schema; private final AppenderatorConfig tuningConfig; private final boolean storeCompactionState; @@ -162,6 +161,7 @@ public class AppenderatorImpl implements Appenderator * Appenderators. */ AppenderatorImpl( + String id, DataSchema schema, AppenderatorConfig tuningConfig, boolean storeCompactionState, @@ -175,6 +175,7 @@ public class AppenderatorImpl implements Appenderator Cache cache ) { + this.myId = id; this.schema = Preconditions.checkNotNull(schema, "schema"); this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig"); this.storeCompactionState = storeCompactionState; @@ -196,7 +197,12 @@ public class AppenderatorImpl implements Appenderator } maxBytesTuningConfig = TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()); - log.info("Created Appenderator for dataSource[%s].", schema.getDataSource()); + } + + @Override + public String getId() + { + return myId; } @Override @@ -306,7 +312,7 @@ public AppenderatorAddResult add( if (persist) { if (allowIncrementalPersists) { // persistAll clears rowsCurrentlyInMemory, no need to update it. - log.info("Persisting rows in memory due to: [%s]", String.join(",", persistReasons)); + log.info("Flushing in-memory data to disk because %s.", String.join(",", persistReasons)); Futures.addCallback( persistAll(committerSupplier == null ? null : committerSupplier.get()), new FutureCallback() @@ -512,7 +518,7 @@ public ListenableFuture persistAll(@Nullable final Committer committer) for (FireHydrant hydrant : hydrants.subList(0, limit)) { if (!hydrant.hasSwapped()) { - log.info("Hydrant[%s] hasn't persisted yet, persisting. Segment[%s]", hydrant, identifier); + log.debug("Hydrant[%s] hasn't persisted yet, persisting. Segment[%s]", hydrant, identifier); indexesToPersist.add(Pair.of(hydrant, identifier)); } } @@ -522,17 +528,16 @@ public ListenableFuture persistAll(@Nullable final Committer committer) } } - log.info("Submitting persist runnable for dataSource[%s]", schema.getDataSource()); + log.debug("Submitting persist runnable for dataSource[%s]", schema.getDataSource()); - final String threadName = StringUtils.format("%s-incremental-persist", schema.getDataSource()); final Object commitMetadata = committer == null ? null : committer.getMetadata(); final Stopwatch runExecStopwatch = Stopwatch.createStarted(); final Stopwatch persistStopwatch = Stopwatch.createStarted(); final ListenableFuture future = persistExecutor.submit( - new ThreadRenamingCallable(threadName) + new Callable() { @Override - public Object doCall() throws IOException + public Object call() throws IOException { try { for (Pair pair : indexesToPersist) { @@ -540,8 +545,10 @@ public Object doCall() throws IOException } if (committer != null) { - log.info( - "Committing metadata[%s] for sinks[%s].", commitMetadata, Joiner.on(", ").join( + log.debug( + "Committing metadata[%s] for sinks[%s].", + commitMetadata, + Joiner.on(", ").join( currentHydrants.entrySet() .stream() .map(entry -> StringUtils.format( @@ -571,6 +578,15 @@ public Object doCall() throws IOException } } + log.info( + "Flushed in-memory data with commit metadata [%s] for segments: %s", + commitMetadata, + indexesToPersist.stream() + .map(itp -> itp.rhs.asSegmentId().toString()) + .distinct() + .collect(Collectors.joining(", ")) + ); + // return null if committer is null return commitMetadata; } @@ -627,9 +643,14 @@ public ListenableFuture push( (Function) commitMetadata -> { final List dataSegments = new ArrayList<>(); + log.debug( + "Building and pushing segments: %s", + theSinks.keySet().stream().map(SegmentIdWithShardSpec::toString).collect(Collectors.joining(", ")) + ); + for (Map.Entry entry : theSinks.entrySet()) { if (droppingSinks.contains(entry.getKey())) { - log.info("Skipping push of currently-dropping sink[%s]", entry.getKey()); + log.warn("Skipping push of currently-dropping sink[%s]", entry.getKey()); continue; } @@ -673,9 +694,15 @@ private ListenableFuture pushBarrier() * * @return segment descriptor, or null if the sink is no longer valid */ - private DataSegment mergeAndPush(final SegmentIdWithShardSpec identifier, final Sink sink, final boolean useUniquePath) + @Nullable + private DataSegment mergeAndPush( + final SegmentIdWithShardSpec identifier, + final Sink sink, + final boolean useUniquePath + ) { // Bail out if this sink is null or otherwise not what we expect. + //noinspection ObjectEquality if (sinks.get(identifier) != sink) { log.warn("Sink for segment[%s] no longer valid, bailing out of mergeAndPush.", identifier); return null; @@ -706,15 +733,16 @@ private DataSegment mergeAndPush(final SegmentIdWithShardSpec identifier, final if (useUniquePath) { // Don't reuse the descriptor, because the caller asked for a unique path. Leave the old one as-is, since // it might serve some unknown purpose. - log.info("Pushing segment[%s] again with new unique path.", identifier); + log.debug( + "Segment[%s] already pushed, but we want a unique path, so will push again with a new path.", + identifier + ); } else { - log.info("Segment[%s] already pushed.", identifier); + log.info("Segment[%s] already pushed, skipping.", identifier); return objectMapper.readValue(descriptorFile, DataSegment.class); } } - log.info("Pushing merged index for segment[%s].", identifier); - removeDirectory(mergedTarget); if (mergedTarget.exists()) { @@ -722,13 +750,15 @@ private DataSegment mergeAndPush(final SegmentIdWithShardSpec identifier, final } final File mergedFile; + final long mergeFinishTime; + final long startTime = System.nanoTime(); List indexes = new ArrayList<>(); Closer closer = Closer.create(); try { for (FireHydrant fireHydrant : sink) { Pair segmentAndCloseable = fireHydrant.getAndIncrementSegment(); final QueryableIndex queryableIndex = segmentAndCloseable.lhs.asQueryableIndex(); - log.info("Adding hydrant[%s]", fireHydrant); + log.debug("Segment[%s] adding hydrant[%s]", identifier, fireHydrant); indexes.add(queryableIndex); closer.register(segmentAndCloseable.rhs); } @@ -741,6 +771,10 @@ private DataSegment mergeAndPush(final SegmentIdWithShardSpec identifier, final tuningConfig.getIndexSpec(), tuningConfig.getSegmentWriteOutMediumFactory() ); + + mergeFinishTime = System.nanoTime(); + + log.debug("Segment[%s] built in %,dms.", identifier, (mergeFinishTime - startTime) / 1000000); } catch (Throwable t) { throw closer.rethrow(t); @@ -763,9 +797,22 @@ private DataSegment mergeAndPush(final SegmentIdWithShardSpec identifier, final 5 ); + final long pushFinishTime = System.nanoTime(); + objectMapper.writeValue(descriptorFile, segment); - log.info("Pushed merged index for segment[%s], descriptor is: %s", identifier, segment); + log.info( + "Segment[%s] of %,d bytes " + + "built from %d incremental persist(s) in %,dms; " + + "pushed to deep storage in %,dms. " + + "Load spec is: %s", + identifier, + segment.getSize(), + indexes.size(), + (mergeFinishTime - startTime) / 1000000, + (pushFinishTime - mergeFinishTime) / 1000000, + objectMapper.writeValueAsString(segment.getLoadSpec()) + ); return segment; } @@ -780,11 +827,11 @@ private DataSegment mergeAndPush(final SegmentIdWithShardSpec identifier, final public void close() { if (!closed.compareAndSet(false, true)) { - log.info("Appenderator already closed"); + log.debug("Appenderator already closed, skipping close() call."); return; } - log.info("Shutting down..."); + log.debug("Shutting down..."); final List> futures = new ArrayList<>(); for (Map.Entry entry : sinks.entrySet()) { @@ -841,11 +888,11 @@ public void close() public void closeNow() { if (!closed.compareAndSet(false, true)) { - log.info("Appenderator already closed"); + log.debug("Appenderator already closed, skipping closeNow() call."); return; } - log.info("Shutting down immediately..."); + log.debug("Shutting down immediately..."); for (Map.Entry entry : sinks.entrySet()) { try { segmentAnnouncer.unannounceSegment(entry.getValue().getSegment()); @@ -918,25 +965,21 @@ private void initializeExecutors() if (persistExecutor == null) { // use a blocking single threaded executor to throttle the firehose when write to disk is slow persistExecutor = MoreExecutors.listeningDecorator( - Execs.newBlockingSingleThreaded( - "appenderator_persist_%d", maxPendingPersists - ) + Execs.newBlockingSingleThreaded("[" + myId + "]-appenderator-persist", maxPendingPersists) ); } + if (pushExecutor == null) { // use a blocking single threaded executor to throttle the firehose when write to disk is slow pushExecutor = MoreExecutors.listeningDecorator( - Execs.newBlockingSingleThreaded( - "appenderator_merge_%d", 1 - ) + Execs.newBlockingSingleThreaded("[" + myId + "]-appenderator-merge", 1) ); } + if (intermediateTempExecutor == null) { // use single threaded executor with SynchronousQueue so that all abandon operations occur sequentially intermediateTempExecutor = MoreExecutors.listeningDecorator( - Execs.newBlockingSingleThreaded( - "appenderator_abandon_%d", 0 - ) + Execs.newBlockingSingleThreaded("[" + myId + "]-appenderator-abandon", 0) ); } } @@ -946,9 +989,11 @@ private void shutdownExecutors() if (persistExecutor != null) { persistExecutor.shutdownNow(); } + if (pushExecutor != null) { pushExecutor.shutdownNow(); } + if (intermediateTempExecutor != null) { intermediateTempExecutor.shutdownNow(); } @@ -999,7 +1044,16 @@ private Object bootstrapSinksFromDisk() int rowsSoFar = 0; - log.info("Loading sinks from[%s]: %s", baseDir, committed.getHydrants().keySet()); + if (committed.equals(Committed.nil())) { + log.debug("No previously committed metadata."); + } else { + log.info( + "Loading partially-persisted segments[%s] from[%s] with commit metadata: %s", + String.join(", ", committed.getHydrants().keySet()), + baseDir, + committed.getMetadata() + ); + } for (File sinkDir : files) { final File identifierFile = new File(sinkDir, IDENTIFIER_FILE_NAME); @@ -1017,7 +1071,7 @@ private Object bootstrapSinksFromDisk() final int committedHydrants = committed.getCommittedHydrants(identifier.toString()); if (committedHydrants <= 0) { - log.info("Removing uncommitted sink at [%s]", sinkDir); + log.info("Removing uncommitted segment at [%s].", sinkDir); FileUtils.deleteDirectory(sinkDir); continue; } @@ -1037,10 +1091,10 @@ private Object bootstrapSinksFromDisk() final int hydrantNumber = Integer.parseInt(hydrantDir.getName()); if (hydrantNumber >= committedHydrants) { - log.info("Removing uncommitted segment at [%s]", hydrantDir); + log.info("Removing uncommitted partial segment at [%s]", hydrantDir); FileUtils.deleteDirectory(hydrantDir); } else { - log.info("Loading previously persisted segment at [%s]", hydrantDir); + log.debug("Loading previously persisted partial segment at [%s]", hydrantDir); if (hydrantNumber != hydrants.size()) { throw new ISE("Missing hydrant [%,d] in sinkDir [%s].", hydrants.size(), sinkDir); } @@ -1132,12 +1186,12 @@ public Void apply(@Nullable Object input) log.error("Sink for segment[%s] no longer valid, not abandoning.", identifier); return null; } - log.info("Removing sink for segment[%s].", identifier); + metrics.setSinkCount(sinks.size()); if (removeOnDiskData) { // Remove this segment from the committed list. This must be done from the persist thread. - log.info("Removing commit metadata for segment[%s].", identifier); + log.debug("Removing commit metadata for segment[%s].", identifier); try { commitLock.lock(); final Committed oldCommit = readCommit(); @@ -1183,6 +1237,8 @@ public Void apply(@Nullable Object input) removeDirectory(computePersistDir(identifier)); } + log.info("Dropped segment[%s].", identifier); + return null; } }, @@ -1258,15 +1314,16 @@ private int persistHydrant(FireHydrant indexToPersist, SegmentIdWithShardSpec id synchronized (indexToPersist) { if (indexToPersist.hasSwapped()) { log.info( - "Segment[%s], Hydrant[%s] already swapped. Ignoring request to persist.", + "Segment[%s] hydrant[%s] already swapped. Ignoring request to persist.", identifier, indexToPersist ); return 0; } - log.info("Segment[%s], persisting Hydrant[%s]", identifier, indexToPersist); + log.debug("Segment[%s], persisting Hydrant[%s]", identifier, indexToPersist); try { + final long startTime = System.nanoTime(); int numRows = indexToPersist.getIndex().size(); final File persistedFile; @@ -1279,14 +1336,24 @@ private int persistHydrant(FireHydrant indexToPersist, SegmentIdWithShardSpec id tuningConfig.getSegmentWriteOutMediumFactory() ); + log.info( + "Flushed in-memory data for segment[%s] spill[%s] to disk in [%,d] ms (%,d rows).", + indexToPersist.getSegmentId(), + indexToPersist.getCount(), + (System.nanoTime() - startTime) / 1000000, + numRows + ); + indexToPersist.swapSegment( new QueryableIndexSegment(indexIO.loadIndex(persistedFile), indexToPersist.getSegmentId()) ); + return numRows; } catch (IOException e) { - log.makeAlert("dataSource[%s] -- incremental persist failed", schema.getDataSource()) + log.makeAlert("Incremental persist failed") .addData("segment", identifier.toString()) + .addData("dataSource", schema.getDataSource()) .addData("count", indexToPersist.getCount()) .emit(); @@ -1299,7 +1366,6 @@ private void removeDirectory(final File target) { if (target.exists()) { try { - log.info("Deleting Index File[%s]", target); FileUtils.deleteDirectory(target); } catch (Exception e) { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java index f0ad14013794..63c456deb106 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java @@ -40,6 +40,7 @@ public class Appenderators { public static Appenderator createRealtime( + String id, DataSchema schema, AppenderatorConfig config, FireDepartmentMetrics metrics, @@ -57,6 +58,7 @@ public static Appenderator createRealtime( ) { return new AppenderatorImpl( + id, schema, config, false, @@ -84,6 +86,7 @@ public static Appenderator createRealtime( } public static Appenderator createOffline( + String id, DataSchema schema, AppenderatorConfig config, boolean storeCompactionState, @@ -95,6 +98,7 @@ public static Appenderator createOffline( ) { return new AppenderatorImpl( + id, schema, config, storeCompactionState, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java index 4a9734246148..0fdac39614ec 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java @@ -34,12 +34,14 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; +import com.google.common.util.concurrent.SettableFuture; import org.apache.druid.data.input.Committer; import org.apache.druid.data.input.InputRow; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.segment.SegmentUtils; import org.apache.druid.segment.loading.DataSegmentKiller; import org.apache.druid.segment.realtime.appenderator.SegmentWithState.SegmentState; import org.apache.druid.timeline.DataSegment; @@ -254,7 +256,7 @@ Stream getAllSegmentsOfInterval() this.segmentAllocator = Preconditions.checkNotNull(segmentAllocator, "segmentAllocator"); this.usedSegmentChecker = Preconditions.checkNotNull(usedSegmentChecker, "usedSegmentChecker"); this.dataSegmentKiller = Preconditions.checkNotNull(dataSegmentKiller, "dataSegmentKiller"); - this.executor = MoreExecutors.listeningDecorator(Execs.singleThreaded("publish-%d")); + this.executor = MoreExecutors.listeningDecorator(Execs.singleThreaded("[" + appenderator.getId() + "]-publish")); } @VisibleForTesting @@ -352,11 +354,11 @@ private SegmentIdWithShardSpec getSegment( } } - log.info("New segment[%s] for row[%s] sequenceName[%s].", newSegment, row, sequenceName); + log.info("New segment[%s] for sequenceName[%s].", newSegment, sequenceName); addSegment(sequenceName, newSegment); } else { // Well, we tried. - log.warn("Cannot allocate segment for timestamp[%s], sequenceName[%s]. ", timestamp, sequenceName); + log.warn("Cannot allocate segment for timestamp[%s], sequenceName[%s].", timestamp, sequenceName); } return newSegment; @@ -474,13 +476,16 @@ ListenableFuture pushInBackground( appenderator.push(segmentIdentifiers, wrappedCommitter, useUniquePath), (Function) segmentsAndMetadata -> { // Sanity check - final Set pushedSegments = segmentsAndMetadata.getSegments().stream() - .map( - SegmentIdWithShardSpec::fromDataSegment) - .collect(Collectors.toSet()); + final Set pushedSegments = segmentsAndMetadata + .getSegments() + .stream() + .map(SegmentIdWithShardSpec::fromDataSegment) + .collect(Collectors.toSet()); + if (!pushedSegments.equals(Sets.newHashSet(segmentIdentifiers))) { log.warn( - "Removing segments from deep storage because sanity check failed: %s", segmentsAndMetadata.getSegments() + "Removing segments from deep storage because sanity check failed: %s", + SegmentUtils.commaSeparateIdentifiers(segmentsAndMetadata.getSegments()) ); segmentsAndMetadata.getSegments().forEach(dataSegmentKiller::killQuietly); @@ -508,7 +513,8 @@ ListenableFuture pushInBackground( */ ListenableFuture dropInBackground(SegmentsAndMetadata segmentsAndMetadata) { - log.info("Dropping segments[%s]", segmentsAndMetadata.getSegments()); + log.debug("Dropping segments: %s", SegmentUtils.commaSeparateIdentifiers(segmentsAndMetadata.getSegments())); + final ListenableFuture dropFuture = Futures.allAsList( segmentsAndMetadata .getSegments() @@ -544,77 +550,89 @@ ListenableFuture publishInBackground( TransactionalSegmentPublisher publisher ) { + if (segmentsAndMetadata.getSegments().isEmpty()) { + log.debug("Nothing to publish, skipping publish step."); + final SettableFuture retVal = SettableFuture.create(); + retVal.set(segmentsAndMetadata); + return retVal; + } + + final Object metadata = segmentsAndMetadata.getCommitMetadata(); + final Object callerMetadata = metadata == null + ? null + : ((AppenderatorDriverMetadata) metadata).getCallerMetadata(); + return executor.submit( () -> { - if (segmentsAndMetadata.getSegments().isEmpty()) { - log.info("Nothing to publish, skipping publish step."); - } else { - log.info( - "Publishing segments with commitMetadata[%s]: [%s]", - segmentsAndMetadata.getCommitMetadata(), - Joiner.on(", ").join(segmentsAndMetadata.getSegments()) + try { + final ImmutableSet ourSegments = ImmutableSet.copyOf(segmentsAndMetadata.getSegments()); + final SegmentPublishResult publishResult = publisher.publishSegments( + segmentsToBeOverwritten, + ourSegments, + callerMetadata ); - try { - final Object metadata = segmentsAndMetadata.getCommitMetadata(); - final ImmutableSet ourSegments = ImmutableSet.copyOf(segmentsAndMetadata.getSegments()); - final SegmentPublishResult publishResult = publisher.publishSegments( - segmentsToBeOverwritten, - ourSegments, - metadata == null ? null : ((AppenderatorDriverMetadata) metadata).getCallerMetadata() + if (publishResult.isSuccess()) { + log.info( + "Published segments with commit metadata [%s]: %s", + callerMetadata, + SegmentUtils.commaSeparateIdentifiers(segmentsAndMetadata.getSegments()) ); - - if (publishResult.isSuccess()) { - log.info("Published segments."); + } else { + // Publishing didn't affirmatively succeed. However, segments with our identifiers may still be active + // now after all, for two possible reasons: + // + // 1) A replica may have beat us to publishing these segments. In this case we want to delete the + // segments we pushed (if they had unique paths) to avoid wasting space on deep storage. + // 2) We may have actually succeeded, but not realized it due to missing the confirmation response + // from the overlord. In this case we do not want to delete the segments we pushed, since they are + // now live! + + final Set segmentsIdentifiers = segmentsAndMetadata + .getSegments() + .stream() + .map(SegmentIdWithShardSpec::fromDataSegment) + .collect(Collectors.toSet()); + + final Set activeSegments = usedSegmentChecker.findUsedSegments(segmentsIdentifiers); + + if (activeSegments.equals(ourSegments)) { + log.info( + "Could not publish segments, but checked and found them already published; continuing: %s", + SegmentUtils.commaSeparateIdentifiers(ourSegments) + ); + + // Clean up pushed segments if they are physically disjoint from the published ones (this means + // they were probably pushed by a replica, and with the unique paths option). + final boolean physicallyDisjoint = Sets.intersection( + activeSegments.stream().map(DataSegment::getLoadSpec).collect(Collectors.toSet()), + ourSegments.stream().map(DataSegment::getLoadSpec).collect(Collectors.toSet()) + ).isEmpty(); + + if (physicallyDisjoint) { + segmentsAndMetadata.getSegments().forEach(dataSegmentKiller::killQuietly); + } } else { - // Publishing didn't affirmatively succeed. However, segments with our identifiers may still be active - // now after all, for two possible reasons: - // - // 1) A replica may have beat us to publishing these segments. In this case we want to delete the - // segments we pushed (if they had unique paths) to avoid wasting space on deep storage. - // 2) We may have actually succeeded, but not realized it due to missing the confirmation response - // from the overlord. In this case we do not want to delete the segments we pushed, since they are - // now live! - - final Set segmentsIdentifiers = segmentsAndMetadata - .getSegments() - .stream() - .map(SegmentIdWithShardSpec::fromDataSegment) - .collect(Collectors.toSet()); - - final Set activeSegments = usedSegmentChecker.findUsedSegments(segmentsIdentifiers); - - if (activeSegments.equals(ourSegments)) { - log.info("Could not publish segments, but checked and found them already published. Continuing."); - - // Clean up pushed segments if they are physically disjoint from the published ones (this means - // they were probably pushed by a replica, and with the unique paths option). - final boolean physicallyDisjoint = Sets.intersection( - activeSegments.stream().map(DataSegment::getLoadSpec).collect(Collectors.toSet()), - ourSegments.stream().map(DataSegment::getLoadSpec).collect(Collectors.toSet()) - ).isEmpty(); - - if (physicallyDisjoint) { - segmentsAndMetadata.getSegments().forEach(dataSegmentKiller::killQuietly); - } + // Our segments aren't active. Publish failed for some reason. Clean them up and then throw an error. + segmentsAndMetadata.getSegments().forEach(dataSegmentKiller::killQuietly); + + if (publishResult.getErrorMsg() != null) { + throw new ISE( + "Failed to publish segments because of [%s]: %s", + publishResult.getErrorMsg(), + SegmentUtils.commaSeparateIdentifiers(ourSegments) + ); } else { - // Our segments aren't active. Publish failed for some reason. Clean them up and then throw an error. - segmentsAndMetadata.getSegments().forEach(dataSegmentKiller::killQuietly); - - if (publishResult.getErrorMsg() != null) { - throw new ISE("Failed to publish segments because of [%s].", publishResult.getErrorMsg()); - } else { - throw new ISE("Failed to publish segments."); - } + throw new ISE("Failed to publish segments: %s", SegmentUtils.commaSeparateIdentifiers(ourSegments)); } } } - catch (Exception e) { - // Must not remove segments here, we aren't sure if our transaction succeeded or not. - log.warn(e, "Failed publish, not removing segments: %s", segmentsAndMetadata.getSegments()); - Throwables.propagateIfPossible(e); - throw new RuntimeException(e); - } + } + catch (Exception e) { + // Must not remove segments here, we aren't sure if our transaction succeeded or not. + log.noStackTrace().warn(e, "Failed publish, not removing segments: %s", segmentsAndMetadata.getSegments()); + Throwables.propagateIfPossible(e); + throw new RuntimeException(e); } return segmentsAndMetadata; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactory.java index 2658fd1dc3c6..7d03492b455a 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactory.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactory.java @@ -55,6 +55,7 @@ public DefaultOfflineAppenderatorFactory( public Appenderator build(DataSchema schema, RealtimeTuningConfig config, FireDepartmentMetrics metrics) { return Appenderators.createOffline( + schema.getDataSource(), schema, config, false, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultRealtimeAppenderatorFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultRealtimeAppenderatorFactory.java index ec1abe3f9997..2c7e1144f52c 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultRealtimeAppenderatorFactory.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultRealtimeAppenderatorFactory.java @@ -24,6 +24,7 @@ import org.apache.druid.client.cache.Cache; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.client.cache.CachePopulatorStats; +import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Processing; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.query.QueryRunnerFactoryConglomerate; @@ -46,7 +47,7 @@ public class DefaultRealtimeAppenderatorFactory implements AppenderatorFactory private final DataSegmentAnnouncer segmentAnnouncer; private final ExecutorService queryExecutorService; private final DataSegmentPusher dataSegmentPusher; - private final ObjectMapper objectMapper; + private final ObjectMapper jsonMapper; private final IndexIO indexIO; private final IndexMerger indexMerger; private final Cache cache; @@ -59,7 +60,7 @@ public DefaultRealtimeAppenderatorFactory( @JacksonInject DataSegmentAnnouncer segmentAnnouncer, @JacksonInject @Processing ExecutorService queryExecutorService, @JacksonInject DataSegmentPusher dataSegmentPusher, - @JacksonInject ObjectMapper objectMapper, + @JacksonInject @Json ObjectMapper jsonMapper, @JacksonInject IndexIO indexIO, @JacksonInject IndexMerger indexMerger, @JacksonInject Cache cache, @@ -72,7 +73,7 @@ public DefaultRealtimeAppenderatorFactory( this.segmentAnnouncer = segmentAnnouncer; this.queryExecutorService = queryExecutorService; this.dataSegmentPusher = dataSegmentPusher; - this.objectMapper = objectMapper; + this.jsonMapper = jsonMapper; this.indexIO = indexIO; this.indexMerger = indexMerger; this.cache = cache; @@ -88,6 +89,7 @@ public Appenderator build( ) { return Appenderators.createRealtime( + schema.getDataSource(), schema, config.withBasePersistDirectory( makeBasePersistSubdirectory( @@ -98,7 +100,7 @@ public Appenderator build( ), metrics, dataSegmentPusher, - objectMapper, + jsonMapper, indexIO, indexMerger, conglomerate, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java index 2a18936b86d6..be860f92ee4f 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java @@ -64,7 +64,7 @@ public Appenderator createRealtimeAppenderatorForTask( AppenderatorConfig config, FireDepartmentMetrics metrics, DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, + ObjectMapper jsonMapper, IndexIO indexIO, IndexMerger indexMerger, QueryRunnerFactoryConglomerate conglomerate, @@ -82,11 +82,12 @@ public Appenderator createRealtimeAppenderatorForTask( throw new ISE("A batch appenderator was already created for this peon's task."); } else { realtimeAppenderator = Appenderators.createRealtime( + taskId, schema, config, metrics, dataSegmentPusher, - objectMapper, + jsonMapper, indexIO, indexMerger, conglomerate, @@ -119,6 +120,7 @@ public Appenderator createOfflineAppenderatorForTask( throw new ISE("A realtime appenderator was already created for this peon's task."); } else { batchAppenderator = Appenderators.createOffline( + taskId, schema, config, storeCompactionState, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpec.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpec.java index 6e85794e69d1..58dad0d36448 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpec.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpec.java @@ -59,16 +59,6 @@ public SegmentId asSegmentId() return id; } - public SegmentIdWithShardSpec withShardSpec(ShardSpec shardSpec) - { - return new SegmentIdWithShardSpec( - id.getDataSource(), - id.getInterval(), - id.getVersion(), - shardSpec - ); - } - @JsonProperty public String getDataSource() { @@ -93,11 +83,6 @@ public ShardSpec getShardSpec() return shardSpec; } - public String getIdentifierAsString() - { - return asString; - } - @Override public boolean equals(Object o) { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java index c68b4a9bb270..d7b0b6ad6ad3 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java @@ -116,8 +116,6 @@ public Object startJob(AppenderatorDriverSegmentLockHelper lockHelper) AppenderatorDriverMetadata.class ); - log.info("Restored metadata[%s].", metadata); - if (metadata != null) { synchronized (segments) { final Map lastSegmentIds = metadata.getLastSegmentIds(); @@ -227,10 +225,10 @@ public void moveSegmentOut(final String sequenceName, final List registerHandoff(SegmentsAndMetadata ); } - log.info("Register handoff of segments: [%s]", waitingSegmentIdList); + log.debug("Register handoff of segments: [%s]", waitingSegmentIdList); final SettableFuture resultFuture = SettableFuture.create(); final AtomicInteger numRemainingHandoffSegments = new AtomicInteger(waitingSegmentIdList.size()); @@ -343,7 +341,7 @@ public ListenableFuture registerHandoff(SegmentsAndMetadata ), Execs.directExecutor(), () -> { - log.info("Segment[%s] successfully handed off, dropping.", segmentIdentifier); + log.debug("Segment[%s] successfully handed off, dropping.", segmentIdentifier); metrics.incrementHandOffCount(); final ListenableFuture dropFuture = appenderator.drop(segmentIdentifier); @@ -355,7 +353,7 @@ public ListenableFuture registerHandoff(SegmentsAndMetadata public void onSuccess(Object result) { if (numRemainingHandoffSegments.decrementAndGet() == 0) { - log.info("Successfully handed off [%d] segments.", segmentsAndMetadata.getSegments().size()); + log.debug("Successfully handed off [%d] segments.", segmentsAndMetadata.getSegments().size()); resultFuture.set( new SegmentsAndMetadata( segmentsAndMetadata.getSegments(), diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java index 5327a7537c0f..96c5ff0ef264 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java @@ -163,6 +163,7 @@ public Appenderator createRealtimeAppenderatorForTask( ); Appenderator appenderator = new AppenderatorImpl( + taskId, schema, rewriteAppenderatorConfigMemoryLimits(config), false, @@ -201,6 +202,7 @@ public Appenderator createOfflineAppenderatorForTask( ); Appenderator appenderator = Appenderators.createOffline( + taskId, schema, rewriteAppenderatorConfigMemoryLimits(config), storeCompactionState, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/LocalFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/LocalFirehoseFactory.java index beb713adf2d5..e555f4ba4c78 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/LocalFirehoseFactory.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/firehose/LocalFirehoseFactory.java @@ -85,13 +85,11 @@ public StringInputRowParser getParser() @Override protected Collection initObjects() { - final Collection files = FileUtils.listFiles( + return FileUtils.listFiles( Preconditions.checkNotNull(baseDir).getAbsoluteFile(), new WildcardFileFilter(filter), TrueFileFilter.INSTANCE ); - log.info("Initialized with " + files + " files"); - return files; } @Override diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProvider.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProvider.java index df2f08ab1e92..802be54cc115 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProvider.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProvider.java @@ -26,7 +26,6 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.server.DruidNode; -import org.apache.druid.server.initialization.ServerConfig; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -43,20 +42,17 @@ public class ServiceAnnouncingChatHandlerProvider implements ChatHandlerProvider private final DruidNode node; private final ServiceAnnouncer serviceAnnouncer; - private final ServerConfig serverConfig; private final ConcurrentMap handlers; private final ConcurrentSkipListSet announcements; @Inject public ServiceAnnouncingChatHandlerProvider( @RemoteChatHandler DruidNode node, - ServiceAnnouncer serviceAnnouncer, - ServerConfig serverConfig + ServiceAnnouncer serviceAnnouncer ) { this.node = node; this.serviceAnnouncer = serviceAnnouncer; - this.serverConfig = serverConfig; this.handlers = new ConcurrentHashMap<>(); this.announcements = new ConcurrentSkipListSet<>(); } @@ -70,7 +66,7 @@ public void register(final String service, ChatHandler handler) @Override public void register(final String service, ChatHandler handler, boolean announce) { - log.info("Registering Eventhandler[%s]", service); + log.debug("Registering Eventhandler[%s]", service); if (handlers.putIfAbsent(service, handler) != null) { throw new ISE("handler already registered for service[%s]", service); @@ -93,7 +89,7 @@ public void register(final String service, ChatHandler handler, boolean announce @Override public void unregister(final String service) { - log.info("Unregistering chat handler[%s]", service); + log.debug("Unregistering chat handler[%s]", service); final ChatHandler handler = handlers.get(service); if (handler == null) { @@ -123,6 +119,14 @@ public Optional get(final String key) private DruidNode makeDruidNode(String key) { - return new DruidNode(key, node.getHost(), node.isBindOnHost(), node.getPlaintextPort(), node.getTlsPort(), node.isEnablePlaintextPort(), node.isEnableTlsPort()); + return new DruidNode( + key, + node.getHost(), + node.isBindOnHost(), + node.getPlaintextPort(), + node.getTlsPort(), + node.isEnablePlaintextPort(), + node.isEnableTlsPort() + ); } } diff --git a/server/src/main/java/org/apache/druid/server/DruidNode.java b/server/src/main/java/org/apache/druid/server/DruidNode.java index 8454dc673948..1a3603936490 100644 --- a/server/src/main/java/org/apache/druid/server/DruidNode.java +++ b/server/src/main/java/org/apache/druid/server/DruidNode.java @@ -32,10 +32,13 @@ import javax.validation.constraints.Max; import javax.validation.constraints.NotNull; import java.net.InetAddress; +import java.net.URI; +import java.net.URISyntaxException; import java.net.UnknownHostException; import java.util.Objects; /** + * */ public class DruidNode { @@ -57,7 +60,7 @@ public class DruidNode /** * This property is now deprecated, this is present just so that JsonConfigurator does not fail if this is set. * Please use {@link DruidNode#plaintextPort} instead, which if set will be used and hence this has -1 as default value. - * */ + */ @Deprecated @JsonProperty @Max(0xffff) @@ -105,7 +108,6 @@ public DruidNode( * host = "[2001:db8:85a3::8a2e:370:7334]", port = 123 -> host = 2001:db8:85a3::8a2e:370:7334, port = 123 * host = "2001:db8:85a3::8a2e:370:7334", port = 123 -> host = 2001:db8:85a3::8a2e:370:7334, port = 123 * host = null , port = 123 -> host = _default_, port = 123 - * */ @JsonCreator public DruidNode( @@ -130,7 +132,15 @@ public DruidNode( ); } - private void init(String serviceName, String host, boolean bindOnHost, Integer plainTextPort, Integer tlsPort, boolean enablePlaintextPort, boolean enableTlsPort) + private void init( + String serviceName, + String host, + boolean bindOnHost, + Integer plainTextPort, + Integer tlsPort, + boolean enablePlaintextPort, + boolean enableTlsPort + ) { Preconditions.checkNotNull(serviceName); @@ -159,7 +169,7 @@ private void init(String serviceName, String host, boolean bindOnHost, Integer p } if (enablePlaintextPort && enableTlsPort && ((plainTextPort == null || tlsPort == null) - || plainTextPort.equals(tlsPort))) { + || plainTextPort.equals(tlsPort))) { // If both plainTExt and tls are enabled then do not allow plaintextPort to be null or throw new IAE("plaintextPort and tlsPort cannot be null or same if both http and https connectors are enabled"); } @@ -273,6 +283,16 @@ public String getHostAndPortToUse() return getHostAndTlsPort() != null ? getHostAndTlsPort() : getHostAndPort(); } + public URI getUriToUse() + { + try { + return new URI(getServiceScheme(), null, host, getPortToUse(), null, null, null); + } + catch (URISyntaxException e) { + throw new RuntimeException(e); + } + } + public static String getDefaultHost() { try { diff --git a/server/src/main/java/org/apache/druid/server/QueryLifecycle.java b/server/src/main/java/org/apache/druid/server/QueryLifecycle.java index 00d982ad37c1..b8e5f7a90c11 100644 --- a/server/src/main/java/org/apache/druid/server/QueryLifecycle.java +++ b/server/src/main/java/org/apache/druid/server/QueryLifecycle.java @@ -321,7 +321,7 @@ public void emitLogsAndMetrics( if (e instanceof QueryInterruptedException) { // Mimic behavior from QueryResource, where this code was originally taken from. - log.warn(e, "Exception while processing queryId [%s]", baseQuery.getId()); + log.noStackTrace().warn(e, "Exception while processing queryId [%s]", baseQuery.getId()); statsMap.put("interrupted", true); statsMap.put("reason", e.toString()); } diff --git a/server/src/main/java/org/apache/druid/server/QueryResource.java b/server/src/main/java/org/apache/druid/server/QueryResource.java index eb4af459f7d0..6bfb3a9cc3bf 100644 --- a/server/src/main/java/org/apache/druid/server/QueryResource.java +++ b/server/src/main/java/org/apache/druid/server/QueryResource.java @@ -249,7 +249,7 @@ public void write(OutputStream outputStream) throws WebApplicationException } catch (Exception ex) { e = ex; - log.error(ex, "Unable to send query response."); + log.noStackTrace().error(ex, "Unable to send query response."); throw new RuntimeException(ex); } finally { @@ -319,9 +319,9 @@ public void write(OutputStream outputStream) throws WebApplicationException failedQueryCount.incrementAndGet(); queryLifecycle.emitLogsAndMetrics(e, req.getRemoteAddr(), -1); - log.makeAlert(e, "Exception handling request") - .addData("exception", e.toString()) - .addData("query", query != null ? query.toString() : "unparseable query") + log.noStackTrace() + .makeAlert(e, "Exception handling request") + .addData("query", query != null ? jsonMapper.writeValueAsString(query) : "unparseable query") .addData("peer", req.getRemoteAddr()) .emit(); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index e98e8d38e9c1..d8d280031222 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -359,7 +359,7 @@ public CoordinatorCompactionConfig getCompactionConfig() public void markSegmentAsUnused(DataSegment segment) { - log.info("Marking segment[%s] as unused", segment.getId()); + log.debug("Marking segment[%s] as unused", segment.getId()); segmentsMetadata.markSegmentAsUnused(segment.getId().toString()); } @@ -518,8 +518,10 @@ private void becomeLeader() return; } - log.info("I am the leader of the coordinators, all must bow!"); - log.info("Starting coordination in [%s]", config.getCoordinatorStartDelay()); + log.info( + "I am the leader of the coordinators, all must bow! Starting coordination in [%s].", + config.getCoordinatorStartDelay() + ); segmentsMetadata.startPollingDatabasePeriodically(); metadataRuleManager.start(); @@ -600,8 +602,8 @@ private List makeIndexingServiceHelpers() helpers.add(segmentCompactor); helpers.addAll(indexingServiceHelpers); - log.info( - "Done making indexing service helpers [%s]", + log.debug( + "Done making indexing service helpers %s", helpers.stream().map(helper -> helper.getClass().getName()).collect(Collectors.toList()) ); return ImmutableList.copyOf(helpers); @@ -721,7 +723,7 @@ public CoordinatorHistoricalManagerRunnable(final int startingLeaderCounter) if (!loadManagementPeons.containsKey(server.getName())) { LoadQueuePeon loadQueuePeon = taskMaster.giveMePeon(server); loadQueuePeon.start(); - log.info("Created LoadQueuePeon for server[%s].", server.getName()); + log.debug("Created LoadQueuePeon for server[%s].", server.getName()); loadManagementPeons.put(server.getName(), loadQueuePeon); } @@ -743,7 +745,7 @@ public CoordinatorHistoricalManagerRunnable(final int startingLeaderCounter) disappeared.remove(server.getName()); } for (String name : disappeared) { - log.info("Removing listener for server[%s] which is no longer there.", name); + log.debug("Removing listener for server[%s] which is no longer there.", name); LoadQueuePeon peon = loadManagementPeons.remove(name); peon.stop(); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/HttpLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/HttpLoadQueuePeon.java index 4652659a35d2..eea87118f9e9 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/HttpLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/HttpLoadQueuePeon.java @@ -142,7 +142,7 @@ public HttpLoadQueuePeon( private void doSegmentManagement() { if (stopped || !mainLoopInProgress.compareAndSet(false, true)) { - log.debug("[%s]Ignoring tick. Either in-progress already or stopped.", serverId); + log.trace("[%s]Ignoring tick. Either in-progress already or stopped.", serverId); return; } @@ -168,7 +168,7 @@ private void doSegmentManagement() } if (newRequests.size() == 0) { - log.debug( + log.trace( "[%s]Found no load/drop requests. SegmentsToLoad[%d], SegmentsToDrop[%d], batchSize[%d].", serverId, segmentsToLoad.size(), @@ -180,7 +180,7 @@ private void doSegmentManagement() } try { - log.debug("Sending [%d] load/drop requests to Server[%s].", newRequests.size(), serverId); + log.trace("Sending [%d] load/drop requests to Server[%s].", newRequests.size(), serverId); BytesAccumulatingResponseHandler responseHandler = new BytesAccumulatingResponseHandler(); ListenableFuture future = httpClient.go( new Request(HttpMethod.POST, changeRequestURL) @@ -201,15 +201,15 @@ public void onSuccess(InputStream result) boolean scheduleNextRunImmediately = true; try { if (responseHandler.getStatus() == HttpServletResponse.SC_NO_CONTENT) { - log.debug("Received NO CONTENT reseponse from [%s]", serverId); + log.trace("Received NO CONTENT reseponse from [%s]", serverId); } else if (HttpServletResponse.SC_OK == responseHandler.getStatus()) { try { List statuses = jsonMapper.readValue(result, RESPONSE_ENTITY_TYPE_REF); - log.debug("Server[%s] returned status response [%s].", serverId, statuses); + log.trace("Server[%s] returned status response [%s].", serverId, statuses); synchronized (lock) { if (stopped) { - log.debug("Ignoring response from Server[%s]. We are already stopped.", serverId); + log.trace("Ignoring response from Server[%s]. We are already stopped.", serverId); scheduleNextRunImmediately = false; return; } @@ -221,7 +221,7 @@ public void onSuccess(InputStream result) handleResponseStatus(e.getRequest(), e.getStatus()); break; case PENDING: - log.debug("Request[%s] is still pending on server[%s].", e.getRequest(), serverId); + log.trace("Request[%s] is still pending on server[%s].", e.getRequest(), serverId); break; default: scheduleNextRunImmediately = false; @@ -380,7 +380,7 @@ public void loadSegment(DataSegment segment, LoadPeonCallback callback) SegmentHolder holder = segmentsToLoad.get(segment); if (holder == null) { - log.debug("Server[%s] to load segment[%s] queued.", serverId, segment.getId()); + log.trace("Server[%s] to load segment[%s] queued.", serverId, segment.getId()); segmentsToLoad.put(segment, new LoadSegmentHolder(segment, callback)); processingExecutor.execute(this::doSegmentManagement); } else { @@ -405,7 +405,7 @@ public void dropSegment(DataSegment segment, LoadPeonCallback callback) SegmentHolder holder = segmentsToDrop.get(segment); if (holder == null) { - log.debug("Server[%s] to drop segment[%s] queued.", serverId, segment.getId()); + log.trace("Server[%s] to drop segment[%s] queued.", serverId, segment.getId()); segmentsToDrop.put(segment, new DropSegmentHolder(segment, callback)); processingExecutor.execute(this::doSegmentManagement); } else { @@ -518,7 +518,7 @@ public boolean hasTimedOut() public void requestSucceeded() { - log.debug( + log.trace( "Server[%s] Successfully processed segment[%s] request[%s].", serverId, segment.getId(), diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentInfoLoader.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentInfoLoader.java index 1b7beab2a960..e1447887ec92 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentInfoLoader.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentInfoLoader.java @@ -39,7 +39,7 @@ public DruidCoordinatorSegmentInfoLoader(DruidCoordinator coordinator) @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - log.info("Starting coordination. Getting used segments."); + log.debug("Starting coordination. Getting used segments."); DataSourcesSnapshot dataSourcesSnapshot = params.getDataSourcesSnapshot(); for (DataSegment segment : dataSourcesSnapshot.iterateAllUsedSegmentsInSnapshot()) { diff --git a/server/src/main/java/org/apache/druid/server/emitter/EmitterModule.java b/server/src/main/java/org/apache/druid/server/emitter/EmitterModule.java index 5b3cf3be070d..c9db2e1dd962 100644 --- a/server/src/main/java/org/apache/druid/server/emitter/EmitterModule.java +++ b/server/src/main/java/org/apache/druid/server/emitter/EmitterModule.java @@ -51,6 +51,7 @@ import java.util.Properties; /** + * */ public class EmitterModule implements Module { @@ -101,8 +102,7 @@ public ServiceEmitter getServiceEmitter( ) { final DruidNode config = configSupplier.get(); - log.info("Underlying emitter for ServiceEmitter: %s", emitter); - log.info("Extra service dimensions: %s", extraServiceDimensions); + log.info("Using emitter [%s] for metrics and alerts, with dimensions [%s].", emitter, extraServiceDimensions); final ServiceEmitter retVal = new ServiceEmitter( config.getServiceName(), config.getHostAndPortToUse(), @@ -129,7 +129,7 @@ private static class EmitterProvider implements Provider @Inject public void inject(Injector injector) { - final List> emitterBindings = injector.findBindingsByType(new TypeLiteral(){}); + final List> emitterBindings = injector.findBindingsByType(new TypeLiteral() {}); if (Strings.isNullOrEmpty(emitterType)) { // If the emitter is unspecified, we want to default to the no-op emitter. Include empty string here too, just diff --git a/server/src/main/java/org/apache/druid/server/initialization/jetty/JettyServerModule.java b/server/src/main/java/org/apache/druid/server/initialization/jetty/JettyServerModule.java index 8d02098a5ea6..8fc998b30603 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/jetty/JettyServerModule.java +++ b/server/src/main/java/org/apache/druid/server/initialization/jetty/JettyServerModule.java @@ -395,7 +395,7 @@ public void lifeCycleStopped(LifeCycle event) @Override public void start() throws Exception { - log.info("Starting Jetty Server..."); + log.debug("Starting Jetty Server..."); server.start(); if (node.isEnableTlsPort()) { // Perform validation @@ -426,12 +426,12 @@ public void stop() try { final long unannounceDelay = config.getUnannouncePropagationDelay().toStandardDuration().getMillis(); if (unannounceDelay > 0) { - log.info("Waiting %s ms for unannouncement to propagate.", unannounceDelay); + log.info("Sleeping %s ms for unannouncement to propagate.", unannounceDelay); Thread.sleep(unannounceDelay); } else { log.debug("Skipping unannounce wait."); } - log.info("Stopping Jetty Server..."); + log.debug("Stopping Jetty Server..."); server.stop(); } catch (InterruptedException e) { diff --git a/server/src/main/java/org/apache/druid/server/metrics/MetricsModule.java b/server/src/main/java/org/apache/druid/server/metrics/MetricsModule.java index 8f0f0db6f83d..71b994541252 100644 --- a/server/src/main/java/org/apache/druid/server/metrics/MetricsModule.java +++ b/server/src/main/java/org/apache/druid/server/metrics/MetricsModule.java @@ -46,6 +46,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; /** * Sets up the {@link MonitorScheduler} to monitor things on a regular schedule. {@link Monitor}s must be explicitly @@ -92,11 +93,15 @@ public MonitorScheduler getMonitorScheduler( List monitors = new ArrayList<>(); for (Class monitorClass : Iterables.concat(monitorsConfig.getMonitors(), monitorSet)) { - final Monitor monitor = injector.getInstance(monitorClass); - - log.info("Adding monitor[%s]", monitor); + monitors.add(injector.getInstance(monitorClass)); + } - monitors.add(monitor); + if (!monitors.isEmpty()) { + log.info( + "Loaded %d monitors: %s", + monitors.size(), + monitors.stream().map(monitor -> monitor.getClass().getName()).collect(Collectors.joining(", ")) + ); } return new MonitorScheduler( diff --git a/server/src/test/java/org/apache/druid/curator/CuratorModuleTest.java b/server/src/test/java/org/apache/druid/curator/CuratorModuleTest.java index b0997b873b1d..6fd5705919d4 100644 --- a/server/src/test/java/org/apache/druid/curator/CuratorModuleTest.java +++ b/server/src/test/java/org/apache/druid/curator/CuratorModuleTest.java @@ -154,7 +154,7 @@ public void exitsJvmWhenMaxRetriesExceeded() throws Exception l.getLevel().equals(Level.ERROR) && l.getMessage() .getFormattedMessage() - .equals("Unhandled error in Curator Framework") + .equals("Unhandled error in Curator, stopping server.") ) ); } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java index 2bca96ac7486..b4432e156491 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java @@ -229,6 +229,7 @@ public Map makeLoadSpec(URI uri) } }; appenderator = Appenderators.createRealtime( + schema.getDataSource(), schema, tuningConfig, metrics, diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java index bbbc64a99041..0254bd31cb1d 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java @@ -239,7 +239,7 @@ public void testFailDuringPublish() throws Exception { expectedException.expect(ExecutionException.class); expectedException.expectCause(CoreMatchers.instanceOf(ISE.class)); - expectedException.expectMessage("Failed to publish segments because of [test]."); + expectedException.expectMessage("Failed to publish segments because of [test]:"); testFailDuringPublishInternal(false); } @@ -390,6 +390,12 @@ public FailableAppenderator interruptPush() return this; } + @Override + public String getId() + { + return null; + } + @Override public String getDataSource() { diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProviderTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProviderTest.java index 5878538477e5..71a3fe308f4d 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProviderTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProviderTest.java @@ -21,7 +21,6 @@ import org.apache.druid.curator.discovery.ServiceAnnouncer; import org.apache.druid.server.DruidNode; -import org.apache.druid.server.initialization.ServerConfig; import org.easymock.Capture; import org.easymock.EasyMock; import org.easymock.EasyMockRunner; @@ -54,7 +53,7 @@ private static class TestChatHandler implements ChatHandler @Before public void setUp() { - chatHandlerProvider = new ServiceAnnouncingChatHandlerProvider(node, serviceAnnouncer, new ServerConfig()); + chatHandlerProvider = new ServiceAnnouncingChatHandlerProvider(node, serviceAnnouncer); } @Test diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentChangeRequestDropTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentChangeRequestDropTest.java index 85c8838b3c63..fb1cd69a144a 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentChangeRequestDropTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentChangeRequestDropTest.java @@ -65,7 +65,7 @@ public void testV1Serialization() throws Exception JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - Assert.assertEquals(12, objectMap.size()); + Assert.assertEquals(11, objectMap.size()); Assert.assertEquals("drop", objectMap.get("action")); Assert.assertEquals("something", objectMap.get("dataSource")); Assert.assertEquals(interval.toString(), objectMap.get("interval")); diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentChangeRequestLoadTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentChangeRequestLoadTest.java index 5f80d8366468..d165e3480f52 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentChangeRequestLoadTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentChangeRequestLoadTest.java @@ -64,7 +64,7 @@ public void testV1Serialization() throws Exception mapper.writeValueAsString(segmentDrop), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - Assert.assertEquals(12, objectMap.size()); + Assert.assertEquals(11, objectMap.size()); Assert.assertEquals("load", objectMap.get("action")); Assert.assertEquals("something", objectMap.get("dataSource")); Assert.assertEquals(interval.toString(), objectMap.get("interval")); diff --git a/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java b/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java index ee36f0ebd677..b0cd175b1284 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java @@ -293,7 +293,7 @@ public void testSingleAnnounceManyTimes() throws Exception } List zNodes = cf.getChildren().forPath(TEST_SEGMENTS_PATH); - Assert.assertEquals(25, zNodes.size()); + Assert.assertEquals(20, zNodes.size()); Set segments = Sets.newHashSet(testSegments); for (String zNode : zNodes) { diff --git a/server/src/test/java/org/apache/druid/server/http/ServersResourceTest.java b/server/src/test/java/org/apache/druid/server/http/ServersResourceTest.java index 8f94ac4c7a32..f96d2a3e74c8 100644 --- a/server/src/test/java/org/apache/druid/server/http/ServersResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/ServersResourceTest.java @@ -73,7 +73,7 @@ public void testGetClusterServersFull() throws Exception + "\"priority\":0," + "\"segments\":{\"dataSource_2016-03-22T14:00:00.000Z_2016-03-22T15:00:00.000Z_v0\":" + "{\"dataSource\":\"dataSource\",\"interval\":\"2016-03-22T14:00:00.000Z/2016-03-22T15:00:00.000Z\",\"version\":\"v0\",\"loadSpec\":{},\"dimensions\":\"\",\"metrics\":\"\"," - + "\"shardSpec\":{\"type\":\"numbered\",\"partitionNum\":0,\"partitions\":1},\"lastCompactionState\":null,\"binaryVersion\":null,\"size\":1,\"identifier\":\"dataSource_2016-03-22T14:00:00.000Z_2016-03-22T15:00:00.000Z_v0\"}}," + + "\"shardSpec\":{\"type\":\"numbered\",\"partitionNum\":0,\"partitions\":1},\"binaryVersion\":null,\"size\":1,\"identifier\":\"dataSource_2016-03-22T14:00:00.000Z_2016-03-22T15:00:00.000Z_v0\"}}," + "\"currSize\":1}]"; Assert.assertEquals(expected, result); } @@ -99,7 +99,7 @@ public void testGetServerFull() throws Exception + "\"priority\":0," + "\"segments\":{\"dataSource_2016-03-22T14:00:00.000Z_2016-03-22T15:00:00.000Z_v0\":" + "{\"dataSource\":\"dataSource\",\"interval\":\"2016-03-22T14:00:00.000Z/2016-03-22T15:00:00.000Z\",\"version\":\"v0\",\"loadSpec\":{},\"dimensions\":\"\",\"metrics\":\"\"," - + "\"shardSpec\":{\"type\":\"numbered\",\"partitionNum\":0,\"partitions\":1},\"lastCompactionState\":null,\"binaryVersion\":null,\"size\":1,\"identifier\":\"dataSource_2016-03-22T14:00:00.000Z_2016-03-22T15:00:00.000Z_v0\"}}," + + "\"shardSpec\":{\"type\":\"numbered\",\"partitionNum\":0,\"partitions\":1},\"binaryVersion\":null,\"size\":1,\"identifier\":\"dataSource_2016-03-22T14:00:00.000Z_2016-03-22T15:00:00.000Z_v0\"}}," + "\"currSize\":1}"; Assert.assertEquals(expected, result); } diff --git a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java index a313e6bc5867..4b3f03558f2c 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -97,6 +97,7 @@ import java.util.concurrent.ExecutorService; /** + * */ @Command( name = "coordinator", @@ -105,6 +106,7 @@ public class CliCoordinator extends ServerRunnable { private static final Logger log = new Logger(CliCoordinator.class); + private static final String AS_OVERLORD_PROPERTY = "druid.coordinator.asOverlord.enabled"; private Properties properties; private boolean beOverlord; @@ -121,7 +123,7 @@ public void configure(Properties properties) beOverlord = isOverlord(properties); if (beOverlord) { - log.info("Coordinator is configured to act as Overlord as well."); + log.info("Coordinator is configured to act as Overlord as well (%s = true).", AS_OVERLORD_PROPERTY); } } @@ -290,6 +292,6 @@ public LoadQueueTaskMaster getLoadQueueTaskMaster( public static boolean isOverlord(Properties properties) { - return Boolean.parseBoolean(properties.getProperty("druid.coordinator.asOverlord.enabled")); + return Boolean.parseBoolean(properties.getProperty(AS_OVERLORD_PROPERTY)); } } diff --git a/services/src/main/java/org/apache/druid/cli/GuiceRunnable.java b/services/src/main/java/org/apache/druid/cli/GuiceRunnable.java index 582293752269..2e8b954af9be 100644 --- a/services/src/main/java/org/apache/druid/cli/GuiceRunnable.java +++ b/services/src/main/java/org/apache/druid/cli/GuiceRunnable.java @@ -88,7 +88,7 @@ public Lifecycle initLifecycle(Injector injector) } log.info( - "Starting up with processors[%,d], memory[%,d], maxMemory[%,d]%s.", + "Starting up with processors[%,d], memory[%,d], maxMemory[%,d]%s. Properties follow.", JvmUtils.getRuntimeInfo().getAvailableProcessors(), JvmUtils.getRuntimeInfo().getTotalHeapSizeBytes(), JvmUtils.getRuntimeInfo().getMaxHeapSizeBytes(), diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java index 5ae810f41cc8..d8c9f2c4d666 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java @@ -274,13 +274,9 @@ public void run() final DruidTable druidTable = buildDruidTable(dataSource); final DruidTable oldTable = tables.put(dataSource, druidTable); if (oldTable == null || !oldTable.getRowSignature().equals(druidTable.getRowSignature())) { - log.debug( - "Table for dataSource[%s] has new signature[%s].", - dataSource, - druidTable.getRowSignature() - ); + log.info("dataSource [%s] has new signature: %s.", dataSource, druidTable.getRowSignature()); } else { - log.debug("Table for dataSource[%s] signature is unchanged.", dataSource); + log.debug("dataSource [%s] signature is unchanged.", dataSource); } } @@ -320,10 +316,10 @@ public void run() ); if (config.isAwaitInitializationOnStart()) { - final long startMillis = System.currentTimeMillis(); - log.info("%s waiting for initialization.", getClass().getSimpleName()); + final long startNanos = System.nanoTime(); + log.debug("%s waiting for initialization.", getClass().getSimpleName()); awaitInitialization(); - log.info("%s initialized in [%,d] ms.", getClass().getSimpleName(), System.currentTimeMillis() - startMillis); + log.info("%s initialized in [%,d] ms.", getClass().getSimpleName(), (System.nanoTime() - startNanos) / 1000000); } } @@ -425,7 +421,7 @@ void removeSegment(final DataSegment segment) if (dataSourceSegments.isEmpty()) { segmentMetadataInfo.remove(segment.getDataSource()); tables.remove(segment.getDataSource()); - log.info("Removed all metadata for dataSource[%s].", segment.getDataSource()); + log.info("dataSource[%s] no longer exists, all metadata removed.", segment.getDataSource()); } lock.notifyAll(); @@ -542,10 +538,7 @@ private Set refreshSegmentsForDataSource(final String dataSource, fin } else { final AvailableSegmentMetadata segmentMetadata = dataSourceSegments.get(segmentId); if (segmentMetadata == null) { - log.warn( - "No segment[%s] found, skipping refresh", - segmentId - ); + log.warn("No segment[%s] found, skipping refresh", segmentId); } else { final AvailableSegmentMetadata updatedSegmentMetadata = AvailableSegmentMetadata .from(segmentMetadata) @@ -567,7 +560,7 @@ private Set refreshSegmentsForDataSource(final String dataSource, fin yielder.close(); } - log.info( + log.debug( "Refreshed metadata for dataSource[%s] in %,d ms (%d segments queried, %d segments left).", dataSource, System.currentTimeMillis() - startTime,