From 96602241db588281d65bbc5c413d51c005706f80 Mon Sep 17 00:00:00 2001 From: leventov Date: Thu, 15 Jun 2017 19:41:39 -0500 Subject: [PATCH 1/6] Remove ability to create segments in v8 format --- .../indexing/IndexMergeBenchmark.java | 30 - .../indexing/IndexPersistBenchmark.java | 25 - .../extensions-core/kafka-ingestion.md | 1 - docs/content/ingestion/batch-ingestion.md | 1 - docs/content/ingestion/stream-pull.md | 1 - docs/content/ingestion/tasks.md | 4 - .../druid/indexing/kafka/KafkaIndexTask.java | 2 +- .../indexing/kafka/KafkaTuningConfig.java | 20 +- .../KafkaSupervisorTuningConfig.java | 4 +- .../indexing/kafka/KafkaIndexTaskTest.java | 20 +- .../indexing/kafka/KafkaTuningConfigTest.java | 4 - .../KafkaSupervisorTuningConfigTest.java | 3 - .../indexer/HadoopDruidIndexerConfig.java | 8 - .../io/druid/indexer/HadoopTuningConfig.java | 18 +- .../io/druid/indexer/IndexGeneratorJob.java | 24 +- .../indexer/updater/HadoopConverterJob.java | 18 +- .../druid/indexer/IndexGeneratorJobTest.java | 24 +- .../io/druid/indexing/common/TaskToolbox.java | 9 - .../indexing/common/TaskToolboxFactory.java | 5 - .../common/index/YeOldePlumberSchool.java | 5 +- .../indexing/common/task/AppendTask.java | 6 +- .../common/task/ConvertSegmentTask.java | 8 +- .../druid/indexing/common/task/IndexTask.java | 16 +- .../druid/indexing/common/task/MergeTask.java | 6 +- .../common/task/RealtimeIndexTask.java | 1 - .../common/task/SameIntervalMergeTask.java | 14 +- .../indexing/common/TaskToolboxTest.java | 3 - .../io/druid/indexing/common/TestUtils.java | 9 - .../indexing/common/task/IndexTaskTest.java | 5 +- .../common/task/RealtimeIndexTaskTest.java | 23 +- .../task/SameIntervalMergeTaskTest.java | 3 +- .../indexing/common/task/TaskSerdeTest.java | 3 - .../IngestSegmentFirehoseFactoryTest.java | 6 +- ...estSegmentFirehoseFactoryTimelineTest.java | 6 +- .../indexing/overlord/TaskLifecycleTest.java | 4 - .../worker/WorkerTaskMonitorTest.java | 4 - .../main/java/io/druid/segment/IndexIO.java | 463 +------ .../java/io/druid/segment/IndexMerger.java | 1064 ++--------------- .../java/io/druid/segment/IndexMergerV9.java | 483 +++++++- .../druid/query/MultiValuedDimensionTest.java | 2 +- .../aggregation/AggregationTestHelper.java | 9 +- .../java/io/druid/segment/EmptyIndexTest.java | 2 +- .../java/io/druid/segment/IndexBuilder.java | 2 +- .../io/druid/segment/IndexMergerTest.java | 11 +- .../IndexMergerV9CompatibilityTest.java | 129 +- .../QueryableIndexIndexableAdapterTest.java | 2 +- .../io/druid/segment/SchemalessIndexTest.java | 2 +- .../java/io/druid/segment/TestHelper.java | 7 - .../test/java/io/druid/segment/TestIndex.java | 2 +- .../druid/segment/filter/BaseFilterTest.java | 3 +- .../filter/SpatialFilterBonusTest.java | 2 +- .../segment/filter/SpatialFilterTest.java | 2 +- .../v8SegmentPersistDir/00000.smoosh | Bin 0 -> 2525 bytes .../resources/v8SegmentPersistDir/meta.smoosh | 7 + .../resources/v8SegmentPersistDir/version.bin | Bin 0 -> 4 bytes .../indexing/RealtimeTuningConfig.java | 16 +- .../plumber/FlushingPlumberSchool.java | 7 +- .../plumber/RealtimePlumberSchool.java | 6 +- .../indexing/RealtimeTuningConfigTest.java | 3 - .../segment/realtime/FireDepartmentTest.java | 1 - .../appenderator/AppenderatorTester.java | 3 +- .../firehose/IngestSegmentFirehoseTest.java | 2 +- .../plumber/RealtimePlumberSchoolTest.java | 14 +- 63 files changed, 748 insertions(+), 1839 deletions(-) create mode 100644 processing/src/test/resources/v8SegmentPersistDir/00000.smoosh create mode 100644 processing/src/test/resources/v8SegmentPersistDir/meta.smoosh create mode 100644 processing/src/test/resources/v8SegmentPersistDir/version.bin diff --git a/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexMergeBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexMergeBenchmark.java index 6d8fdaac4d3b..e67321d67f4e 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexMergeBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexMergeBenchmark.java @@ -32,7 +32,6 @@ import io.druid.java.util.common.logger.Logger; import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMerger; import io.druid.segment.IndexMergerV9; import io.druid.segment.IndexSpec; import io.druid.segment.QueryableIndex; @@ -82,7 +81,6 @@ public class IndexMergeBenchmark private static final Logger log = new Logger(IndexMergeBenchmark.class); private static final int RNG_SEED = 9999; - private static final IndexMerger INDEX_MERGER; private static final IndexMergerV9 INDEX_MERGER_V9; private static final IndexIO INDEX_IO; public static final ObjectMapper JSON_MAPPER; @@ -104,7 +102,6 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER = new IndexMerger(JSON_MAPPER, INDEX_IO); INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); } @@ -175,33 +172,6 @@ private IncrementalIndex makeIncIndex() ); } - @Benchmark - @BenchmarkMode(Mode.AverageTime) - @OutputTimeUnit(TimeUnit.MICROSECONDS) - public void merge(Blackhole blackhole) throws Exception - { - File tmpFile = File.createTempFile("IndexMergeBenchmark-MERGEDFILE-" + System.currentTimeMillis(), ".TEMPFILE"); - tmpFile.delete(); - tmpFile.mkdirs(); - try { - log.info(tmpFile.getAbsolutePath() + " isFile: " + tmpFile.isFile() + " isDir:" + tmpFile.isDirectory()); - - File mergedFile = INDEX_MERGER.mergeQueryableIndex( - indexesToMerge, - rollup, - schemaInfo.getAggsArray(), - tmpFile, - new IndexSpec() - ); - - blackhole.consume(mergedFile); - } - finally { - tmpFile.delete(); - } - - } - @Benchmark @BenchmarkMode(Mode.AverageTime) @OutputTimeUnit(TimeUnit.MICROSECONDS) diff --git a/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexPersistBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexPersistBenchmark.java index 7bd825262815..9991b48dee01 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexPersistBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexPersistBenchmark.java @@ -32,7 +32,6 @@ import io.druid.java.util.common.logger.Logger; import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMerger; import io.druid.segment.IndexMergerV9; import io.druid.segment.IndexSpec; import io.druid.segment.column.ColumnConfig; @@ -84,7 +83,6 @@ public class IndexPersistBenchmark private BenchmarkSchemaInfo schemaInfo; - private static final IndexMerger INDEX_MERGER; private static final IndexMergerV9 INDEX_MERGER_V9; private static final IndexIO INDEX_IO; public static final ObjectMapper JSON_MAPPER; @@ -102,7 +100,6 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER = new IndexMerger(JSON_MAPPER, INDEX_IO); INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); } @@ -169,28 +166,6 @@ private IncrementalIndex makeIncIndex() ); } - @Benchmark - @BenchmarkMode(Mode.AverageTime) - @OutputTimeUnit(TimeUnit.MICROSECONDS) - public void persist(Blackhole blackhole) throws Exception - { - File tmpDir = Files.createTempDir(); - log.info("Using temp dir: " + tmpDir.getAbsolutePath()); - try { - File indexFile = INDEX_MERGER.persist( - incIndex, - tmpDir, - new IndexSpec() - ); - - blackhole.consume(indexFile); - } - finally { - FileUtils.deleteDirectory(tmpDir); - } - - } - @Benchmark @BenchmarkMode(Mode.AverageTime) @OutputTimeUnit(TimeUnit.MICROSECONDS) diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md index 7049f212b8b4..3660e27f8171 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -120,7 +120,6 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |`intermediatePersistPeriod`|ISO8601 Period|The period that determines the rate at which intermediate persists occur.|no (default == PT10M)| |`maxPendingPersists`|Integer|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 0, meaning one persist can be running concurrently with ingestion, and none can be queued up)| |`indexSpec`|Object|Tune how data is indexed, see 'IndexSpec' below for more details.|no| -|`buildV9Directly`|Boolean|Whether to build a v9 index directly instead of first building a v8 index and then converting it to v9 format.|no (default == true)| |`reportParseExceptions`|Boolean|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|no (default == false)| |`handoffConditionTimeout`|Long|Milliseconds to wait for segment handoff. It must be >= 0, where 0 means to wait forever. This option is deprecated. Use `completionTimeout` of KafkaSupervisorIOConfig instead.|no (default == 0)| |`resetOffsetAutomatically`|Boolean|Whether to reset the consumer offset if the next offset that it is trying to fetch is less than the earliest available offset for that particular partition. The consumer offset will be reset to either the earliest or latest offset depending on `useEarliestOffset` property of `KafkaSupervisorIOConfig` (see below). This situation typically occurs when messages in Kafka are no longer available for consumption and therefore won't be ingested into Druid. If set to false then ingestion for that particular partition will halt and manual intervention is required to correct the situation, please see `Reset Supervisor` API below.|no (default == false)| diff --git a/docs/content/ingestion/batch-ingestion.md b/docs/content/ingestion/batch-ingestion.md index aca687c22dff..b3625839d9c8 100644 --- a/docs/content/ingestion/batch-ingestion.md +++ b/docs/content/ingestion/batch-ingestion.md @@ -163,7 +163,6 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |useCombiner|Boolean|Use Hadoop combiner to merge rows at mapper if possible.|no (default == false)| |jobProperties|Object|A map of properties to add to the Hadoop job configuration, see below for details.|no (default == null)| |indexSpec|Object|Tune how data is indexed. See below for more information.|no| -|buildV9Directly|Boolean|Whether to build a v9 index directly instead of first building a v8 index and then converting it to v9 format.|no (default == true)| |numBackgroundPersistThreads|Integer|The number of new background threads to use for incremental persists. Using this feature causes a notable increase in memory pressure and cpu usage but will make the job finish more quickly. If changing from the default of 0 (use current thread for persists), we recommend setting it to 1.|no (default == 0)| |forceExtendableShardSpecs|Boolean|Forces use of extendable shardSpecs. Experimental feature intended for use with the [Kafka indexing service extension](../development/extensions-core/kafka-ingestion.html).|no (default = false)| |useExplicitVersion|Boolean|Forces HadoopIndexTask to use version.|no (default = false)| diff --git a/docs/content/ingestion/stream-pull.md b/docs/content/ingestion/stream-pull.md index ba37c2de0c61..7cd73c441af6 100644 --- a/docs/content/ingestion/stream-pull.md +++ b/docs/content/ingestion/stream-pull.md @@ -150,7 +150,6 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |rejectionPolicy|Object|Controls how data sets the data acceptance policy for creating and handing off segments. More on this below.|no (default == 'serverTime')| |maxPendingPersists|Integer|Maximum number of persists that can be pending, but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 0; meaning one persist can be running concurrently with ingestion, and none can be queued up)| |shardSpec|Object|This describes the shard that is represented by this server. This must be specified properly in order to have multiple realtime nodes indexing the same data stream in a [sharded fashion](#sharding).|no (default == 'NoneShardSpec')| -|buildV9Directly|Boolean|Whether to build a v9 index directly instead of first building a v8 index and then converting it to v9 format.|no (default == true)| |persistThreadPriority|int|If `-XX:+UseThreadPriorities` is properly enabled, this will set the thread priority of the persisting thread to `Thread.NORM_PRIORITY` plus this value within the bounds of `Thread.MIN_PRIORITY` and `Thread.MAX_PRIORITY`. A value of 0 indicates to not change the thread priority.|no (default == 0; inherit and do not override)| |mergeThreadPriority|int|If `-XX:+UseThreadPriorities` is properly enabled, this will set the thread priority of the merging thread to `Thread.NORM_PRIORITY` plus this value within the bounds of `Thread.MIN_PRIORITY` and `Thread.MAX_PRIORITY`. A value of 0 indicates to not change the thread priority.|no (default == 0; inherit and do not override)| |reportParseExceptions|Boolean|If true, exceptions encountered during parsing will be thrown and will halt ingestion. If false, unparseable rows and fields will be skipped. If an entire row is skipped, the "unparseable" counter will be incremented. If some fields in a row were parseable and some were not, the parseable fields will be indexed and the "unparseable" counter will not be incremented.|no (default == false)| diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md index db8be91e57eb..0647ff13f58b 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -117,7 +117,6 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |numShards|Directly specify the number of shards to create. If this is specified and 'intervals' is specified in the granularitySpec, the index task can skip the determine intervals/partitions pass through the data. numShards cannot be specified if targetPartitionSize is set.|null|no| |indexSpec|defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no| |maxPendingPersists|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|0 (meaning one persist can be running concurrently with ingestion, and none can be queued up)|no| -|buildV9Directly|Whether to build a v9 index directly instead of first building a v8 index and then converting it to v9 format.|true|no| |forceExtendableShardSpecs|Forces use of extendable shardSpecs. Experimental feature intended for use with the [Kafka indexing service extension](../development/extensions-core/kafka-ingestion.html).|false|no| |reportParseExceptions|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|false|no| |publishTimeout|Milliseconds to wait for publishing segments. It must be >= 0, where 0 means to wait forever.|0|no| @@ -162,7 +161,6 @@ Append tasks append a list of segments together into a single segment (one after "id": , "dataSource": , "segments": , - "buildV9Directly": , "aggregations": } ``` @@ -181,7 +179,6 @@ The grammar is: "dataSource": , "aggregations": , "rollup": , - "buildV9Directly": , "segments": } ``` @@ -199,7 +196,6 @@ The grammar is: "dataSource": , "aggregations": , "rollup": , - "buildV9Directly": , "interval": } ``` diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index f37b31c7de50..7e44c17d5379 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -869,7 +869,7 @@ private Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox toolbox.getSegmentPusher(), toolbox.getObjectMapper(), toolbox.getIndexIO(), - tuningConfig.getBuildV9Directly() ? toolbox.getIndexMergerV9() : toolbox.getIndexMerger(), + toolbox.getIndexMergerV9(), toolbox.getQueryRunnerFactoryConglomerate(), toolbox.getSegmentAnnouncer(), toolbox.getEmitter(), diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java index d257d7c98ec5..e9dc4463f80e 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java @@ -40,7 +40,6 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig private final File basePersistDirectory; private final int maxPendingPersists; private final IndexSpec indexSpec; - private final boolean buildV9Directly; private final boolean reportParseExceptions; @Deprecated private final long handoffConditionTimeout; @@ -54,6 +53,7 @@ public KafkaTuningConfig( @JsonProperty("basePersistDirectory") File basePersistDirectory, @JsonProperty("maxPendingPersists") Integer maxPendingPersists, @JsonProperty("indexSpec") IndexSpec indexSpec, + // This parameter is left for compatibility when reading existing configs, to be removed in Druid 0.12. @JsonProperty("buildV9Directly") Boolean buildV9Directly, @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, @@ -71,7 +71,6 @@ public KafkaTuningConfig( this.basePersistDirectory = defaults.getBasePersistDirectory(); this.maxPendingPersists = maxPendingPersists == null ? defaults.getMaxPendingPersists() : maxPendingPersists; this.indexSpec = indexSpec == null ? defaults.getIndexSpec() : indexSpec; - this.buildV9Directly = buildV9Directly == null ? defaults.getBuildV9Directly() : buildV9Directly; this.reportParseExceptions = reportParseExceptions == null ? defaults.isReportParseExceptions() : reportParseExceptions; @@ -92,7 +91,7 @@ public static KafkaTuningConfig copyOf(KafkaTuningConfig config) config.basePersistDirectory, config.maxPendingPersists, config.indexSpec, - config.buildV9Directly, + true, config.reportParseExceptions, config.handoffConditionTimeout, config.resetOffsetAutomatically @@ -140,10 +139,14 @@ public IndexSpec getIndexSpec() return indexSpec; } + /** + * Always returns true, doesn't affect the version being built. + */ + @Deprecated @JsonProperty public boolean getBuildV9Directly() { - return buildV9Directly; + return true; } @Override @@ -175,7 +178,7 @@ public KafkaTuningConfig withBasePersistDirectory(File dir) dir, maxPendingPersists, indexSpec, - buildV9Directly, + true, reportParseExceptions, handoffConditionTimeout, resetOffsetAutomatically @@ -191,7 +194,7 @@ public KafkaTuningConfig withMaxRowsInMemory(int rows) basePersistDirectory, maxPendingPersists, indexSpec, - buildV9Directly, + true, reportParseExceptions, handoffConditionTimeout, resetOffsetAutomatically @@ -219,9 +222,6 @@ public boolean equals(Object o) if (maxPendingPersists != that.maxPendingPersists) { return false; } - if (buildV9Directly != that.buildV9Directly) { - return false; - } if (reportParseExceptions != that.reportParseExceptions) { return false; } @@ -254,7 +254,6 @@ public int hashCode() result = 31 * result + (basePersistDirectory != null ? basePersistDirectory.hashCode() : 0); result = 31 * result + maxPendingPersists; result = 31 * result + (indexSpec != null ? indexSpec.hashCode() : 0); - result = 31 * result + (buildV9Directly ? 1 : 0); result = 31 * result + (reportParseExceptions ? 1 : 0); result = 31 * result + (int) (handoffConditionTimeout ^ (handoffConditionTimeout >>> 32)); result = 31 * result + (resetOffsetAutomatically ? 1 : 0); @@ -271,7 +270,6 @@ public String toString() ", basePersistDirectory=" + basePersistDirectory + ", maxPendingPersists=" + maxPendingPersists + ", indexSpec=" + indexSpec + - ", buildV9Directly=" + buildV9Directly + ", reportParseExceptions=" + reportParseExceptions + ", handoffConditionTimeout=" + handoffConditionTimeout + ", resetOffsetAutomatically=" + resetOffsetAutomatically + diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java index 39b9666396a1..ef83165e8ebd 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java @@ -43,6 +43,7 @@ public KafkaSupervisorTuningConfig( @JsonProperty("basePersistDirectory") File basePersistDirectory, @JsonProperty("maxPendingPersists") Integer maxPendingPersists, @JsonProperty("indexSpec") IndexSpec indexSpec, + // This parameter is left for compatibility when reading existing configs, to be removed in Druid 0.12. @JsonProperty("buildV9Directly") Boolean buildV9Directly, @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, // for backward compatibility @@ -62,7 +63,7 @@ public KafkaSupervisorTuningConfig( basePersistDirectory, maxPendingPersists, indexSpec, - buildV9Directly, + true, reportParseExceptions, // Supervised kafka tasks should respect KafkaSupervisorIOConfig.completionTimeout instead of // handoffConditionTimeout @@ -124,7 +125,6 @@ public String toString() ", basePersistDirectory=" + getBasePersistDirectory() + ", maxPendingPersists=" + getMaxPendingPersists() + ", indexSpec=" + getIndexSpec() + - ", buildV9Directly=" + getBuildV9Directly() + ", reportParseExceptions=" + isReportParseExceptions() + ", handoffConditionTimeout=" + getHandoffConditionTimeout() + ", resetOffsetAutomatically=" + isResetOffsetAutomatically() + diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index b908af860475..fd3761debff0 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -44,7 +44,6 @@ import com.metamx.metrics.MonitorScheduler; import io.druid.client.cache.CacheConfig; import io.druid.client.cache.MapCache; -import io.druid.java.util.common.StringUtils; import io.druid.concurrent.Execs; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.JSONParseSpec; @@ -75,6 +74,7 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.CompressionUtils; import io.druid.java.util.common.ISE; +import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequences; import io.druid.metadata.EntryExistsException; @@ -131,8 +131,6 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; import java.io.File; import java.io.IOException; @@ -146,7 +144,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -@RunWith(Parameterized.class) public class KafkaIndexTaskTest { private static final Logger log = new Logger(KafkaIndexTaskTest.class); @@ -160,7 +157,6 @@ public class KafkaIndexTaskTest private static int topicPostfix; private final List runningTasks = Lists.newArrayList(); - private final boolean buildV9Directly; private long handoffConditionTimeout = 0; private boolean reportParseExceptions = false; @@ -221,17 +217,6 @@ private static String getTopicName() @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); - @Parameterized.Parameters(name = "buildV9Directly = {0}") - public static Iterable constructorFeeder() - { - return ImmutableList.of(new Object[]{true}, new Object[]{false}); - } - - public KafkaIndexTaskTest(boolean buildV9Directly) - { - this.buildV9Directly = buildV9Directly; - } - @Rule public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule(); @@ -1401,7 +1386,7 @@ private KafkaIndexTask createTask( null, null, null, - buildV9Directly, + true, reportParseExceptions, handoffConditionTimeout, resetOffsetAutomatically @@ -1566,7 +1551,6 @@ public List getLocations() ) ), testUtils.getTestObjectMapper(), - testUtils.getTestIndexMerger(), testUtils.getTestIndexIO(), MapCache.create(1024), new CacheConfig(), diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java index d42b44c5475d..81aa36666836 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java @@ -61,7 +61,6 @@ public void testSerdeWithDefaults() throws Exception Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); Assert.assertEquals(0, config.getMaxPendingPersists()); Assert.assertEquals(new IndexSpec(), config.getIndexSpec()); - Assert.assertEquals(true, config.getBuildV9Directly()); Assert.assertEquals(false, config.isReportParseExceptions()); Assert.assertEquals(0, config.getHandoffConditionTimeout()); } @@ -76,7 +75,6 @@ public void testSerdeWithNonDefaults() throws Exception + " \"maxRowsPerSegment\": 100,\n" + " \"intermediatePersistPeriod\": \"PT1H\",\n" + " \"maxPendingPersists\": 100,\n" - + " \"buildV9Directly\": true,\n" + " \"reportParseExceptions\": true,\n" + " \"handoffConditionTimeout\": 100\n" + "}"; @@ -96,7 +94,6 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertEquals(100, config.getMaxRowsPerSegment()); Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); Assert.assertEquals(100, config.getMaxPendingPersists()); - Assert.assertEquals(true, config.getBuildV9Directly()); Assert.assertEquals(true, config.isReportParseExceptions()); Assert.assertEquals(100, config.getHandoffConditionTimeout()); } @@ -124,7 +121,6 @@ public void testCopyOf() throws Exception Assert.assertEquals(new File("/tmp/xxx"), copy.getBasePersistDirectory()); Assert.assertEquals(4, copy.getMaxPendingPersists()); Assert.assertEquals(new IndexSpec(), copy.getIndexSpec()); - Assert.assertEquals(true, copy.getBuildV9Directly()); Assert.assertEquals(true, copy.isReportParseExceptions()); Assert.assertEquals(5L, copy.getHandoffConditionTimeout()); } diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java index a7b69db857d7..e399dd0bd4a0 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java @@ -63,7 +63,6 @@ public void testSerdeWithDefaults() throws Exception Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); Assert.assertEquals(0, config.getMaxPendingPersists()); Assert.assertEquals(new IndexSpec(), config.getIndexSpec()); - Assert.assertEquals(true, config.getBuildV9Directly()); Assert.assertEquals(false, config.isReportParseExceptions()); Assert.assertEquals(0, config.getHandoffConditionTimeout()); Assert.assertNull(config.getWorkerThreads()); @@ -84,7 +83,6 @@ public void testSerdeWithNonDefaults() throws Exception + " \"maxRowsPerSegment\": 100,\n" + " \"intermediatePersistPeriod\": \"PT1H\",\n" + " \"maxPendingPersists\": 100,\n" - + " \"buildV9Directly\": false,\n" + " \"reportParseExceptions\": true,\n" + " \"handoffConditionTimeout\": 100,\n" + " \"workerThreads\": 12,\n" @@ -110,7 +108,6 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertEquals(100, config.getMaxRowsPerSegment()); Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); Assert.assertEquals(100, config.getMaxPendingPersists()); - Assert.assertEquals(false, config.getBuildV9Directly()); Assert.assertEquals(true, config.isReportParseExceptions()); Assert.assertEquals(100, config.getHandoffConditionTimeout()); Assert.assertEquals(12, (int) config.getWorkerThreads()); diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java index d035536cba0b..ef2b77155b03 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java @@ -50,7 +50,6 @@ import io.druid.java.util.common.guava.FunctionalIterable; import io.druid.java.util.common.logger.Logger; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMerger; import io.druid.segment.IndexMergerV9; import io.druid.segment.IndexSpec; import io.druid.segment.indexing.granularity.GranularitySpec; @@ -91,7 +90,6 @@ public class HadoopDruidIndexerConfig public static final Joiner TAB_JOINER = Joiner.on("\t"); public static final ObjectMapper JSON_MAPPER; public static final IndexIO INDEX_IO; - public static final IndexMerger INDEX_MERGER; public static final IndexMergerV9 INDEX_MERGER_V9; public static final HadoopKerberosConfig HADOOP_KERBEROS_CONFIG; public static final DataSegmentPusher DATA_SEGMENT_PUSHER; @@ -119,7 +117,6 @@ public void configure(Binder binder) ); JSON_MAPPER = injector.getInstance(ObjectMapper.class); INDEX_IO = injector.getInstance(IndexIO.class); - INDEX_MERGER = injector.getInstance(IndexMerger.class); INDEX_MERGER_V9 = injector.getInstance(IndexMergerV9.class); HADOOP_KERBEROS_CONFIG = injector.getInstance(HadoopKerberosConfig.class); DATA_SEGMENT_PUSHER = injector.getInstance(DataSegmentPusher.class); @@ -378,11 +375,6 @@ public int getShardSpecCount(Bucket bucket) return schema.getTuningConfig().getShardSpecs().get(bucket.time.getMillis()).size(); } - public boolean isBuildV9Directly() - { - return schema.getTuningConfig().getBuildV9Directly(); - } - /** * Job instance should have Configuration set (by calling {@link #addJobProperties(Job)} * or via injected system properties) before this method is called. The {@link PathSpec} may diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java index e64c0e7884df..bdf7fac00909 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java @@ -44,7 +44,6 @@ public class HadoopTuningConfig implements TuningConfig private static final IndexSpec DEFAULT_INDEX_SPEC = new IndexSpec(); private static final int DEFAULT_ROW_FLUSH_BOUNDARY = 75000; private static final boolean DEFAULT_USE_COMBINER = false; - private static final Boolean DEFAULT_BUILD_V9_DIRECTLY = Boolean.TRUE; private static final int DEFAULT_NUM_BACKGROUND_PERSIST_THREADS = 0; public static HadoopTuningConfig makeDefaultTuningConfig() @@ -64,7 +63,7 @@ public static HadoopTuningConfig makeDefaultTuningConfig() false, false, null, - DEFAULT_BUILD_V9_DIRECTLY, + true, DEFAULT_NUM_BACKGROUND_PERSIST_THREADS, false, false, @@ -85,7 +84,6 @@ public static HadoopTuningConfig makeDefaultTuningConfig() private final Map jobProperties; private final boolean combineText; private final boolean useCombiner; - private final Boolean buildV9Directly; private final int numBackgroundPersistThreads; private final boolean forceExtendableShardSpecs; private final boolean useExplicitVersion; @@ -108,6 +106,7 @@ public HadoopTuningConfig( final @JsonProperty("useCombiner") Boolean useCombiner, // See https://github.com/druid-io/druid/pull/1922 final @JsonProperty("rowFlushBoundary") Integer maxRowsInMemoryCOMPAT, + // This parameter is left for compatibility when reading existing configs, to be removed in Druid 0.12. final @JsonProperty("buildV9Directly") Boolean buildV9Directly, final @JsonProperty("numBackgroundPersistThreads") Integer numBackgroundPersistThreads, final @JsonProperty("forceExtendableShardSpecs") boolean forceExtendableShardSpecs, @@ -132,7 +131,6 @@ public HadoopTuningConfig( : ImmutableMap.copyOf(jobProperties)); this.combineText = combineText; this.useCombiner = useCombiner == null ? DEFAULT_USE_COMBINER : useCombiner.booleanValue(); - this.buildV9Directly = buildV9Directly == null ? DEFAULT_BUILD_V9_DIRECTLY : buildV9Directly; this.numBackgroundPersistThreads = numBackgroundPersistThreads == null ? DEFAULT_NUM_BACKGROUND_PERSIST_THREADS : numBackgroundPersistThreads; @@ -222,10 +220,14 @@ public boolean getUseCombiner() return useCombiner; } + /** + * Always returns true, doesn't affect the version being built. + */ + @Deprecated @JsonProperty public Boolean getBuildV9Directly() { - return buildV9Directly; + return true; } @JsonProperty @@ -263,7 +265,7 @@ public HadoopTuningConfig withWorkingPath(String path) combineText, useCombiner, null, - buildV9Directly, + true, numBackgroundPersistThreads, forceExtendableShardSpecs, useExplicitVersion, @@ -288,7 +290,7 @@ public HadoopTuningConfig withVersion(String ver) combineText, useCombiner, null, - buildV9Directly, + true, numBackgroundPersistThreads, forceExtendableShardSpecs, useExplicitVersion, @@ -313,7 +315,7 @@ public HadoopTuningConfig withShardSpecs(Map> specs combineText, useCombiner, null, - buildV9Directly, + true, numBackgroundPersistThreads, forceExtendableShardSpecs, useExplicitVersion, diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java index 2a07fa521590..e4f9a0f1c2b4 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -505,15 +505,9 @@ private File persist( final ProgressIndicator progressIndicator ) throws IOException { - if (config.isBuildV9Directly()) { - return HadoopDruidIndexerConfig.INDEX_MERGER_V9.persist( - index, interval, file, config.getIndexSpec(), progressIndicator - ); - } else { - return HadoopDruidIndexerConfig.INDEX_MERGER.persist( - index, interval, file, config.getIndexSpec(), progressIndicator - ); - } + return HadoopDruidIndexerConfig.INDEX_MERGER_V9.persist( + index, interval, file, config.getIndexSpec(), progressIndicator + ); } protected File mergeQueryableIndex( @@ -524,15 +518,9 @@ protected File mergeQueryableIndex( ) throws IOException { boolean rollup = config.getSchema().getDataSchema().getGranularitySpec().isRollup(); - if (config.isBuildV9Directly()) { - return HadoopDruidIndexerConfig.INDEX_MERGER_V9.mergeQueryableIndex( - indexes, rollup, aggs, file, config.getIndexSpec(), progressIndicator - ); - } else { - return HadoopDruidIndexerConfig.INDEX_MERGER.mergeQueryableIndex( - indexes, rollup, aggs, file, config.getIndexSpec(), progressIndicator - ); - } + return HadoopDruidIndexerConfig.INDEX_MERGER_V9.mergeQueryableIndex( + indexes, rollup, aggs, file, config.getIndexSpec(), progressIndicator + ); } @Override diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/updater/HadoopConverterJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/updater/HadoopConverterJob.java index ab92c7032051..5fd7b8d6677f 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/updater/HadoopConverterJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/updater/HadoopConverterJob.java @@ -529,12 +529,18 @@ protected void map( context.progress(); final File outDir = new File(tmpDir, "out"); FileUtils.forceMkdir(outDir); - HadoopDruidConverterConfig.INDEX_MERGER.convert( - inDir, - outDir, - config.getIndexSpec(), - JobHelper.progressIndicatorForContext(context) - ); + try { + HadoopDruidConverterConfig.INDEX_MERGER.convert( + inDir, + outDir, + config.getIndexSpec(), + JobHelper.progressIndicatorForContext(context) + ); + } + catch (Exception e) { + log.error(e, "Conversion failed."); + throw e; + } if (config.isValidate()) { context.setStatus("Validating"); HadoopDruidConverterConfig.INDEX_IO.validateTwoSegments(inDir, outDir); diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java index 183eb1362a87..c8e079fbb86f 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java @@ -89,7 +89,7 @@ public class IndexGeneratorJobTest @Parameterized.Parameters(name = "useCombiner={0}, partitionType={1}, interval={2}, shardInfoForEachSegment={3}, " + "data={4}, inputFormatName={5}, inputRowParser={6}, maxRowsInMemory={7}, " + - "aggs={8}, datasourceName={9}, forceExtendableShardSpecs={10}, buildV9Directly={11}") + "aggs={8}, datasourceName={9}, forceExtendableShardSpecs={10}") public static Collection constructFeed() { final List baseConstructors = Arrays.asList( @@ -371,17 +371,14 @@ public static Collection constructFeed() } ); - // Run each baseConstructor with/without buildV9Directly and forceExtendableShardSpecs. + // Run each baseConstructor with/without forceExtendableShardSpecs. final List constructors = Lists.newArrayList(); for (Object[] baseConstructor : baseConstructors) { - for (int buildV9Directly = 0; buildV9Directly < 2; buildV9Directly++) { - for (int forceExtendableShardSpecs = 0; forceExtendableShardSpecs < 2 ; forceExtendableShardSpecs++) { - final Object[] fullConstructor = new Object[baseConstructor.length + 2]; - System.arraycopy(baseConstructor, 0, fullConstructor, 0, baseConstructor.length); - fullConstructor[baseConstructor.length] = forceExtendableShardSpecs == 0; - fullConstructor[baseConstructor.length + 1] = buildV9Directly == 0; - constructors.add(fullConstructor); - } + for (int forceExtendableShardSpecs = 0; forceExtendableShardSpecs < 2 ; forceExtendableShardSpecs++) { + final Object[] fullConstructor = new Object[baseConstructor.length + 2]; + System.arraycopy(baseConstructor, 0, fullConstructor, 0, baseConstructor.length); + fullConstructor[baseConstructor.length] = forceExtendableShardSpecs == 0; + constructors.add(fullConstructor); } } @@ -402,7 +399,6 @@ public static Collection constructFeed() private final AggregatorFactory[] aggs; private final String datasourceName; private final boolean forceExtendableShardSpecs; - private final boolean buildV9Directly; private ObjectMapper mapper; private HadoopDruidIndexerConfig config; @@ -420,8 +416,7 @@ public IndexGeneratorJobTest( Integer maxRowsInMemory, AggregatorFactory[] aggs, String datasourceName, - boolean forceExtendableShardSpecs, - boolean buildV9Directly + boolean forceExtendableShardSpecs ) throws IOException { this.useCombiner = useCombiner; @@ -435,7 +430,6 @@ public IndexGeneratorJobTest( this.aggs = aggs; this.datasourceName = datasourceName; this.forceExtendableShardSpecs = forceExtendableShardSpecs; - this.buildV9Directly = buildV9Directly; } private void writeDataToLocalSequenceFile(File outputFile, List data) throws IOException @@ -522,7 +516,7 @@ public void setUp() throws Exception false, useCombiner, null, - buildV9Directly, + true, null, forceExtendableShardSpecs, false, diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java index 8671432b77be..3365471359b9 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java @@ -36,7 +36,6 @@ import io.druid.indexing.common.task.Task; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMerger; import io.druid.segment.IndexMergerV9; import io.druid.segment.loading.DataSegmentArchiver; import io.druid.segment.loading.DataSegmentKiller; @@ -79,7 +78,6 @@ public class TaskToolbox private final SegmentLoader segmentLoader; private final ObjectMapper objectMapper; private final File taskWorkDir; - private final IndexMerger indexMerger; private final IndexIO indexIO; private final Cache cache; private final CacheConfig cacheConfig; @@ -103,7 +101,6 @@ public TaskToolbox( SegmentLoader segmentLoader, ObjectMapper objectMapper, File taskWorkDir, - IndexMerger indexMerger, IndexIO indexIO, Cache cache, CacheConfig cacheConfig, @@ -127,7 +124,6 @@ public TaskToolbox( this.segmentLoader = segmentLoader; this.objectMapper = objectMapper; this.taskWorkDir = taskWorkDir; - this.indexMerger = Preconditions.checkNotNull(indexMerger, "Null IndexMerger"); this.indexIO = Preconditions.checkNotNull(indexIO, "Null IndexIO"); this.cache = cache; this.cacheConfig = cacheConfig; @@ -244,11 +240,6 @@ public IndexIO getIndexIO() return indexIO; } - public IndexMerger getIndexMerger() - { - return indexMerger; - } - public Cache getCache() { return cache; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java index 7527d1b8761b..5bf963bdca59 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java @@ -32,7 +32,6 @@ import io.druid.indexing.common.task.Task; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMerger; import io.druid.segment.IndexMergerV9; import io.druid.segment.loading.DataSegmentArchiver; import io.druid.segment.loading.DataSegmentKiller; @@ -65,7 +64,6 @@ public class TaskToolboxFactory private final MonitorScheduler monitorScheduler; private final SegmentLoaderFactory segmentLoaderFactory; private final ObjectMapper objectMapper; - private final IndexMerger indexMerger; private final IndexIO indexIO; private final Cache cache; private final CacheConfig cacheConfig; @@ -88,7 +86,6 @@ public TaskToolboxFactory( MonitorScheduler monitorScheduler, SegmentLoaderFactory segmentLoaderFactory, ObjectMapper objectMapper, - IndexMerger indexMerger, IndexIO indexIO, Cache cache, CacheConfig cacheConfig, @@ -110,7 +107,6 @@ public TaskToolboxFactory( this.monitorScheduler = monitorScheduler; this.segmentLoaderFactory = segmentLoaderFactory; this.objectMapper = objectMapper; - this.indexMerger = Preconditions.checkNotNull(indexMerger, "Null IndexMerger"); this.indexIO = Preconditions.checkNotNull(indexIO, "Null IndexIO"); this.cache = cache; this.cacheConfig = cacheConfig; @@ -138,7 +134,6 @@ public TaskToolbox build(Task task) segmentLoaderFactory.manufacturate(taskWorkDir), objectMapper, taskWorkDir, - indexMerger, indexIO, cache, cacheConfig, diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java index 375493acfebd..24dde83b8f30 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java @@ -69,7 +69,6 @@ public class YeOldePlumberSchool implements PlumberSchool private final String version; private final DataSegmentPusher dataSegmentPusher; private final File tmpSegmentDir; - private final IndexMerger indexMerger; private final IndexMergerV9 indexMergerV9; private final IndexIO indexIO; @@ -81,7 +80,6 @@ public YeOldePlumberSchool( @JsonProperty("version") String version, @JacksonInject("segmentPusher") DataSegmentPusher dataSegmentPusher, @JacksonInject("tmpSegmentDir") File tmpSegmentDir, - @JacksonInject IndexMerger indexMerger, @JacksonInject IndexMergerV9 indexMergerV9, @JacksonInject IndexIO indexIO ) @@ -90,7 +88,6 @@ public YeOldePlumberSchool( this.version = version; this.dataSegmentPusher = dataSegmentPusher; this.tmpSegmentDir = tmpSegmentDir; - this.indexMerger = Preconditions.checkNotNull(indexMerger, "Null IndexMerger"); this.indexMergerV9 = Preconditions.checkNotNull(indexMergerV9, "Null IndexMergerV9"); this.indexIO = Preconditions.checkNotNull(indexIO, "Null IndexIO"); } @@ -119,7 +116,7 @@ public Plumber findPlumber( final Set spilled = Sets.newHashSet(); // IndexMerger implementation. - final IndexMerger theIndexMerger = config.getBuildV9Directly() ? indexMergerV9 : indexMerger; + final IndexMerger theIndexMerger = indexMergerV9; return new Plumber() { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java index 98e0329163db..8cbb8c434d3b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java @@ -50,10 +50,8 @@ */ public class AppendTask extends MergeTaskBase { - private static final Boolean defaultBuildV9Directly = Boolean.TRUE; private final IndexSpec indexSpec; private final List aggregators; - private final Boolean buildV9Directly; @JsonCreator public AppendTask( @@ -62,6 +60,7 @@ public AppendTask( @JsonProperty("segments") List segments, @JsonProperty("aggregations") List aggregators, @JsonProperty("indexSpec") IndexSpec indexSpec, + // This parameter is left for compatibility when reading existing JSONs, to be removed in Druid 0.12. @JsonProperty("buildV9Directly") Boolean buildV9Directly, @JsonProperty("context") Map context ) @@ -69,7 +68,6 @@ public AppendTask( super(id, dataSource, segments, context); this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec; this.aggregators = aggregators; - this.buildV9Directly = buildV9Directly == null ? defaultBuildV9Directly : buildV9Directly; } @Override @@ -135,7 +133,7 @@ public boolean apply(Rowboat input) ); } - IndexMerger indexMerger = buildV9Directly ? toolbox.getIndexMergerV9() : toolbox.getIndexMerger(); + IndexMerger indexMerger = toolbox.getIndexMergerV9(); return indexMerger.append( adapters, aggregators == null ? null : aggregators.toArray(new AggregatorFactory[aggregators.size()]), diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ConvertSegmentTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ConvertSegmentTask.java index dc4bb3b59d53..94b295ac3829 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ConvertSegmentTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ConvertSegmentTask.java @@ -351,7 +351,13 @@ public String getType() public TaskStatus run(TaskToolbox toolbox) throws Exception { log.info("Subs are good! Italian BMT and Meatball are probably my favorite."); - convertSegment(toolbox, segment, indexSpec, force, validate); + try { + convertSegment(toolbox, segment, indexSpec, force, validate); + } + catch (Exception e) { + log.error(e, "Conversion failed."); + throw e; + } return success(); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index f6774843ee7c..bf8e008288b2 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -539,7 +539,7 @@ private Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox toolbox.getSegmentPusher(), toolbox.getObjectMapper(), toolbox.getIndexIO(), - ingestionSchema.getTuningConfig().isBuildV9Directly() ? toolbox.getIndexMergerV9() : toolbox.getIndexMerger() + toolbox.getIndexMergerV9() ); } @@ -579,7 +579,7 @@ public IndexIngestionSpec( this.ioConfig = ioConfig; this.tuningConfig = tuningConfig == null ? - new IndexTuningConfig(null, null, null, null, null, null, null, null, null, (File) null) + new IndexTuningConfig(null, null, null, null, null, null, null, null, (File) null) : tuningConfig; } @@ -655,7 +655,6 @@ public static class IndexTuningConfig implements TuningConfig, AppenderatorConfi private final IndexSpec indexSpec; private final File basePersistDirectory; private final int maxPendingPersists; - private final boolean buildV9Directly; private final boolean forceExtendableShardSpecs; private final boolean reportParseExceptions; private final long publishTimeout; @@ -668,6 +667,7 @@ public IndexTuningConfig( @JsonProperty("numShards") @Nullable Integer numShards, @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec, @JsonProperty("maxPendingPersists") @Nullable Integer maxPendingPersists, + // This parameter is left for compatibility when reading existing JSONs, to be removed in Druid 0.12. @JsonProperty("buildV9Directly") @Nullable Boolean buildV9Directly, @JsonProperty("forceExtendableShardSpecs") @Nullable Boolean forceExtendableShardSpecs, @JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions, @@ -680,7 +680,6 @@ public IndexTuningConfig( numShards, indexSpec, maxPendingPersists, - buildV9Directly, forceExtendableShardSpecs, reportParseExceptions, publishTimeout, @@ -694,7 +693,6 @@ private IndexTuningConfig( @Nullable Integer numShards, @Nullable IndexSpec indexSpec, @Nullable Integer maxPendingPersists, - @Nullable Boolean buildV9Directly, @Nullable Boolean forceExtendableShardSpecs, @Nullable Boolean reportParseExceptions, @Nullable Long publishTimeout, @@ -715,7 +713,6 @@ private IndexTuningConfig( this.numShards = numShards == null || numShards.equals(-1) ? null : numShards; this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec; this.maxPendingPersists = maxPendingPersists == null ? DEFAULT_MAX_PENDING_PERSISTS : maxPendingPersists; - this.buildV9Directly = buildV9Directly == null ? DEFAULT_BUILD_V9_DIRECTLY : buildV9Directly; this.forceExtendableShardSpecs = forceExtendableShardSpecs == null ? DEFAULT_FORCE_EXTENDABLE_SHARD_SPECS : forceExtendableShardSpecs; @@ -734,7 +731,6 @@ public IndexTuningConfig withBasePersistDirectory(File dir) numShards, indexSpec, maxPendingPersists, - buildV9Directly, forceExtendableShardSpecs, reportParseExceptions, publishTimeout, @@ -781,10 +777,14 @@ public int getMaxPendingPersists() return maxPendingPersists; } + /** + * Always returns true, doesn't affect the version being built. + */ + @Deprecated @JsonProperty public boolean isBuildV9Directly() { - return buildV9Directly; + return true; } @JsonProperty diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java index 812dcc06e61c..d897df5c089f 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java @@ -43,12 +43,10 @@ */ public class MergeTask extends MergeTaskBase { - private static final Boolean defaultBuildV9Directly = Boolean.TRUE; @JsonIgnore private final List aggregators; private final Boolean rollup; private final IndexSpec indexSpec; - private final Boolean buildV9Directly; @JsonCreator public MergeTask( @@ -58,6 +56,7 @@ public MergeTask( @JsonProperty("aggregations") List aggregators, @JsonProperty("rollup") Boolean rollup, @JsonProperty("indexSpec") IndexSpec indexSpec, + // This parameter is left for compatibility when reading existing JSONs, to be removed in Druid 0.12. @JsonProperty("buildV9Directly") Boolean buildV9Directly, @JsonProperty("context") Map context ) @@ -66,14 +65,13 @@ public MergeTask( this.aggregators = Preconditions.checkNotNull(aggregators, "null aggregations"); this.rollup = rollup == null ? Boolean.TRUE : rollup; this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec; - this.buildV9Directly = buildV9Directly == null ? defaultBuildV9Directly : buildV9Directly; } @Override public File merge(final TaskToolbox toolbox, final Map segments, final File outDir) throws Exception { - IndexMerger indexMerger = buildV9Directly ? toolbox.getIndexMergerV9() : toolbox.getIndexMerger(); + IndexMerger indexMerger = toolbox.getIndexMergerV9(); return indexMerger.mergeQueryableIndex( Lists.transform( ImmutableList.copyOf(segments.values()), diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index 1e5724f8b8e3..3e36acff1213 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -308,7 +308,6 @@ public String getVersion(final Interval interval) segmentPublisher, toolbox.getSegmentHandoffNotifierFactory(), toolbox.getQueryExecutorService(), - toolbox.getIndexMerger(), toolbox.getIndexMergerV9(), toolbox.getIndexIO(), toolbox.getCache(), diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SameIntervalMergeTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SameIntervalMergeTask.java index 0af2b7612823..3a112e83ebb6 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SameIntervalMergeTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SameIntervalMergeTask.java @@ -38,13 +38,11 @@ */ public class SameIntervalMergeTask extends AbstractFixedIntervalTask { - private static final Boolean defaultBuildV9Directly = Boolean.TRUE; private static final String TYPE = "same_interval_merge"; @JsonIgnore private final List aggregators; private final Boolean rollup; private final IndexSpec indexSpec; - private final Boolean buildV9Directly; public SameIntervalMergeTask( @JsonProperty("id") String id, @@ -53,6 +51,7 @@ public SameIntervalMergeTask( @JsonProperty("aggregations") List aggregators, @JsonProperty("rollup") Boolean rollup, @JsonProperty("indexSpec") IndexSpec indexSpec, + // This parameter is left for compatibility when reading existing JSONs, to be removed in Druid 0.12. @JsonProperty("buildV9Directly") Boolean buildV9Directly, @JsonProperty("context") Map context ) @@ -66,7 +65,6 @@ public SameIntervalMergeTask( this.aggregators = Preconditions.checkNotNull(aggregators, "null aggregations"); this.rollup = rollup == null ? Boolean.TRUE : rollup; this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec; - this.buildV9Directly = buildV9Directly == null ? defaultBuildV9Directly : buildV9Directly; } @JsonProperty("aggregations") @@ -87,10 +85,14 @@ public IndexSpec getIndexSpec() return indexSpec; } + /** + * Always returns true, doesn't affect the version being built. + */ + @Deprecated @JsonProperty public Boolean getBuildV9Directly() { - return buildV9Directly; + return true; } public static String makeId(String id, final String typeName, String dataSource, Interval interval) @@ -127,7 +129,6 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception aggregators, rollup, indexSpec, - buildV9Directly, getContext() ); final TaskStatus status = mergeTask.run(toolbox); @@ -146,7 +147,6 @@ public SubTask( List aggregators, Boolean rollup, IndexSpec indexSpec, - Boolean buildV9Directly, Map context ) { @@ -157,7 +157,7 @@ public SubTask( aggregators, rollup, indexSpec, - buildV9Directly, + true, context ); } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java b/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java index 76e23a4f393f..be8d028b9415 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java @@ -30,7 +30,6 @@ import io.druid.indexing.common.task.Task; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMerger; import io.druid.segment.IndexMergerV9; import io.druid.segment.loading.DataSegmentArchiver; import io.druid.segment.loading.DataSegmentKiller; @@ -78,7 +77,6 @@ public class TaskToolboxTest private ObjectMapper ObjectMapper = new ObjectMapper(); private SegmentLoaderLocalCacheManager mockSegmentLoaderLocalCacheManager = EasyMock.createMock(SegmentLoaderLocalCacheManager.class); private Task task = EasyMock.createMock(Task.class); - private IndexMerger mockIndexMerger = EasyMock.createMock(IndexMerger.class); private IndexMergerV9 mockIndexMergerV9 = EasyMock.createMock(IndexMergerV9.class); private IndexIO mockIndexIO = EasyMock.createMock(IndexIO.class); private Cache mockCache = EasyMock.createMock(Cache.class); @@ -110,7 +108,6 @@ public void setUp() throws IOException mockMonitorScheduler, new SegmentLoaderFactory(mockSegmentLoaderLocalCacheManager), ObjectMapper, - mockIndexMerger, mockIndexIO, mockCache, mockCacheConfig, diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java index 39ebba28fc90..506799085c31 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java @@ -29,7 +29,6 @@ import io.druid.math.expr.ExprMacroTable; import io.druid.query.expression.TestExprMacroTable; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMerger; import io.druid.segment.IndexMergerV9; import io.druid.segment.column.ColumnConfig; import io.druid.segment.realtime.firehose.ChatHandlerProvider; @@ -43,7 +42,6 @@ public class TestUtils { private final ObjectMapper jsonMapper; - private final IndexMerger indexMerger; private final IndexMergerV9 indexMergerV9; private final IndexIO indexIO; @@ -61,7 +59,6 @@ public int columnCacheSizeBytes() } } ); - indexMerger = new IndexMerger(jsonMapper, indexIO); indexMergerV9 = new IndexMergerV9(jsonMapper, indexIO); final List list = new ServerModule().getJacksonModules(); @@ -73,7 +70,6 @@ public int columnCacheSizeBytes() new InjectableValues.Std() .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) .addValue(IndexIO.class, indexIO) - .addValue(IndexMerger.class, indexMerger) .addValue(ObjectMapper.class, jsonMapper) .addValue(ChatHandlerProvider.class, new NoopChatHandlerProvider()) ); @@ -84,11 +80,6 @@ public ObjectMapper getTestObjectMapper() return jsonMapper; } - public IndexMerger getTestIndexMerger() - { - return indexMerger; - } - public IndexMergerV9 getTestIndexMergerV9() { return indexMergerV9; diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index 2881ab0c5196..4eccfabb5aeb 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -42,7 +42,6 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMerger; import io.druid.segment.IndexMergerV9; import io.druid.segment.IndexSpec; import io.druid.segment.indexing.DataSchema; @@ -98,7 +97,6 @@ public class IndexTaskTest private final IndexSpec indexSpec; private final ObjectMapper jsonMapper; - private IndexMerger indexMerger; private IndexMergerV9 indexMergerV9; private IndexIO indexIO; private volatile int segmentAllocatePartitionCounter; @@ -108,7 +106,6 @@ public IndexTaskTest() indexSpec = new IndexSpec(); TestUtils testUtils = new TestUtils(); jsonMapper = testUtils.getTestObjectMapper(); - indexMerger = testUtils.getTestIndexMerger(); indexMergerV9 = testUtils.getTestIndexMergerV9(); indexIO = testUtils.getTestIndexIO(); } @@ -559,7 +556,7 @@ public Map makeLoadSpec(URI uri) throw new UnsupportedOperationException(); } }, null, null, null, null, null, null, null, null, null, null, jsonMapper, temporaryFolder.newFolder(), - indexMerger, indexIO, null, null, indexMergerV9 + indexIO, null, null, indexMergerV9 ) ); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index 78c48b0734dd..82d09be4c44d 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -116,22 +116,18 @@ import org.junit.internal.matchers.ThrowableMessageMatcher; import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; import java.io.File; import java.io.IOException; import java.nio.file.Files; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.List; import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; -@RunWith(Parameterized.class) public class RealtimeIndexTaskTest { private static final Logger log = new Logger(RealtimeIndexTaskTest.class); @@ -232,26 +228,10 @@ public Firehose connect(InputRowParser parser, File temporaryDirectory) throws I @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); - private final boolean buildV9Directly; - private DateTime now; private ListeningExecutorService taskExec; private Map> handOffCallbacks; - @Parameterized.Parameters(name = "buildV9Directly = {0}") - public static Collection constructorFeeder() throws IOException - { - return ImmutableList.of( - new Object[]{true}, - new Object[]{false} - ); - } - - public RealtimeIndexTaskTest(boolean buildV9Directly) - { - this.buildV9Directly = buildV9Directly; - } - @Before public void setUp() { @@ -907,7 +887,7 @@ private RealtimeIndexTask makeRealtimeTask(final String taskId, boolean reportPa null, null, null, - buildV9Directly, + true, 0, 0, reportParseExceptions, @@ -1062,7 +1042,6 @@ public List getLocations() ) ), testUtils.getTestObjectMapper(), - testUtils.getTestIndexMerger(), testUtils.getTestIndexIO(), MapCache.create(1024), new CacheConfig(), diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java index 9f2bd7f78232..790349eb01ea 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java @@ -34,7 +34,6 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMerger; import io.druid.segment.IndexMergerV9; import io.druid.segment.IndexSpec; import io.druid.segment.Segment; @@ -234,7 +233,7 @@ public void cleanup(DataSegment segment) throws SegmentLoadingException { } }, jsonMapper, temporaryFolder.newFolder(), - EasyMock.createMock(IndexMerger.class), indexIO, null, null, EasyMock.createMock(IndexMergerV9.class) + indexIO, null, null, EasyMock.createMock(IndexMergerV9.class) ) ); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index b74780fc38a3..5a3161dec802 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -98,7 +98,6 @@ public void testIndexTaskTuningConfigDefaults() throws Exception IndexTask.IndexTuningConfig.class ); - Assert.assertEquals(true, tuningConfig.isBuildV9Directly()); Assert.assertEquals(false, tuningConfig.isForceExtendableShardSpecs()); Assert.assertEquals(false, tuningConfig.isReportParseExceptions()); Assert.assertEquals(new IndexSpec(), tuningConfig.getIndexSpec()); @@ -223,7 +222,6 @@ public void testIndexTaskSerde() throws Exception Assert.assertEquals(taskTuningConfig.getMaxRowsInMemory(), task2TuningConfig.getMaxRowsInMemory()); Assert.assertEquals(taskTuningConfig.getNumShards(), task2TuningConfig.getNumShards()); Assert.assertEquals(taskTuningConfig.getTargetPartitionSize(), task2TuningConfig.getTargetPartitionSize()); - Assert.assertEquals(taskTuningConfig.isBuildV9Directly(), task2TuningConfig.isBuildV9Directly()); Assert.assertEquals( taskTuningConfig.isForceExtendableShardSpecs(), task2TuningConfig.isForceExtendableShardSpecs() @@ -362,7 +360,6 @@ public void testSameIntervalMergeTaskSerde() throws Exception Assert.assertEquals(task.getDataSource(), task2.getDataSource()); Assert.assertEquals(task.getInterval(), task2.getInterval()); Assert.assertEquals(task.getRollup(), task2.getRollup()); - Assert.assertEquals(task.getBuildV9Directly(), task2.getBuildV9Directly()); Assert.assertEquals(task.getIndexSpec(), task2.getIndexSpec()); Assert.assertEquals( task.getAggregators().get(0).getName(), diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java index 71b656e57b44..1cb107ad6f4c 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java @@ -62,7 +62,6 @@ import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.filter.SelectorDimFilter; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMerger; import io.druid.segment.IndexMergerV9; import io.druid.segment.IndexSpec; import io.druid.segment.incremental.IncrementalIndexSchema; @@ -110,14 +109,12 @@ public class IngestSegmentFirehoseFactoryTest { private static final ObjectMapper MAPPER; - private static final IndexMerger INDEX_MERGER; private static final IndexMergerV9 INDEX_MERGER_V9; private static final IndexIO INDEX_IO; static { TestUtils testUtils = new TestUtils(); MAPPER = setupInjectablesInObjectMapper(testUtils.getTestObjectMapper()); - INDEX_MERGER = testUtils.getTestIndexMerger(); INDEX_MERGER_V9 = testUtils.getTestIndexMergerV9(); INDEX_IO = testUtils.getTestIndexIO(); } @@ -156,7 +153,7 @@ public static Collection constructorFeeder() throws IOException if (!persistDir.mkdirs() && !persistDir.exists()) { throw new IOException(String.format("Could not create directory at [%s]", persistDir.getAbsolutePath())); } - INDEX_MERGER.persist(index, persistDir, indexSpec); + INDEX_MERGER_V9.persist(index, persistDir, indexSpec); final TaskLockbox tl = new TaskLockbox(ts); final IndexerSQLMetadataStorageCoordinator mdc = new IndexerSQLMetadataStorageCoordinator(null, null, null) @@ -296,7 +293,6 @@ public List getLocations() ) ), MAPPER, - INDEX_MERGER, INDEX_IO, null, null, diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java index 7b9b1c972540..2092ef6c98c2 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java @@ -54,7 +54,6 @@ import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.filter.NoopDimFilter; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMerger; import io.druid.segment.IndexMergerV9; import io.druid.segment.IndexSpec; import io.druid.segment.incremental.IncrementalIndexSchema; @@ -112,14 +111,12 @@ public class IngestSegmentFirehoseFactoryTimelineTest private final long expectedSum; private static final ObjectMapper MAPPER; - private static final IndexMerger INDEX_MERGER; private static final IndexIO INDEX_IO; private static final IndexMergerV9 INDEX_MERGER_V9; static { TestUtils testUtils = new TestUtils(); MAPPER = IngestSegmentFirehoseFactoryTest.setupInjectablesInObjectMapper(testUtils.getTestObjectMapper()); - INDEX_MERGER = testUtils.getTestIndexMerger(); INDEX_IO = testUtils.getTestIndexIO(); INDEX_MERGER_V9 = testUtils.getTestIndexMergerV9(); } @@ -232,7 +229,7 @@ private static Map persist(File tmpDir, InputRow... rows) } try { - INDEX_MERGER.persist(index, persistDir, new IndexSpec()); + INDEX_MERGER_V9.persist(index, persistDir, new IndexSpec()); } catch (IOException e) { throw Throwables.propagate(e); @@ -337,7 +334,6 @@ public List getLocations() ) ), MAPPER, - INDEX_MERGER, INDEX_IO, null, null, diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index b001cfceda8c..64d0ca48c3d3 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -80,7 +80,6 @@ import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMerger; import io.druid.segment.IndexMergerV9; import io.druid.segment.IndexSpec; import io.druid.segment.indexing.DataSchema; @@ -136,7 +135,6 @@ public class TaskLifecycleTest { private static final ObjectMapper MAPPER; - private static final IndexMerger INDEX_MERGER; private static final IndexMergerV9 INDEX_MERGER_V9; private static final IndexIO INDEX_IO; private static final TestUtils TEST_UTILS; @@ -144,7 +142,6 @@ public class TaskLifecycleTest static { TEST_UTILS = new TestUtils(); MAPPER = TEST_UTILS.getTestObjectMapper(); - INDEX_MERGER = TEST_UTILS.getTestIndexMerger(); INDEX_MERGER_V9 = TEST_UTILS.getTestIndexMergerV9(); INDEX_IO = TEST_UTILS.getTestIndexIO(); } @@ -594,7 +591,6 @@ public List getLocations() ) ), MAPPER, - INDEX_MERGER, INDEX_IO, MapCache.create(0), FireDepartmentTest.NO_CACHE_CONFIG, diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java index 05e48011b84b..524115a5b263 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -39,7 +39,6 @@ import io.druid.indexing.overlord.TestRemoteTaskRunnerConfig; import io.druid.indexing.overlord.ThreadPoolTaskRunner; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMerger; import io.druid.segment.IndexMergerV9; import io.druid.segment.loading.SegmentLoaderConfig; import io.druid.segment.loading.SegmentLoaderLocalCacheManager; @@ -81,7 +80,6 @@ public class WorkerTaskMonitorTest private Worker worker; private ObjectMapper jsonMapper; - private IndexMerger indexMerger; private IndexMergerV9 indexMergerV9; private IndexIO indexIO; @@ -89,7 +87,6 @@ public WorkerTaskMonitorTest() { TestUtils testUtils = new TestUtils(); jsonMapper = testUtils.getTestObjectMapper(); - indexMerger = testUtils.getTestIndexMerger(); indexMergerV9 = testUtils.getTestIndexMergerV9(); indexIO = testUtils.getTestIndexIO(); } @@ -184,7 +181,6 @@ public List getLocations() ) ), jsonMapper, - indexMerger, indexIO, null, null, diff --git a/processing/src/main/java/io/druid/segment/IndexIO.java b/processing/src/main/java/io/druid/segment/IndexIO.java index c1b9d65f3055..0ea368d945a0 100644 --- a/processing/src/main/java/io/druid/segment/IndexIO.java +++ b/processing/src/main/java/io/druid/segment/IndexIO.java @@ -23,33 +23,22 @@ import com.fasterxml.jackson.databind.JsonMappingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import com.google.common.io.ByteStreams; import com.google.common.io.Closeables; import com.google.common.io.Files; import com.google.common.primitives.Ints; import com.google.inject.Inject; import com.metamx.emitter.EmittingLogger; -import io.druid.collections.bitmap.BitmapFactory; import io.druid.collections.bitmap.ConciseBitmapFactory; import io.druid.collections.bitmap.ImmutableBitmap; -import io.druid.collections.bitmap.MutableBitmap; import io.druid.collections.spatial.ImmutableRTree; import io.druid.common.utils.SerializerUtils; -import io.druid.java.util.common.io.Closer; -import io.druid.io.ZeroCopyByteArrayOutputStream; -import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; -import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.java.util.common.io.smoosh.Smoosh; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; -import io.druid.java.util.common.io.smoosh.SmooshedWriter; import io.druid.java.util.common.logger.Logger; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnBuilder; @@ -62,8 +51,6 @@ import io.druid.segment.data.BitmapSerdeFactory; import io.druid.segment.data.ByteBufferSerializer; import io.druid.segment.data.CompressedLongsIndexedSupplier; -import io.druid.segment.data.CompressedObjectStrategy; -import io.druid.segment.data.CompressedVSizeIntsIndexedSupplier; import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; @@ -71,33 +58,22 @@ import io.druid.segment.data.IndexedMultivalue; import io.druid.segment.data.IndexedRTree; import io.druid.segment.data.VSizeIndexed; -import io.druid.segment.data.VSizeIndexedInts; import io.druid.segment.serde.BitmapIndexColumnPartSupplier; -import io.druid.segment.serde.ComplexColumnPartSerde; import io.druid.segment.serde.ComplexColumnPartSupplier; -import io.druid.segment.serde.DictionaryEncodedColumnPartSerde; import io.druid.segment.serde.DictionaryEncodedColumnSupplier; -import io.druid.segment.serde.FloatGenericColumnPartSerde; import io.druid.segment.serde.FloatGenericColumnSupplier; -import io.druid.segment.serde.LongGenericColumnPartSerde; import io.druid.segment.serde.LongGenericColumnSupplier; import io.druid.segment.serde.SpatialIndexColumnPartSupplier; -import org.apache.commons.io.FileUtils; import org.joda.time.Interval; -import java.io.ByteArrayOutputStream; import java.io.File; import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.util.AbstractList; import java.util.Arrays; -import java.util.Collections; import java.util.Iterator; -import java.util.LinkedHashSet; -import java.util.List; import java.util.Map; import java.util.Set; @@ -115,26 +91,19 @@ public class IndexIO private static final SerializerUtils serializerUtils = new SerializerUtils(); private final ObjectMapper mapper; - private final DefaultIndexIOHandler defaultIndexIOHandler; @Inject public IndexIO(ObjectMapper mapper, ColumnConfig columnConfig) { this.mapper = Preconditions.checkNotNull(mapper, "null ObjectMapper"); Preconditions.checkNotNull(columnConfig, "null ColumnConfig"); - defaultIndexIOHandler = new DefaultIndexIOHandler(mapper); - indexLoaders = ImmutableMap.builder() - .put(0, new LegacyIndexLoader(defaultIndexIOHandler, columnConfig)) - .put(1, new LegacyIndexLoader(defaultIndexIOHandler, columnConfig)) - .put(2, new LegacyIndexLoader(defaultIndexIOHandler, columnConfig)) - .put(3, new LegacyIndexLoader(defaultIndexIOHandler, columnConfig)) - .put(4, new LegacyIndexLoader(defaultIndexIOHandler, columnConfig)) - .put(5, new LegacyIndexLoader(defaultIndexIOHandler, columnConfig)) - .put(6, new LegacyIndexLoader(defaultIndexIOHandler, columnConfig)) - .put(7, new LegacyIndexLoader(defaultIndexIOHandler, columnConfig)) - .put(8, new LegacyIndexLoader(defaultIndexIOHandler, columnConfig)) - .put(9, new V9IndexLoader(columnConfig)) - .build(); + ImmutableMap.Builder indexLoadersBuilder = ImmutableMap.builder(); + LegacyIndexLoader legacyIndexLoader = new LegacyIndexLoader(new DefaultIndexIOHandler(), columnConfig); + for (int i = 0; i <= V8_VERSION; i++) { + indexLoadersBuilder.put(i, legacyIndexLoader); + } + indexLoadersBuilder.put((int) V9_VERSION, new V9IndexLoader(columnConfig)); + indexLoaders = indexLoadersBuilder.build(); } public void validateTwoSegments(File dir1, File dir2) throws IOException @@ -246,11 +215,6 @@ public static void checkFileSize(File indexFile) throws IOException } } - public boolean convertSegment(File toConvert, File converted, IndexSpec indexSpec) throws IOException - { - return convertSegment(toConvert, converted, indexSpec, false, true); - } - public boolean convertSegment( File toConvert, File converted, @@ -260,50 +224,19 @@ public boolean convertSegment( ) throws IOException { final int version = SegmentUtils.getVersionFromDir(toConvert); - switch (version) { - case 1: - case 2: - case 3: - log.makeAlert("Attempt to load segment of version <= 3.") - .addData("version", version) - .emit(); - return false; - case 4: - case 5: - case 6: - case 7: - log.info("Old version, re-persisting."); - try (QueryableIndex segmentToConvert = loadIndex(toConvert)) { - new IndexMerger(mapper, this).append( - Arrays.asList(new QueryableIndexIndexableAdapter(segmentToConvert)), - null, - converted, - indexSpec - ); - } - return true; - case 8: - defaultIndexIOHandler.convertV8toV9(toConvert, converted, indexSpec); - return true; - default: - if (forceIfCurrent) { - new IndexMerger(mapper, this).convert(toConvert, converted, indexSpec); - if (validate) { - validateTwoSegments(toConvert, converted); - } - return true; - } else { - log.info("Version[%s], skipping.", version); - return false; - } + boolean current = version == CURRENT_VERSION_ID; + if (!current || forceIfCurrent) { + new IndexMergerV9(mapper, this).convert(toConvert, converted, indexSpec); + if (validate) { + validateTwoSegments(toConvert, converted); + } + return true; + } else { + log.info("Current version[%d], skipping.", version); + return false; } } - public DefaultIndexIOHandler getDefaultIndexIOHandler() - { - return defaultIndexIOHandler; - } - static interface IndexIOHandler { public MMappedIndex mapDir(File inDir) throws IOException; @@ -366,12 +299,6 @@ public static void validateRowValues( public static class DefaultIndexIOHandler implements IndexIOHandler { private static final Logger log = new Logger(DefaultIndexIOHandler.class); - private final ObjectMapper mapper; - - public DefaultIndexIOHandler(ObjectMapper mapper) - { - this.mapper = mapper; - } @Override public MMappedIndex mapDir(File inDir) throws IOException @@ -481,362 +408,6 @@ public MMappedIndex mapDir(File inDir) throws IOException return retVal; } - public void convertV8toV9(File v8Dir, File v9Dir, IndexSpec indexSpec) - throws IOException - { - log.info("Converting v8[%s] to v9[%s]", v8Dir, v9Dir); - - InputStream indexIn = null; - try { - indexIn = new FileInputStream(new File(v8Dir, "index.drd")); - byte theVersion = (byte) indexIn.read(); - if (theVersion != V8_VERSION) { - throw new IAE("Unknown version[%s]", theVersion); - } - } - finally { - Closeables.close(indexIn, false); - } - - Closer closer = Closer.create(); - try { - SmooshedFileMapper v8SmooshedFiles = closer.register(Smoosh.map(v8Dir)); - - FileUtils.forceMkdir(v9Dir); - final FileSmoosher v9Smoosher = closer.register(new FileSmoosher(v9Dir)); - - ByteStreams.write(Ints.toByteArray(9), Files.newOutputStreamSupplier(new File(v9Dir, "version.bin"))); - - Map> bitmapIndexes = Maps.newHashMap(); - final ByteBuffer invertedBuffer = v8SmooshedFiles.mapFile("inverted.drd"); - BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory(); - - while (invertedBuffer.hasRemaining()) { - final String dimName = serializerUtils.readString(invertedBuffer); - bitmapIndexes.put( - dimName, - GenericIndexed.read(invertedBuffer, bitmapSerdeFactory.getObjectStrategy(), v8SmooshedFiles) - ); - } - - Map spatialIndexes = Maps.newHashMap(); - final ByteBuffer spatialBuffer = v8SmooshedFiles.mapFile("spatial.drd"); - while (spatialBuffer != null && spatialBuffer.hasRemaining()) { - spatialIndexes.put( - serializerUtils.readString(spatialBuffer), - ByteBufferSerializer.read( - spatialBuffer, new IndexedRTree.ImmutableRTreeObjectStrategy( - bitmapSerdeFactory.getBitmapFactory() - ) - ) - ); - } - - final LinkedHashSet skippedFiles = Sets.newLinkedHashSet(); - final Set skippedDimensions = Sets.newLinkedHashSet(); - for (String filename : v8SmooshedFiles.getInternalFilenames()) { - log.info("Processing file[%s]", filename); - if (filename.startsWith("dim_")) { - final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); - builder.setValueType(ValueType.STRING); - - final List outParts = Lists.newArrayList(); - - ByteBuffer dimBuffer = v8SmooshedFiles.mapFile(filename); - String dimension = serializerUtils.readString(dimBuffer); - if (!filename.equals(String.format("dim_%s.drd", dimension))) { - throw new ISE("loaded dimension[%s] from file[%s]", dimension, filename); - } - - ByteArrayOutputStream nameBAOS = new ByteArrayOutputStream(); - serializerUtils.writeString(nameBAOS, dimension); - outParts.add(ByteBuffer.wrap(nameBAOS.toByteArray())); - - GenericIndexed dictionary = GenericIndexed.read( - dimBuffer, GenericIndexed.STRING_STRATEGY - ); - - if (dictionary.size() == 0) { - log.info("Dimension[%s] had cardinality 0, equivalent to no column, so skipping.", dimension); - skippedDimensions.add(dimension); - continue; - } - - int emptyStrIdx = dictionary.indexOf(""); - List singleValCol = null; - VSizeIndexed multiValCol = VSizeIndexed.readFromByteBuffer(dimBuffer.asReadOnlyBuffer()); - GenericIndexed bitmaps = bitmapIndexes.get(dimension); - ImmutableRTree spatialIndex = spatialIndexes.get(dimension); - - final BitmapFactory bitmapFactory = bitmapSerdeFactory.getBitmapFactory(); - boolean onlyOneValue = true; - MutableBitmap nullsSet = null; - for (int i = 0; i < multiValCol.size(); ++i) { - VSizeIndexedInts rowValue = multiValCol.get(i); - if (!onlyOneValue) { - break; - } - if (rowValue.size() > 1) { - onlyOneValue = false; - } - if (rowValue.size() == 0 || rowValue.get(0) == emptyStrIdx) { - if (nullsSet == null) { - nullsSet = bitmapFactory.makeEmptyMutableBitmap(); - } - nullsSet.add(i); - } - } - - if (onlyOneValue) { - log.info("Dimension[%s] is single value, converting...", dimension); - final boolean bumpedDictionary; - if (nullsSet != null) { - log.info("Dimension[%s] has null rows.", dimension); - final ImmutableBitmap theNullSet = bitmapFactory.makeImmutableBitmap(nullsSet); - - if (dictionary.get(0) != null) { - log.info("Dimension[%s] has no null value in the dictionary, expanding...", dimension); - bumpedDictionary = true; - final List nullList = Lists.newArrayList(); - nullList.add(null); - - dictionary = GenericIndexed.fromIterable( - Iterables.concat(nullList, dictionary), - GenericIndexed.STRING_STRATEGY - ); - - bitmaps = GenericIndexed.fromIterable( - Iterables.concat(Collections.singletonList(theNullSet), bitmaps), - bitmapSerdeFactory.getObjectStrategy() - ); - } else { - bumpedDictionary = false; - bitmaps = GenericIndexed.fromIterable( - Iterables.concat( - Collections.singletonList( - bitmapFactory - .union(Arrays.asList(theNullSet, bitmaps.get(0))) - ), - Iterables.skip(bitmaps, 1) - ), - bitmapSerdeFactory.getObjectStrategy() - ); - } - } else { - bumpedDictionary = false; - } - - final VSizeIndexed finalMultiValCol = multiValCol; - singleValCol = new AbstractList() - { - @Override - public Integer get(int index) - { - final VSizeIndexedInts ints = finalMultiValCol.get(index); - return ints.size() == 0 ? 0 : ints.get(0) + (bumpedDictionary ? 1 : 0); - } - - @Override - public int size() - { - return finalMultiValCol.size(); - } - }; - - multiValCol = null; - } else { - builder.setHasMultipleValues(true); - } - - final CompressedObjectStrategy.CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression(); - - final DictionaryEncodedColumnPartSerde.LegacySerializerBuilder columnPartBuilder = DictionaryEncodedColumnPartSerde - .legacySerializerBuilder() - .withDictionary(dictionary) - .withBitmapSerdeFactory(bitmapSerdeFactory) - .withBitmaps(bitmaps) - .withSpatialIndex(spatialIndex) - .withByteOrder(BYTE_ORDER); - - if (singleValCol != null) { - if (compressionStrategy != CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED) { - columnPartBuilder.withSingleValuedColumn( - CompressedVSizeIntsIndexedSupplier.fromList( - singleValCol, - dictionary.size(), - CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(dictionary.size()), - BYTE_ORDER, - compressionStrategy - ) - ); - } else { - columnPartBuilder.withSingleValuedColumn(VSizeIndexedInts.fromList(singleValCol, dictionary.size())); - } - } else if (compressionStrategy != CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED) { - columnPartBuilder.withMultiValuedColumn( - CompressedVSizeIndexedSupplier.fromIterable( - multiValCol, - dictionary.size(), - BYTE_ORDER, - compressionStrategy - ) - ); - } else { - columnPartBuilder.withMultiValuedColumn(multiValCol); - } - - final ColumnDescriptor serdeficator = builder - .addSerde(columnPartBuilder.build()) - .build(); - makeColumn(v9Smoosher, dimension, serdeficator); - } else if (filename.startsWith("met_") || filename.startsWith("numeric_dim_")) { - // NOTE: identifying numeric dimensions by using a different filename pattern is meant to allow the - // legacy merger (which will be deprecated) to support long/float dims. Going forward, the V9 merger - // should be used instead if any dimension types beyond String are needed. - if (!filename.endsWith(String.format("%s.drd", BYTE_ORDER))) { - skippedFiles.add(filename); - continue; - } - - MetricHolder holder = MetricHolder.fromByteBuffer(v8SmooshedFiles.mapFile(filename), v8SmooshedFiles); - final String metric = holder.getName(); - - final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); - - switch (holder.getType()) { - case LONG: - builder.setValueType(ValueType.LONG); - builder.addSerde( - LongGenericColumnPartSerde.legacySerializerBuilder() - .withByteOrder(BYTE_ORDER) - .withDelegate(holder.longType) - .build() - ); - break; - case FLOAT: - builder.setValueType(ValueType.FLOAT); - builder.addSerde( - FloatGenericColumnPartSerde.legacySerializerBuilder() - .withByteOrder(BYTE_ORDER) - .withDelegate(holder.floatType) - .build() - ); - break; - case COMPLEX: - if (!(holder.complexType instanceof GenericIndexed)) { - throw new ISE("Serialized complex types must be GenericIndexed objects."); - } - final GenericIndexed column = (GenericIndexed) holder.complexType; - final String complexType = holder.getTypeName(); - builder.setValueType(ValueType.COMPLEX); - builder.addSerde( - ComplexColumnPartSerde.legacySerializerBuilder() - .withTypeName(complexType) - .withDelegate(column).build() - ); - break; - default: - throw new ISE("Unknown type[%s]", holder.getType()); - } - - final ColumnDescriptor serdeficator = builder.build(); - makeColumn(v9Smoosher, metric, serdeficator); - } else if (String.format("time_%s.drd", BYTE_ORDER).equals(filename)) { - CompressedLongsIndexedSupplier timestamps = CompressedLongsIndexedSupplier.fromByteBuffer( - v8SmooshedFiles.mapFile(filename), - BYTE_ORDER, - v8SmooshedFiles - ); - - final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); - builder.setValueType(ValueType.LONG); - builder.addSerde( - LongGenericColumnPartSerde.legacySerializerBuilder() - .withByteOrder(BYTE_ORDER) - .withDelegate(timestamps) - .build() - ); - final ColumnDescriptor serdeficator = builder.build(); - makeColumn(v9Smoosher, "__time", serdeficator); - } else { - skippedFiles.add(filename); - } - } - - final ByteBuffer indexBuffer = v8SmooshedFiles.mapFile("index.drd"); - - indexBuffer.get(); // Skip the version byte - final GenericIndexed dims8 = GenericIndexed.read( - indexBuffer, GenericIndexed.STRING_STRATEGY - ); - final GenericIndexed dims9 = GenericIndexed.fromIterable( - Iterables.filter( - dims8, new Predicate() - { - @Override - public boolean apply(String s) - { - return !skippedDimensions.contains(s); - } - } - ), - GenericIndexed.STRING_STRATEGY - ); - final GenericIndexed availableMetrics = GenericIndexed.read( - indexBuffer, GenericIndexed.STRING_STRATEGY - ); - final Interval dataInterval = new Interval(serializerUtils.readString(indexBuffer)); - final BitmapSerdeFactory segmentBitmapSerdeFactory = mapper.readValue( - serializerUtils.readString(indexBuffer), - BitmapSerdeFactory.class - ); - - Set columns = Sets.newTreeSet(); - columns.addAll(Lists.newArrayList(dims9)); - columns.addAll(Lists.newArrayList(availableMetrics)); - GenericIndexed cols = GenericIndexed.fromIterable(columns, GenericIndexed.STRING_STRATEGY); - - final String segmentBitmapSerdeFactoryString = mapper.writeValueAsString(segmentBitmapSerdeFactory); - - final long numBytes = cols.getSerializedSize() + dims9.getSerializedSize() + 16 - + serializerUtils.getSerializedStringByteSize(segmentBitmapSerdeFactoryString); - final SmooshedWriter writer = v9Smoosher.addWithSmooshedWriter("index.drd", numBytes); - cols.writeToChannel(writer); - dims9.writeToChannel(writer); - serializerUtils.writeLong(writer, dataInterval.getStartMillis()); - serializerUtils.writeLong(writer, dataInterval.getEndMillis()); - serializerUtils.writeString(writer, segmentBitmapSerdeFactoryString); - writer.close(); - - final ByteBuffer metadataBuffer = v8SmooshedFiles.mapFile("metadata.drd"); - if (metadataBuffer != null) { - v9Smoosher.add("metadata.drd", metadataBuffer); - } - - log.info("Skipped files[%s]", skippedFiles); - - } - catch (Throwable t) { - throw closer.rethrow(t); - } - finally { - closer.close(); - } - } - - private void makeColumn(FileSmoosher v9Smoosher, String dimension, ColumnDescriptor serdeficator) - throws IOException - { - ZeroCopyByteArrayOutputStream specBytes = new ZeroCopyByteArrayOutputStream(); - serializerUtils.writeString(specBytes, mapper.writeValueAsString(serdeficator)); - - try (SmooshedWriter channel = v9Smoosher.addWithSmooshedWriter( - dimension, serdeficator.numBytes() + specBytes.size() - )) { - specBytes.writeTo(channel); - serdeficator.write(channel, v9Smoosher); - } - } } static interface IndexLoader diff --git a/processing/src/main/java/io/druid/segment/IndexMerger.java b/processing/src/main/java/io/druid/segment/IndexMerger.java index 3251232a74b6..88b33b381be6 100644 --- a/processing/src/main/java/io/druid/segment/IndexMerger.java +++ b/processing/src/main/java/io/druid/segment/IndexMerger.java @@ -19,261 +19,78 @@ package io.druid.segment; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; -import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Ordering; import com.google.common.collect.PeekingIterator; import com.google.common.collect.Sets; -import com.google.common.io.ByteSink; -import com.google.common.io.FileWriteMode; -import com.google.common.io.Files; -import com.google.common.io.OutputSupplier; import com.google.common.primitives.Ints; -import com.google.common.primitives.Longs; -import com.google.inject.Inject; -import io.druid.collections.CombiningIterable; -import io.druid.common.guava.FileOutputSupplier; -import io.druid.common.guava.GuavaUtils; -import io.druid.common.utils.JodaUtils; +import com.google.inject.ImplementedBy; import io.druid.common.utils.SerializerUtils; -import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Pair; import io.druid.java.util.common.guava.Comparators; -import io.druid.java.util.common.guava.FunctionalIterable; -import io.druid.java.util.common.guava.MergeIterable; import io.druid.java.util.common.guava.nary.BinaryFn; -import io.druid.java.util.common.io.Closer; -import io.druid.java.util.common.io.smoosh.Smoosh; import io.druid.java.util.common.logger.Logger; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnCapabilitiesImpl; -import io.druid.segment.column.ValueType; -import io.druid.segment.data.BitmapSerdeFactory; -import io.druid.segment.data.CompressedObjectStrategy; -import io.druid.segment.data.CompressionFactory; -import io.druid.segment.data.GenericIndexed; -import io.druid.segment.data.IOPeon; import io.druid.segment.data.Indexed; -import io.druid.segment.data.LongSupplierSerializer; -import io.druid.segment.data.TmpFileIOPeon; import io.druid.segment.incremental.IncrementalIndex; -import io.druid.segment.incremental.IncrementalIndexAdapter; -import io.druid.segment.serde.ComplexMetricColumnSerializer; -import io.druid.segment.serde.ComplexMetricSerde; -import io.druid.segment.serde.ComplexMetrics; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import it.unimi.dsi.fastutil.ints.IntIterator; import it.unimi.dsi.fastutil.ints.IntSortedSet; -import org.apache.commons.io.FileUtils; -import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; -import java.io.Closeable; import java.io.File; -import java.io.FileOutputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.IntBuffer; -import java.nio.channels.FileChannel; import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.NoSuchElementException; import java.util.PriorityQueue; import java.util.Set; +import java.util.stream.Collectors; -/** - */ -public class IndexMerger +@ImplementedBy(IndexMergerV9.class) +public interface IndexMerger { - private static final Logger log = new Logger(IndexMerger.class); - - protected static final SerializerUtils serializerUtils = new SerializerUtils(); - protected static final int INVALID_ROW = -1; - - protected final ObjectMapper mapper; - protected final IndexIO indexIO; + Logger log = new Logger(IndexMerger.class); - @Inject - public IndexMerger( - ObjectMapper mapper, - IndexIO indexIO - ) - { - this.mapper = Preconditions.checkNotNull(mapper, "null ObjectMapper"); - this.indexIO = Preconditions.checkNotNull(indexIO, "null IndexIO"); - - } + SerializerUtils serializerUtils = new SerializerUtils(); + int INVALID_ROW = -1; - public File persist( - final IncrementalIndex index, - File outDir, - IndexSpec indexSpec - ) throws IOException + static List getMergedDimensionsFromQueryableIndexes(List indexes) { - return persist(index, index.getInterval(), outDir, indexSpec); + return getMergedDimensions(toIndexableAdapters(indexes)); } - /** - * This is *not* thread-safe and havok will ensue if this is called and writes are still occurring - * on the IncrementalIndex object. - * - * @param index the IncrementalIndex to persist - * @param dataInterval the Interval that the data represents - * @param outDir the directory to persist the data to - * - * @return the index output directory - * - * @throws java.io.IOException if an IO error occurs persisting the index - */ - public File persist( - final IncrementalIndex index, - final Interval dataInterval, - File outDir, - IndexSpec indexSpec - ) throws IOException + static List toIndexableAdapters(List indexes) { - return persist(index, dataInterval, outDir, indexSpec, new BaseProgressIndicator()); + return indexes.stream().map(QueryableIndexIndexableAdapter::new).collect(Collectors.toList()); } - public File persist( - final IncrementalIndex index, - final Interval dataInterval, - File outDir, - IndexSpec indexSpec, - ProgressIndicator progress - ) throws IOException + static List getMergedDimensions(List indexes) { - if (index.isEmpty()) { - throw new IAE("Trying to persist an empty index!"); + if (indexes.size() == 0) { + return ImmutableList.of(); } - - final long firstTimestamp = index.getMinTime().getMillis(); - final long lastTimestamp = index.getMaxTime().getMillis(); - if (!(dataInterval.contains(firstTimestamp) && dataInterval.contains(lastTimestamp))) { - throw new IAE( - "interval[%s] does not encapsulate the full range of timestamps[%s, %s]", - dataInterval, - new DateTime(firstTimestamp), - new DateTime(lastTimestamp) - ); + List commonDimOrder = getLongestSharedDimOrder(indexes); + if (commonDimOrder == null) { + log.warn("Indexes have incompatible dimension orders, using lexicographic order."); + return getLexicographicMergedDimensions(indexes); + } else { + return commonDimOrder; } - - FileUtils.forceMkdir(outDir); - - log.info("Starting persist for interval[%s], rows[%,d]", dataInterval, index.size()); - return merge( - Arrays.asList( - new IncrementalIndexAdapter( - dataInterval, - index, - indexSpec.getBitmapSerdeFactory().getBitmapFactory() - ) - ), - // if index is not rolled up, then it should be not rollup here - // if index is rolled up, then it is no need to rollup again. - // In this case, true/false won't cause reOrdering in merge stage - // while merging a single iterable - false, - index.getMetricAggs(), - outDir, - indexSpec, - progress - ); - } - - public File mergeQueryableIndex( - List indexes, - boolean rollup, - final AggregatorFactory[] metricAggs, - File outDir, - IndexSpec indexSpec - ) throws IOException - { - return mergeQueryableIndex(indexes, rollup, metricAggs, outDir, indexSpec, new BaseProgressIndicator()); - } - - public File mergeQueryableIndex( - List indexes, - boolean rollup, - final AggregatorFactory[] metricAggs, - File outDir, - IndexSpec indexSpec, - ProgressIndicator progress - ) throws IOException - { - return merge( - toIndexableAdapters(indexes), - rollup, - metricAggs, - outDir, - indexSpec, - progress - ); - } - - public File merge( - List indexes, - boolean rollup, - final AggregatorFactory[] metricAggs, - File outDir, - IndexSpec indexSpec - ) throws IOException - { - return merge(indexes, rollup, metricAggs, outDir, indexSpec, new BaseProgressIndicator()); - } - - private static List getLexicographicMergedDimensions(List indexes) - { - return mergeIndexed( - Lists.transform( - indexes, - new Function>() - { - @Override - public Iterable apply(@Nullable IndexableAdapter input) - { - return input.getDimensionNames(); - } - } - ) - ); - } - - private static List toIndexableAdapters(List indexes) - { - // We are materializing the list for performance reasons. Lists.transform - // only creates a "view" of the original list, meaning the function gets - // applied every time you access an element. - return Lists.newArrayList( - Iterables.transform( - indexes, - new Function() - { - @Override - public IndexableAdapter apply(final QueryableIndex input) - { - return new QueryableIndexIndexableAdapter(input); - } - } - ) - ); } - private static List getLongestSharedDimOrder(List indexes) + static List getLongestSharedDimOrder(List indexes) { int maxSize = 0; Iterable orderingCandidate = null; @@ -308,196 +125,9 @@ private static List getLongestSharedDimOrder(List inde return ImmutableList.copyOf(orderingCandidate); } - public static List getMergedDimensionsFromQueryableIndexes(List indexes) + static List getLexicographicMergedDimensions(List indexes) { - return getMergedDimensions(toIndexableAdapters(indexes)); - } - - public static List getMergedDimensions(List indexes) - { - if (indexes.size() == 0) { - return ImmutableList.of(); - } - List commonDimOrder = getLongestSharedDimOrder(indexes); - if (commonDimOrder == null) { - log.warn("Indexes have incompatible dimension orders, using lexicographic order."); - return getLexicographicMergedDimensions(indexes); - } else { - return commonDimOrder; - } - } - - public File merge( - List indexes, - final boolean rollup, - final AggregatorFactory[] metricAggs, - File outDir, - IndexSpec indexSpec, - ProgressIndicator progress - ) throws IOException - { - FileUtils.deleteDirectory(outDir); - FileUtils.forceMkdir(outDir); - - final List mergedDimensions = getMergedDimensions(indexes); - - final List mergedMetrics = Lists.transform( - mergeIndexed( - Lists.newArrayList( - FunctionalIterable - .create(indexes) - .transform( - new Function>() - { - @Override - public Iterable apply(@Nullable IndexableAdapter input) - { - return input.getMetricNames(); - } - } - ) - ) - ), - new Function() - { - @Override - public String apply(@Nullable String input) - { - return input; - } - } - ); - - final AggregatorFactory[] sortedMetricAggs = new AggregatorFactory[mergedMetrics.size()]; - for (int i = 0; i < metricAggs.length; i++) { - AggregatorFactory metricAgg = metricAggs[i]; - int metricIndex = mergedMetrics.indexOf(metricAgg.getName()); - /* - If metricIndex is negative, one of the metricAggs was not present in the union of metrics from the indices - we are merging - */ - if (metricIndex > -1) { - sortedMetricAggs[metricIndex] = metricAgg; - } - } - - /* - If there is nothing at sortedMetricAggs[i], then we did not have a metricAgg whose name matched the name - of the ith element of mergedMetrics. I.e. There was a metric in the indices to merge that we did not ask for. - */ - for (int i = 0; i < sortedMetricAggs.length; i++) { - if (sortedMetricAggs[i] == null) { - throw new IAE("Indices to merge contained metric[%s], but requested metrics did not", mergedMetrics.get(i)); - } - } - - for (int i = 0; i < mergedMetrics.size(); i++) { - if (!sortedMetricAggs[i].getName().equals(mergedMetrics.get(i))) { - throw new IAE( - "Metric mismatch, index[%d] [%s] != [%s]", - i, - sortedMetricAggs[i].getName(), - mergedMetrics.get(i) - ); - } - } - - Function>, Iterable> rowMergerFn = new Function>, Iterable>() - { - @Override - public Iterable apply( - @Nullable ArrayList> boats - ) - { - if (rollup) { - return CombiningIterable.create( - new MergeIterable<>(Comparators.naturalNullsFirst(), boats), - Comparators.naturalNullsFirst(), - new RowboatMergeFunction(sortedMetricAggs) - ); - } else { - return new MergeIterable( - new Ordering() - { - @Override - public int compare(Rowboat left, Rowboat right) - { - return Longs.compare(left.getTimestamp(), right.getTimestamp()); - } - }.nullsFirst(), - boats - ); - } - } - }; - - return makeIndexFiles( - indexes, - sortedMetricAggs, - outDir, - progress, - mergedDimensions, - mergedMetrics, - rowMergerFn, - indexSpec - ); - } - - // Faster than IndexMaker - public File convert(final File inDir, final File outDir, final IndexSpec indexSpec) throws IOException - { - return convert(inDir, outDir, indexSpec, new BaseProgressIndicator()); - } - - public File convert( - final File inDir, final File outDir, final IndexSpec indexSpec, final ProgressIndicator progress - ) throws IOException - { - try (QueryableIndex index = indexIO.loadIndex(inDir)) { - final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(index); - return makeIndexFiles( - ImmutableList.of(adapter), - null, - outDir, - progress, - Lists.newArrayList(adapter.getDimensionNames()), - Lists.newArrayList(adapter.getMetricNames()), - new Function>, Iterable>() - { - @Nullable - @Override - public Iterable apply(ArrayList> input) - { - return input.get(0); - } - }, - indexSpec - ); - } - } - - - public File append( - List indexes, AggregatorFactory[] aggregators, File outDir, IndexSpec indexSpec - ) throws IOException - { - return append(indexes, aggregators, outDir, indexSpec, new BaseProgressIndicator()); - } - - public File append( - List indexes, - AggregatorFactory[] aggregators, - File outDir, - IndexSpec indexSpec, - ProgressIndicator progress - ) throws IOException - { - FileUtils.deleteDirectory(outDir); - FileUtils.forceMkdir(outDir); - - final List mergedDimensions = getMergedDimensions(indexes); - - final List mergedMetrics = mergeIndexed( + return mergeIndexed( Lists.transform( indexes, new Function>() @@ -505,555 +135,102 @@ public File append( @Override public Iterable apply(@Nullable IndexableAdapter input) { - return Iterables.transform( - input.getMetricNames(), - new Function() - { - @Override - public String apply(@Nullable String input) - { - return input; - } - } - ); + return input.getDimensionNames(); } } ) ); - - Function>, Iterable> rowMergerFn = new Function>, Iterable>() - { - @Override - public Iterable apply( - @Nullable final ArrayList> boats - ) - { - return new MergeIterable<>(Comparators.naturalNullsFirst(), boats); - } - }; - - return makeIndexFiles( - indexes, - aggregators, - outDir, - progress, - mergedDimensions, - mergedMetrics, - rowMergerFn, - indexSpec - ); } - protected File makeIndexFiles( - final List indexes, - final AggregatorFactory[] metricAggs, - final File outDir, - final ProgressIndicator progress, - final List mergedDimensions, - final List mergedMetrics, - final Function>, Iterable> rowMergerFn, - final IndexSpec indexSpec - ) throws IOException + static > ArrayList mergeIndexed(List> indexedLists) { - List metadataList = Lists.transform( - indexes, - new Function() - { - @Nullable - @Override - public Metadata apply(IndexableAdapter input) - { - return input.getMetadata(); - } - } - ); - - Metadata segmentMetadata = null; - if (metricAggs != null) { - AggregatorFactory[] combiningMetricAggs = new AggregatorFactory[metricAggs.length]; - for (int i = 0; i < metricAggs.length; i++) { - combiningMetricAggs[i] = metricAggs[i].getCombiningFactory(); - } - segmentMetadata = Metadata.merge( - metadataList, - combiningMetricAggs - ); - } else { - segmentMetadata = Metadata.merge( - metadataList, - null - ); - } - - final Map valueTypes = Maps.newTreeMap(Comparators.naturalNullsFirst()); - final Map metricTypeNames = Maps.newTreeMap(Comparators.naturalNullsFirst()); - final Map columnCapabilities = Maps.newHashMap(); - final List dimCapabilities = new ArrayList<>(); - - for (IndexableAdapter adapter : indexes) { - for (String dimension : adapter.getDimensionNames()) { - ColumnCapabilitiesImpl mergedCapabilities = columnCapabilities.get(dimension); - ColumnCapabilities capabilities = adapter.getCapabilities(dimension); - if (mergedCapabilities == null) { - mergedCapabilities = new ColumnCapabilitiesImpl(); - } - columnCapabilities.put(dimension, mergedCapabilities.merge(capabilities)); - } - for (String metric : adapter.getMetricNames()) { - ColumnCapabilitiesImpl mergedCapabilities = columnCapabilities.get(metric); - ColumnCapabilities capabilities = adapter.getCapabilities(metric); - if (mergedCapabilities == null) { - mergedCapabilities = new ColumnCapabilitiesImpl(); - } - columnCapabilities.put(metric, mergedCapabilities.merge(capabilities)); - - valueTypes.put(metric, capabilities.getType()); - metricTypeNames.put(metric, adapter.getMetricType(metric)); - } - } - - for (String dimension : mergedDimensions) { - dimCapabilities.add(columnCapabilities.get(dimension)); - } - - Closer closer = Closer.create(); - try { - final Interval dataInterval; - final File v8OutDir = new File(outDir, "v8-tmp"); - FileUtils.forceMkdir(v8OutDir); - registerDeleteDirectory(closer, v8OutDir); - File tmpPeonFilesDir = new File(v8OutDir, "tmpPeonFiles"); - FileUtils.forceMkdir(tmpPeonFilesDir); - registerDeleteDirectory(closer, tmpPeonFilesDir); - final IOPeon ioPeon = new TmpFileIOPeon(tmpPeonFilesDir, true); - closer.register(ioPeon); - /************* Main index.drd file **************/ - progress.progress(); - long startTime = System.currentTimeMillis(); - File indexFile = new File(v8OutDir, "index.drd"); - - try (FileOutputStream fileOutputStream = new FileOutputStream(indexFile); - FileChannel channel = fileOutputStream.getChannel()) { - channel.write(ByteBuffer.wrap(new byte[]{IndexIO.V8_VERSION})); - - GenericIndexed.fromIterable(mergedDimensions, GenericIndexed.STRING_STRATEGY).writeToChannel(channel); - GenericIndexed.fromIterable(mergedMetrics, GenericIndexed.STRING_STRATEGY).writeToChannel(channel); - - DateTime minTime = new DateTime(JodaUtils.MAX_INSTANT); - DateTime maxTime = new DateTime(JodaUtils.MIN_INSTANT); - - for (IndexableAdapter index : indexes) { - minTime = JodaUtils.minDateTime(minTime, index.getDataInterval().getStart()); - maxTime = JodaUtils.maxDateTime(maxTime, index.getDataInterval().getEnd()); - } - - dataInterval = new Interval(minTime, maxTime); - serializerUtils.writeString(channel, String.format("%s/%s", minTime, maxTime)); - serializerUtils.writeString(channel, mapper.writeValueAsString(indexSpec.getBitmapSerdeFactory())); - } - IndexIO.checkFileSize(indexFile); - log.info("outDir[%s] completed index.drd in %,d millis.", v8OutDir, System.currentTimeMillis() - startTime); - - /************* Setup Dim Conversions **************/ - progress.progress(); - startTime = System.currentTimeMillis(); - - final ArrayList dimOuts = Lists.newArrayListWithCapacity(mergedDimensions.size()); - final DimensionHandler[] handlers = makeDimensionHandlers(mergedDimensions, dimCapabilities); - final List mergers = new ArrayList<>(); - for (int i = 0; i < mergedDimensions.size(); i++) { - DimensionMergerLegacy merger = handlers[i].makeLegacyMerger( - indexSpec, - v8OutDir, - ioPeon, - dimCapabilities.get(i), - progress - ); - mergers.add(merger); - merger.writeMergedValueMetadata(indexes); - - FileOutputSupplier dimOut = new FileOutputSupplier(merger.makeDimFile(), true); - merger.writeValueMetadataToFile(dimOut); - dimOuts.add(dimOut); - } - log.info("outDir[%s] completed dim conversions in %,d millis.", v8OutDir, System.currentTimeMillis() - startTime); - - /************* Walk through data sets and merge them *************/ - progress.progress(); - startTime = System.currentTimeMillis(); - - Iterable theRows = makeRowIterable( - indexes, - mergedDimensions, - mergedMetrics, - rowMergerFn, - dimCapabilities, - handlers, - mergers - ); - - LongSupplierSerializer timeWriter = CompressionFactory.getLongSerializer( - ioPeon, "little_end_time", IndexIO.BYTE_ORDER, indexSpec.getLongEncoding(), - CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY - ); - - timeWriter.open(); - - ArrayList metWriters = Lists.newArrayListWithCapacity(mergedMetrics.size()); - final CompressedObjectStrategy.CompressionStrategy metCompression = indexSpec.getMetricCompression(); - final CompressionFactory.LongEncodingStrategy longEncoding = indexSpec.getLongEncoding(); - for (String metric : mergedMetrics) { - ValueType type = valueTypes.get(metric); - switch (type) { - case LONG: - metWriters.add(new LongMetricColumnSerializer(metric, v8OutDir, ioPeon, metCompression, longEncoding)); - break; - case FLOAT: - metWriters.add(new FloatMetricColumnSerializer(metric, v8OutDir, ioPeon, metCompression)); - break; - case COMPLEX: - final String typeName = metricTypeNames.get(metric); - ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName); - - if (serde == null) { - throw new ISE("Unknown type[%s]", typeName); - } - - metWriters.add(new ComplexMetricColumnSerializer(metric, v8OutDir, ioPeon, serde)); - break; - default: - throw new ISE("Unknown type[%s]", type); - } - } - - for (MetricColumnSerializer metWriter : metWriters) { - metWriter.open(); - } - - int rowCount = 0; - long time = System.currentTimeMillis(); - List rowNumConversions = Lists.newArrayListWithCapacity(indexes.size()); - for (IndexableAdapter index : indexes) { - int[] arr = new int[index.getNumRows()]; - Arrays.fill(arr, INVALID_ROW); - rowNumConversions.add(IntBuffer.wrap(arr)); - } - - for (Rowboat theRow : theRows) { - progress.progress(); - timeWriter.add(theRow.getTimestamp()); - - final Object[] metrics = theRow.getMetrics(); - for (int i = 0; i < metrics.length; ++i) { - metWriters.get(i).serialize(metrics[i]); - } - - Object[] dims = theRow.getDims(); - for (int i = 0; i < dims.length; ++i) { - mergers.get(i).processMergedRow(dims[i]); - } - - Iterator> rowsIterator = theRow.getComprisedRows().int2ObjectEntrySet().fastIterator(); - while (rowsIterator.hasNext()) { - Int2ObjectMap.Entry comprisedRow = rowsIterator.next(); - - final IntBuffer conversionBuffer = rowNumConversions.get(comprisedRow.getIntKey()); - - for (IntIterator setIterator = comprisedRow.getValue().iterator(); setIterator.hasNext(); /* NOP */) { - int rowNum = setIterator.nextInt(); - while (conversionBuffer.position() < rowNum) { - conversionBuffer.put(INVALID_ROW); - } - conversionBuffer.put(rowCount); - } - } - - if ((++rowCount % 500000) == 0) { - log.info( - "outDir[%s] walked 500,000/%,d rows in %,d millis.", v8OutDir, rowCount, System.currentTimeMillis() - time - ); - time = System.currentTimeMillis(); - } - } - - for (IntBuffer rowNumConversion : rowNumConversions) { - rowNumConversion.rewind(); - } - - final File timeFile = IndexIO.makeTimeFile(v8OutDir, IndexIO.BYTE_ORDER); - timeFile.delete(); - ByteSink out = Files.asByteSink(timeFile, FileWriteMode.APPEND); - timeWriter.closeAndConsolidate(out); - IndexIO.checkFileSize(timeFile); - - for (MetricColumnSerializer metWriter : metWriters) { - metWriter.close(); - } - - log.info( - "outDir[%s] completed walk through of %,d rows in %,d millis.", - v8OutDir, - rowCount, - System.currentTimeMillis() - startTime - ); - - /************ Create Inverted Indexes and Finalize Columns *************/ - startTime = System.currentTimeMillis(); - - final File invertedFile = new File(v8OutDir, "inverted.drd"); - Files.touch(invertedFile); - out = Files.asByteSink(invertedFile, FileWriteMode.APPEND); - - final File geoFile = new File(v8OutDir, "spatial.drd"); - Files.touch(geoFile); - OutputSupplier spatialOut = Files.newOutputStreamSupplier(geoFile, true); - - for (int i = 0; i < mergedDimensions.size(); i++) { - DimensionMergerLegacy legacyMerger = (DimensionMergerLegacy) mergers.get(i); - legacyMerger.writeIndexes(rowNumConversions, closer); - legacyMerger.writeIndexesToFiles(out, spatialOut); - legacyMerger.writeRowValuesToFile(dimOuts.get(i)); - } - log.info("outDir[%s] completed inverted.drd and wrote dimensions in %,d millis.", v8OutDir, System.currentTimeMillis() - startTime); - - final Function dimFilenameFunction = new Function() - { - @Override - public String apply(@Nullable String input) - { - String formatString; - if (columnCapabilities.get(input).isDictionaryEncoded()) { - formatString = "dim_%s.drd"; - } else { - formatString = String.format("numeric_dim_%%s_%s.drd", IndexIO.BYTE_ORDER); - } - return GuavaUtils.formatFunction(formatString).apply(input); - } - }; - - final ArrayList expectedFiles = Lists.newArrayList( - Iterables.concat( - Arrays.asList( - "index.drd", "inverted.drd", "spatial.drd", String.format("time_%s.drd", IndexIO.BYTE_ORDER) - ), - Iterables.transform(mergedDimensions, dimFilenameFunction), - Iterables.transform( - mergedMetrics, GuavaUtils.formatFunction(String.format("met_%%s_%s.drd", IndexIO.BYTE_ORDER)) - ) - ) - ); - - if (segmentMetadata != null) { - writeMetadataToFile(new File(v8OutDir, "metadata.drd"), segmentMetadata); - log.info("wrote metadata.drd in outDir[%s].", v8OutDir); - - expectedFiles.add("metadata.drd"); - } - - Map files = Maps.newLinkedHashMap(); - for (String fileName : expectedFiles) { - files.put(fileName, new File(v8OutDir, fileName)); - } - - File smooshDir = new File(v8OutDir, "smoosher"); - FileUtils.forceMkdir(smooshDir); - - for (Map.Entry entry : Smoosh.smoosh(v8OutDir, smooshDir, files).entrySet()) { - entry.getValue().delete(); - } - - for (File file : smooshDir.listFiles()) { - Files.move(file, new File(v8OutDir, file.getName())); - } + Set retVal = Sets.newTreeSet(Comparators.naturalNullsFirst()); - if (!smooshDir.delete()) { - log.info("Unable to delete temporary dir[%s], contains[%s]", smooshDir, Arrays.asList(smooshDir.listFiles())); - throw new IOException(String.format("Unable to delete temporary dir[%s]", smooshDir)); + for (Iterable indexedList : indexedLists) { + for (T val : indexedList) { + retVal.add(val); } - - createIndexDrdFile( - IndexIO.V8_VERSION, - v8OutDir, - GenericIndexed.fromIterable(mergedDimensions, GenericIndexed.STRING_STRATEGY), - GenericIndexed.fromIterable(mergedMetrics, GenericIndexed.STRING_STRATEGY), - dataInterval, - indexSpec.getBitmapSerdeFactory() - ); - - indexIO.getDefaultIndexIOHandler().convertV8toV9(v8OutDir, outDir, indexSpec); - return outDir; } - catch (Throwable t) { - throw closer.rethrow(t); - } - finally { - closer.close(); - } - } - static void registerDeleteDirectory(Closer closer, final File dir) - { - closer.register(new Closeable() - { - @Override - public void close() throws IOException - { - FileUtils.deleteDirectory(dir); - } - }); + return Lists.newArrayList(retVal); } - protected DimensionHandler[] makeDimensionHandlers(final List mergedDimensions, final List dimCapabilities) - { - final DimensionHandler[] handlers = new DimensionHandler[mergedDimensions.size()]; - for (int i = 0; i < mergedDimensions.size(); i++) { - ColumnCapabilities capabilities = dimCapabilities.get(i); - String dimName = mergedDimensions.get(i); - handlers[i] = DimensionHandlerUtils.getHandlerFromCapabilities(dimName, capabilities, null); - } - return handlers; - } + File persist(IncrementalIndex index, File outDir, IndexSpec indexSpec) throws IOException; - protected Iterable makeRowIterable( - List indexes, - final List mergedDimensions, - final List mergedMetrics, - Function>, Iterable> rowMergerFn, - final List dimCapabilities, - final DimensionHandler[] handlers, - final List mergers - ) - { - ArrayList> boats = Lists.newArrayListWithCapacity(indexes.size()); - - for (int i = 0; i < indexes.size(); ++i) { - final IndexableAdapter adapter = indexes.get(i); - - final int[] dimLookup = getColumnIndexReorderingMap(adapter.getDimensionNames(), mergedDimensions); - final int[] metricLookup = getColumnIndexReorderingMap(adapter.getMetricNames(), mergedMetrics); - - Iterable target = indexes.get(i).getRows(); - if (dimLookup != null || metricLookup != null) { - // resize/reorder index table if needed - target = Iterables.transform( - target, - new Function() - { - @Override - public Rowboat apply(Rowboat input) - { - Object[] newDims; - if (dimLookup != null) { - newDims = new Object[mergedDimensions.size()]; - int j = 0; - for (Object dim : input.getDims()) { - newDims[dimLookup[j]] = dim; - j++; - } - } else { - // It's possible for getColumnIndexReorderingMap to return null when - // both column lists are identical. Copy the old array, no dimension reordering is needed. - newDims = input.getDims(); - } - - Object[] newMetrics = input.getMetrics(); - if (metricLookup != null) { - newMetrics = new Object[mergedMetrics.size()]; - int j = 0; - for (Object met : input.getMetrics()) { - newMetrics[metricLookup[j]] = met; - j++; - } - } - - return new Rowboat( - input.getTimestamp(), - newDims, - newMetrics, - input.getRowNum(), - handlers - ); - } - } - ); - } - boats.add( - new MMappedIndexRowIterable( - target, mergedDimensions, i, dimCapabilities, mergers - ) - ); - } + /** + * This is *not* thread-safe and havok will ensue if this is called and writes are still occurring + * on the IncrementalIndex object. + * + * @param index the IncrementalIndex to persist + * @param dataInterval the Interval that the data represents + * @param outDir the directory to persist the data to + * + * @return the index output directory + * + * @throws java.io.IOException if an IO error occurs persisting the index + */ + File persist(IncrementalIndex index, Interval dataInterval, File outDir, IndexSpec indexSpec) throws IOException; - return rowMergerFn.apply(boats); - } + File persist( + IncrementalIndex index, + Interval dataInterval, + File outDir, + IndexSpec indexSpec, + ProgressIndicator progress + ) throws IOException; - // If an adapter's column list differs from the merged column list across multiple indexes, - // return an array that maps the adapter's column orderings to the larger, merged column ordering - private int[] getColumnIndexReorderingMap(Indexed adapterColumnNames, List mergedColumnNames) - { - if (isSame(adapterColumnNames, mergedColumnNames)) { - return null; // no need to convert if column lists are identical - } - int[] dimLookup = new int[mergedColumnNames.size()]; - for (int i = 0; i < adapterColumnNames.size(); i++) { - dimLookup[i] = mergedColumnNames.indexOf(adapterColumnNames.get(i)); - } - return dimLookup; - } + File mergeQueryableIndex( + List indexes, + boolean rollup, + AggregatorFactory[] metricAggs, + File outDir, + IndexSpec indexSpec + ) throws IOException; - private boolean isSame(Indexed indexed, List values) - { - if (indexed.size() != values.size()) { - return false; - } - for (int i = 0; i < indexed.size(); i++) { - if (!indexed.get(i).equals(values.get(i))) { - return false; - } - } - return true; - } + File mergeQueryableIndex( + List indexes, + boolean rollup, + AggregatorFactory[] metricAggs, + File outDir, + IndexSpec indexSpec, + ProgressIndicator progress + ) throws IOException; - public static > ArrayList mergeIndexed(final List> indexedLists) - { - Set retVal = Sets.newTreeSet(Comparators.naturalNullsFirst()); + File merge( + List indexes, + boolean rollup, + AggregatorFactory[] metricAggs, + File outDir, + IndexSpec indexSpec + ) throws IOException; - for (Iterable indexedList : indexedLists) { - for (T val : indexedList) { - retVal.add(val); - } - } + File merge( + List indexes, + boolean rollup, + AggregatorFactory[] metricAggs, + File outDir, + IndexSpec indexSpec, + ProgressIndicator progress + ) throws IOException; - return Lists.newArrayList(retVal); - } + // Faster than IndexMaker + File convert(File inDir, File outDir, IndexSpec indexSpec) throws IOException; - public void createIndexDrdFile( - byte versionId, - File inDir, - GenericIndexed availableDimensions, - GenericIndexed availableMetrics, - Interval dataInterval, - BitmapSerdeFactory bitmapSerdeFactory - ) throws IOException - { - File indexFile = new File(inDir, "index.drd"); + File convert(File inDir, File outDir, IndexSpec indexSpec, ProgressIndicator progress) + throws IOException; - try (FileChannel channel = new FileOutputStream(indexFile).getChannel()) { - channel.write(ByteBuffer.wrap(new byte[]{versionId})); + File append(List indexes, AggregatorFactory[] aggregators, File outDir, IndexSpec indexSpec) + throws IOException; - availableDimensions.writeToChannel(channel); - availableMetrics.writeToChannel(channel); - serializerUtils.writeString( - channel, String.format("%s/%s", dataInterval.getStart(), dataInterval.getEnd()) - ); - serializerUtils.writeString( - channel, mapper.writeValueAsString(bitmapSerdeFactory) - ); - } - IndexIO.checkFileSize(indexFile); - } + File append( + List indexes, + AggregatorFactory[] aggregators, + File outDir, + IndexSpec indexSpec, + ProgressIndicator progress + ) throws IOException; - static interface IndexSeeker + interface IndexSeeker { int NOT_EXIST = -1; int NOT_INIT = -1; @@ -1064,7 +241,7 @@ static interface IndexSeeker /** * Get old dictId from new dictId, and only support access in order */ - static class IndexSeekerWithConversion implements IndexSeeker + class IndexSeekerWithConversion implements IndexSeeker { private final IntBuffer dimConversions; private int currIndex; @@ -1075,26 +252,26 @@ static class IndexSeekerWithConversion implements IndexSeeker { this.dimConversions = dimConversions; this.currIndex = 0; - this.currVal = NOT_INIT; - this.lastVal = NOT_INIT; + this.currVal = IndexSeeker.NOT_INIT; + this.lastVal = IndexSeeker.NOT_INIT; } @Override public int seek(int dictId) { if (dimConversions == null) { - return NOT_EXIST; + return IndexSeeker.NOT_EXIST; } - if (lastVal != NOT_INIT) { + if (lastVal != IndexSeeker.NOT_INIT) { if (dictId <= lastVal) { throw new ISE( "Value dictId[%d] is less than the last value dictId[%d] I have, cannot be.", dictId, lastVal ); } - return NOT_EXIST; + return IndexSeeker.NOT_EXIST; } - if (currVal == NOT_INIT) { + if (currVal == IndexSeeker.NOT_INIT) { currVal = dimConversions.get(); } if (currVal == dictId) { @@ -1112,12 +289,12 @@ public int seek(int dictId) currVal, currIndex, dictId ); } else { - return NOT_EXIST; + return IndexSeeker.NOT_EXIST; } } } - public static class MMappedIndexRowIterable implements Iterable + class MMappedIndexRowIterable implements Iterable { private final Iterable index; private final List convertedDims; @@ -1182,7 +359,7 @@ public Rowboat apply(@Nullable Rowboat input) } } - public static class RowboatMergeFunction implements BinaryFn + class RowboatMergeFunction implements BinaryFn { private final AggregatorFactory[] metricAggs; @@ -1226,7 +403,8 @@ public Rowboat apply(Rowboat lhs, Rowboat rhs) ); for (Rowboat rowboat : Arrays.asList(lhs, rhs)) { - Iterator> entryIterator = rowboat.getComprisedRows().int2ObjectEntrySet().fastIterator(); + Iterator> entryIterator = + rowboat.getComprisedRows().int2ObjectEntrySet().fastIterator(); while (entryIterator.hasNext()) { Int2ObjectMap.Entry entry = entryIterator.next(); @@ -1240,33 +418,7 @@ public Rowboat apply(Rowboat lhs, Rowboat rhs) } } - public static boolean isNullColumn(Iterable dimValues) - { - if (dimValues == null) { - return true; - } - for (String val : dimValues) { - if (val != null) { - return false; - } - } - return true; - } - - private void writeMetadataToFile(File metadataFile, Metadata metadata) throws IOException - { - try (FileOutputStream metadataFileOutputStream = new FileOutputStream(metadataFile); - FileChannel metadataFilechannel = metadataFileOutputStream.getChannel() - ) { - byte[] metadataBytes = mapper.writeValueAsBytes(metadata); - if (metadataBytes.length != metadataFilechannel.write(ByteBuffer.wrap(metadataBytes))) { - throw new IOException("Failed to write metadata for file"); - } - } - IndexIO.checkFileSize(metadataFile); - } - - static class DictionaryMergeIterator implements Iterator + class DictionaryMergeIterator implements Iterator { protected final IntBuffer[] conversions; protected final PriorityQueue>> pQueue; diff --git a/processing/src/main/java/io/druid/segment/IndexMergerV9.java b/processing/src/main/java/io/druid/segment/IndexMergerV9.java index f470b15b4d74..ec340e2ba629 100644 --- a/processing/src/main/java/io/druid/segment/IndexMergerV9.java +++ b/processing/src/main/java/io/druid/segment/IndexMergerV9.java @@ -21,17 +21,26 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.google.common.io.ByteStreams; import com.google.common.io.Files; import com.google.common.primitives.Ints; +import com.google.common.primitives.Longs; import com.google.inject.Inject; +import io.druid.collections.CombiningIterable; import io.druid.common.utils.JodaUtils; import io.druid.io.ZeroCopyByteArrayOutputStream; +import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; import io.druid.java.util.common.guava.Comparators; +import io.druid.java.util.common.guava.FunctionalIterable; +import io.druid.java.util.common.guava.MergeIterable; import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.java.util.common.io.smoosh.SmooshedWriter; @@ -46,7 +55,10 @@ import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.IOPeon; +import io.druid.segment.data.Indexed; import io.druid.segment.data.TmpFileIOPeon; +import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexAdapter; import io.druid.segment.loading.MMappedQueryableSegmentizerFactory; import io.druid.segment.serde.ComplexColumnPartSerde; import io.druid.segment.serde.ComplexMetricSerde; @@ -60,6 +72,8 @@ import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.annotation.Nullable; +import java.io.Closeable; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; @@ -72,9 +86,11 @@ import java.util.Map; import java.util.Set; -public class IndexMergerV9 extends IndexMerger +public class IndexMergerV9 implements IndexMerger { private static final Logger log = new Logger(IndexMergerV9.class); + protected final ObjectMapper mapper; + protected final IndexIO indexIO; @Inject public IndexMergerV9( @@ -82,11 +98,24 @@ public IndexMergerV9( IndexIO indexIO ) { - super(mapper, indexIO); + this.mapper = Preconditions.checkNotNull(mapper, "null ObjectMapper"); + this.indexIO = Preconditions.checkNotNull(indexIO, "null IndexIO"); + } - @Override - protected File makeIndexFiles( + private static void registerDeleteDirectory(Closer closer, final File dir) + { + closer.register(new Closeable() + { + @Override + public void close() throws IOException + { + FileUtils.deleteDirectory(dir); + } + }); + } + + private File makeIndexFiles( final List adapters, final AggregatorFactory[] metricAggs, final File outDir, @@ -581,4 +610,450 @@ private void mergeCapabilities( dimCapabilities.add(capabilitiesMap.get(dim)); } } + + @Override + public File persist( + final IncrementalIndex index, + File outDir, + IndexSpec indexSpec + ) throws IOException + { + return persist(index, index.getInterval(), outDir, indexSpec); + } + + @Override + public File persist( + final IncrementalIndex index, + final Interval dataInterval, + File outDir, + IndexSpec indexSpec + ) throws IOException + { + return persist(index, dataInterval, outDir, indexSpec, new BaseProgressIndicator()); + } + + @Override + public File persist( + final IncrementalIndex index, + final Interval dataInterval, + File outDir, + IndexSpec indexSpec, + ProgressIndicator progress + ) throws IOException + { + if (index.isEmpty()) { + throw new IAE("Trying to persist an empty index!"); + } + + final long firstTimestamp = index.getMinTime().getMillis(); + final long lastTimestamp = index.getMaxTime().getMillis(); + if (!(dataInterval.contains(firstTimestamp) && dataInterval.contains(lastTimestamp))) { + throw new IAE( + "interval[%s] does not encapsulate the full range of timestamps[%s, %s]", + dataInterval, + new DateTime(firstTimestamp), + new DateTime(lastTimestamp) + ); + } + + FileUtils.forceMkdir(outDir); + + log.info("Starting persist for interval[%s], rows[%,d]", dataInterval, index.size()); + return merge( + Arrays.asList( + new IncrementalIndexAdapter( + dataInterval, + index, + indexSpec.getBitmapSerdeFactory().getBitmapFactory() + ) + ), + // if index is not rolled up, then it should be not rollup here + // if index is rolled up, then it is no need to rollup again. + // In this case, true/false won't cause reOrdering in merge stage + // while merging a single iterable + false, + index.getMetricAggs(), + outDir, + indexSpec, + progress + ); + } + + @Override + public File mergeQueryableIndex( + List indexes, + boolean rollup, + final AggregatorFactory[] metricAggs, + File outDir, + IndexSpec indexSpec + ) throws IOException + { + return mergeQueryableIndex(indexes, rollup, metricAggs, outDir, indexSpec, new BaseProgressIndicator()); + } + + @Override + public File mergeQueryableIndex( + List indexes, + boolean rollup, + final AggregatorFactory[] metricAggs, + File outDir, + IndexSpec indexSpec, + ProgressIndicator progress + ) throws IOException + { + return merge( + IndexMerger.toIndexableAdapters(indexes), + rollup, + metricAggs, + outDir, + indexSpec, + progress + ); + } + + @Override + public File merge( + List indexes, + boolean rollup, + final AggregatorFactory[] metricAggs, + File outDir, + IndexSpec indexSpec + ) throws IOException + { + return merge(indexes, rollup, metricAggs, outDir, indexSpec, new BaseProgressIndicator()); + } + + @Override + public File merge( + List indexes, + final boolean rollup, + final AggregatorFactory[] metricAggs, + File outDir, + IndexSpec indexSpec, + ProgressIndicator progress + ) throws IOException + { + FileUtils.deleteDirectory(outDir); + FileUtils.forceMkdir(outDir); + + final List mergedDimensions = IndexMerger.getMergedDimensions(indexes); + + final List mergedMetrics = Lists.transform( + IndexMerger.mergeIndexed( + Lists.newArrayList( + FunctionalIterable + .create(indexes) + .transform( + new Function>() + { + @Override + public Iterable apply(@Nullable IndexableAdapter input) + { + return input.getMetricNames(); + } + } + ) + ) + ), + new Function() + { + @Override + public String apply(@Nullable String input) + { + return input; + } + } + ); + + final AggregatorFactory[] sortedMetricAggs = new AggregatorFactory[mergedMetrics.size()]; + for (int i = 0; i < metricAggs.length; i++) { + AggregatorFactory metricAgg = metricAggs[i]; + int metricIndex = mergedMetrics.indexOf(metricAgg.getName()); + /* + If metricIndex is negative, one of the metricAggs was not present in the union of metrics from the indices + we are merging + */ + if (metricIndex > -1) { + sortedMetricAggs[metricIndex] = metricAgg; + } + } + + /* + If there is nothing at sortedMetricAggs[i], then we did not have a metricAgg whose name matched the name + of the ith element of mergedMetrics. I.e. There was a metric in the indices to merge that we did not ask for. + */ + for (int i = 0; i < sortedMetricAggs.length; i++) { + if (sortedMetricAggs[i] == null) { + throw new IAE("Indices to merge contained metric[%s], but requested metrics did not", mergedMetrics.get(i)); + } + } + + for (int i = 0; i < mergedMetrics.size(); i++) { + if (!sortedMetricAggs[i].getName().equals(mergedMetrics.get(i))) { + throw new IAE( + "Metric mismatch, index[%d] [%s] != [%s]", + i, + sortedMetricAggs[i].getName(), + mergedMetrics.get(i) + ); + } + } + + Function>, Iterable> rowMergerFn = new Function>, Iterable>() + { + @Override + public Iterable apply( + @Nullable ArrayList> boats + ) + { + if (rollup) { + return CombiningIterable.create( + new MergeIterable<>(Comparators.naturalNullsFirst(), boats), + Comparators.naturalNullsFirst(), + new RowboatMergeFunction(sortedMetricAggs) + ); + } else { + return new MergeIterable( + new Ordering() + { + @Override + public int compare(Rowboat left, Rowboat right) + { + return Longs.compare(left.getTimestamp(), right.getTimestamp()); + } + }.nullsFirst(), + boats + ); + } + } + }; + + return makeIndexFiles( + indexes, + sortedMetricAggs, + outDir, + progress, + mergedDimensions, + mergedMetrics, + rowMergerFn, + indexSpec + ); + } + + @Override + public File convert(final File inDir, final File outDir, final IndexSpec indexSpec) throws IOException + { + return convert(inDir, outDir, indexSpec, new BaseProgressIndicator()); + } + + @Override + public File convert(final File inDir, final File outDir, final IndexSpec indexSpec, final ProgressIndicator progress) + throws IOException + { + try (QueryableIndex index = indexIO.loadIndex(inDir)) { + final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(index); + return makeIndexFiles( + ImmutableList.of(adapter), + null, + outDir, + progress, + Lists.newArrayList(adapter.getDimensionNames()), + Lists.newArrayList(adapter.getMetricNames()), + new Function>, Iterable>() + { + @Nullable + @Override + public Iterable apply(ArrayList> input) + { + return input.get(0); + } + }, + indexSpec + ); + } + } + + @Override + public File append( + List indexes, AggregatorFactory[] aggregators, File outDir, IndexSpec indexSpec + ) throws IOException + { + return append(indexes, aggregators, outDir, indexSpec, new BaseProgressIndicator()); + } + + @Override + public File append( + List indexes, + AggregatorFactory[] aggregators, + File outDir, + IndexSpec indexSpec, + ProgressIndicator progress + ) throws IOException + { + FileUtils.deleteDirectory(outDir); + FileUtils.forceMkdir(outDir); + + final List mergedDimensions = IndexMerger.getMergedDimensions(indexes); + + final List mergedMetrics = IndexMerger.mergeIndexed( + Lists.transform( + indexes, + new Function>() + { + @Override + public Iterable apply(@Nullable IndexableAdapter input) + { + return Iterables.transform( + input.getMetricNames(), + new Function() + { + @Override + public String apply(@Nullable String input) + { + return input; + } + } + ); + } + } + ) + ); + + Function>, Iterable> rowMergerFn = new Function>, Iterable>() + { + @Override + public Iterable apply( + @Nullable final ArrayList> boats + ) + { + return new MergeIterable<>(Comparators.naturalNullsFirst(), boats); + } + }; + + return makeIndexFiles( + indexes, + aggregators, + outDir, + progress, + mergedDimensions, + mergedMetrics, + rowMergerFn, + indexSpec + ); + } + + private DimensionHandler[] makeDimensionHandlers( + final List mergedDimensions, + final List dimCapabilities + ) + { + final DimensionHandler[] handlers = new DimensionHandler[mergedDimensions.size()]; + for (int i = 0; i < mergedDimensions.size(); i++) { + ColumnCapabilities capabilities = dimCapabilities.get(i); + String dimName = mergedDimensions.get(i); + handlers[i] = DimensionHandlerUtils.getHandlerFromCapabilities(dimName, capabilities, null); + } + return handlers; + } + + private Iterable makeRowIterable( + List indexes, + final List mergedDimensions, + final List mergedMetrics, + Function>, Iterable> rowMergerFn, + final List dimCapabilities, + final DimensionHandler[] handlers, + final List mergers + ) + { + ArrayList> boats = Lists.newArrayListWithCapacity(indexes.size()); + + for (int i = 0; i < indexes.size(); ++i) { + final IndexableAdapter adapter = indexes.get(i); + + final int[] dimLookup = getColumnIndexReorderingMap(adapter.getDimensionNames(), mergedDimensions); + final int[] metricLookup = getColumnIndexReorderingMap(adapter.getMetricNames(), mergedMetrics); + + Iterable target = indexes.get(i).getRows(); + if (dimLookup != null || metricLookup != null) { + // resize/reorder index table if needed + target = Iterables.transform( + target, + new Function() + { + @Override + public Rowboat apply(Rowboat input) + { + Object[] newDims; + if (dimLookup != null) { + newDims = new Object[mergedDimensions.size()]; + int j = 0; + for (Object dim : input.getDims()) { + newDims[dimLookup[j]] = dim; + j++; + } + } else { + // It's possible for getColumnIndexReorderingMap to return null when + // both column lists are identical. Copy the old array, no dimension reordering is needed. + newDims = input.getDims(); + } + + Object[] newMetrics = input.getMetrics(); + if (metricLookup != null) { + newMetrics = new Object[mergedMetrics.size()]; + int j = 0; + for (Object met : input.getMetrics()) { + newMetrics[metricLookup[j]] = met; + j++; + } + } + + return new Rowboat( + input.getTimestamp(), + newDims, + newMetrics, + input.getRowNum(), + handlers + ); + } + } + ); + } + boats.add( + new MMappedIndexRowIterable( + target, mergedDimensions, i, dimCapabilities, mergers + ) + ); + } + + return rowMergerFn.apply(boats); + } + + // If an adapter's column list differs from the merged column list across multiple indexes, + // return an array that maps the adapter's column orderings to the larger, merged column ordering + private int[] getColumnIndexReorderingMap(Indexed adapterColumnNames, List mergedColumnNames) + { + if (isSame(adapterColumnNames, mergedColumnNames)) { + return null; // no need to convert if column lists are identical + } + int[] dimLookup = new int[mergedColumnNames.size()]; + for (int i = 0; i < adapterColumnNames.size(); i++) { + dimLookup[i] = mergedColumnNames.indexOf(adapterColumnNames.get(i)); + } + return dimLookup; + } + + private boolean isSame(Indexed indexed, List values) + { + if (indexed.size() != values.size()) { + return false; + } + for (int i = 0; i < indexed.size(); i++) { + if (!indexed.get(i).equals(values.get(i))) { + return false; + } + } + return true; + } + } diff --git a/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java b/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java index 614d45ff18b0..ece87db637d7 100644 --- a/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java +++ b/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java @@ -149,7 +149,7 @@ public static void setupClass() throws Exception } persistedSegmentDir = Files.createTempDir(); - TestHelper.getTestIndexMerger() + TestHelper.getTestIndexMergerV9() .persist(incrementalIndex, persistedSegmentDir, new IndexSpec()); queryableIndex = TestHelper.getTestIndexIO().loadIndex(persistedSegmentDir); diff --git a/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java index d85979fd78b6..a71053d916a4 100644 --- a/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java +++ b/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java @@ -70,6 +70,7 @@ import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; +import io.druid.segment.IndexMergerV9; import io.druid.segment.IndexSpec; import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndexSegment; @@ -154,7 +155,7 @@ public int columnCacheSizeBytes() return new AggregationTestHelper( mapper, - new IndexMerger(mapper, indexIO), + new IndexMergerV9(mapper, indexIO), indexIO, factory.getToolchest(), factory, @@ -210,7 +211,7 @@ public int columnCacheSizeBytes() return new AggregationTestHelper( mapper, - new IndexMerger(mapper, indexIO), + new IndexMergerV9(mapper, indexIO), indexIO, toolchest, factory, @@ -250,7 +251,7 @@ public int columnCacheSizeBytes() return new AggregationTestHelper( mapper, - new IndexMerger(mapper, indexIO), + new IndexMergerV9(mapper, indexIO), indexIO, toolchest, factory, @@ -301,7 +302,7 @@ public int columnCacheSizeBytes() return new AggregationTestHelper( mapper, - new IndexMerger(mapper, indexIO), + new IndexMergerV9(mapper, indexIO), indexIO, toolchest, factory, diff --git a/processing/src/test/java/io/druid/segment/EmptyIndexTest.java b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java index 58c3f3a53142..2518fb0625c9 100644 --- a/processing/src/test/java/io/druid/segment/EmptyIndexTest.java +++ b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java @@ -60,7 +60,7 @@ public void testEmptyIndex() throws Exception emptyIndex, new ConciseBitmapFactory() ); - TestHelper.getTestIndexMerger().merge( + TestHelper.getTestIndexMergerV9().merge( Lists.newArrayList(emptyIndexAdapter), true, new AggregatorFactory[0], diff --git a/processing/src/test/java/io/druid/segment/IndexBuilder.java b/processing/src/test/java/io/druid/segment/IndexBuilder.java index 35d2611ba3ef..3679e19277ea 100644 --- a/processing/src/test/java/io/druid/segment/IndexBuilder.java +++ b/processing/src/test/java/io/druid/segment/IndexBuilder.java @@ -50,7 +50,7 @@ public class IndexBuilder private IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder().withMetrics(new AggregatorFactory[]{ new CountAggregatorFactory("count") }).build(); - private IndexMerger indexMerger = TestHelper.getTestIndexMerger(); + private IndexMerger indexMerger = TestHelper.getTestIndexMergerV9(); private File tmpDir; private IndexSpec indexSpec = new IndexSpec(); private int maxRows = DEFAULT_MAX_ROWS; diff --git a/processing/src/test/java/io/druid/segment/IndexMergerTest.java b/processing/src/test/java/io/druid/segment/IndexMergerTest.java index 565b26ec4d14..d9d0f8fe8f06 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerTest.java @@ -97,10 +97,6 @@ public static Collection data() return Collections2.transform( Sets.cartesianProduct( ImmutableList.of( - ImmutableSet.of( - true, - false - ), ImmutableSet.of( new RoaringBitmapSerdeFactory(null), new ConciseBitmapSerdeFactory() @@ -145,7 +141,6 @@ static IndexSpec makeIndexSpec( public final CloserRule closer = new CloserRule(false); public IndexMergerTest( - boolean useV9, BitmapSerdeFactory bitmapSerdeFactory, CompressedObjectStrategy.CompressionStrategy compressionStrategy, CompressedObjectStrategy.CompressionStrategy dimCompressionStrategy, @@ -153,11 +148,7 @@ public IndexMergerTest( ) { this.indexSpec = makeIndexSpec(bitmapSerdeFactory, compressionStrategy, dimCompressionStrategy, longEncodingStrategy); - if (useV9) { - INDEX_MERGER = TestHelper.getTestIndexMergerV9(); - } else { - INDEX_MERGER = TestHelper.getTestIndexMerger(); - } + INDEX_MERGER = TestHelper.getTestIndexMergerV9(); } @Test diff --git a/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java b/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java index fc629594bc21..ad4e255bee6a 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java @@ -19,11 +19,9 @@ package io.druid.segment; -import com.google.common.base.Function; -import com.google.common.collect.Collections2; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; +import com.google.common.io.ByteSource; import com.google.common.io.Files; import io.druid.common.utils.JodaUtils; import io.druid.data.input.InputRow; @@ -43,12 +41,10 @@ import org.junit.Before; import org.junit.Rule; import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import javax.annotation.Nullable; import java.io.File; import java.io.IOException; +import java.io.InputStream; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -57,7 +53,6 @@ import java.util.List; import java.util.Map; -@RunWith(Parameterized.class) public class IndexMergerV9CompatibilityTest { @Rule @@ -71,7 +66,6 @@ public class IndexMergerV9CompatibilityTest private static final IndexMergerV9 INDEX_MERGER_V9 = TestHelper.getTestIndexMergerV9(); private static final IndexIO INDEX_IO = TestHelper.getTestIndexIO(); - private static final IndexMerger INDEX_MERGER = TestHelper.getTestIndexMerger(); private static final IndexSpec INDEX_SPEC = IndexMergerTest.makeIndexSpec( new ConciseBitmapSerdeFactory(), @@ -81,38 +75,12 @@ public class IndexMergerV9CompatibilityTest ); private static final List DIMS = ImmutableList.of("dim0", "dim1"); - private static final Function>, Object[]> OBJECT_MAKER = new Function>, Object[]>() - { - @Nullable - @Override - public Object[] apply(Collection> input) - { - final ArrayList list = new ArrayList<>(); - int i = 0; - for (final Map map : input) { - list.add(new MapBasedInputRow(TIMESTAMP + i++, DIMS, map)); - } - return new Object[]{list}; - } - }; + private final Collection events; - @SafeVarargs - public static Collection permute(Map... maps) + public IndexMergerV9CompatibilityTest() { - if (maps == null) { - return ImmutableList.of(); - } - return Collections2.transform( - Collections2.permutations( - Arrays.asList(maps) - ), - OBJECT_MAKER - ); - } + events = new ArrayList<>(); - @Parameterized.Parameters - public static Iterable paramFeeder() - { final Map map1 = ImmutableMap.of( DIMS.get(0), ImmutableList.of("dim00", "dim01"), DIMS.get(1), "dim10" @@ -138,25 +106,10 @@ public static Iterable paramFeeder() final Map map6 = new HashMap<>(); map6.put(DIMS.get(1), null); // ImmutableMap cannot take null - - return Iterables.concat( - permute(map1) - , permute(map1, map4) - , permute(map1, map5) - , permute(map5, map6) - , permute(map4, map5) - , Iterables.transform(ImmutableList.of(Arrays.asList(map1, map2, map3, map4, map5, map6)), OBJECT_MAKER) - ); - - } - - private final Collection events; - - public IndexMergerV9CompatibilityTest( - final Collection events - ) - { - this.events = events; + int i = 0; + for (final Map map : Arrays.asList(map1, map2, map3, map4, map5, map6)) { + events.add(new MapBasedInputRow(TIMESTAMP + i++, DIMS, map)); + } } IncrementalIndex toPersist; @@ -178,7 +131,18 @@ public void setUp() throws IOException } tmpDir = Files.createTempDir(); persistTmpDir = new File(tmpDir, "persistDir"); - INDEX_MERGER.persist(toPersist, persistTmpDir, INDEX_SPEC); + FileUtils.forceMkdir(persistTmpDir); + String[] files = new String[] {"00000.smoosh", "meta.smoosh", "version.bin"}; + for (String file : files) { + new ByteSource() + { + @Override + public InputStream openStream() throws IOException + { + return IndexMergerV9CompatibilityTest.class.getResourceAsStream("/v8SegmentPersistDir/" + file); + } + }.copyTo(Files.asByteSink(new File(persistTmpDir, file))); + } } @After @@ -234,18 +198,6 @@ private File reprocessAndValidate(File inDir, File tmpDir) throws IOException return outDir; } - private File appendAndValidate(File inDir, File tmpDir) throws IOException - { - final File outDir = INDEX_MERGER.append( - ImmutableList.of(new QueryableIndexIndexableAdapter(closer.closeLater(INDEX_IO.loadIndex(inDir)))), - null, - tmpDir, - INDEX_SPEC - ); - INDEX_IO.validateTwoSegments(persistTmpDir, outDir); - return outDir; - } - @Test public void testIdempotentReprocess() throws IOException { @@ -270,43 +222,4 @@ public void testIdempotentReprocess() throws IOException Assert.assertEquals(events.size(), adapter3.getNumRows()); reprocessAndValidate(tmpDir2, tmpDir3); } - - @Test - public void testSimpleAppend() throws IOException - { - final IndexableAdapter adapter = new QueryableIndexIndexableAdapter( - closer.closeLater( - INDEX_IO.loadIndex( - persistTmpDir - ) - ) - ); - Assert.assertEquals(events.size(), adapter.getNumRows()); - appendAndValidate(persistTmpDir, new File(tmpDir, "reprocessed")); - } - - @Test - public void testIdempotentAppend() throws IOException - { - final IndexableAdapter adapter = new QueryableIndexIndexableAdapter( - closer.closeLater( - INDEX_IO.loadIndex( - persistTmpDir - ) - ) - ); - Assert.assertEquals(events.size(), adapter.getNumRows()); - final File tmpDir1 = new File(tmpDir, "reprocessed1"); - appendAndValidate(persistTmpDir, tmpDir1); - - final File tmpDir2 = new File(tmpDir, "reprocessed2"); - final IndexableAdapter adapter2 = new QueryableIndexIndexableAdapter(closer.closeLater(INDEX_IO.loadIndex(tmpDir1))); - Assert.assertEquals(events.size(), adapter2.getNumRows()); - appendAndValidate(tmpDir1, tmpDir2); - - final File tmpDir3 = new File(tmpDir, "reprocessed3"); - final IndexableAdapter adapter3 = new QueryableIndexIndexableAdapter(closer.closeLater(INDEX_IO.loadIndex(tmpDir2))); - Assert.assertEquals(events.size(), adapter3.getNumRows()); - appendAndValidate(tmpDir2, tmpDir3); - } } diff --git a/processing/src/test/java/io/druid/segment/QueryableIndexIndexableAdapterTest.java b/processing/src/test/java/io/druid/segment/QueryableIndexIndexableAdapterTest.java index e3a02fabac2e..5dc2b502fd39 100644 --- a/processing/src/test/java/io/druid/segment/QueryableIndexIndexableAdapterTest.java +++ b/processing/src/test/java/io/druid/segment/QueryableIndexIndexableAdapterTest.java @@ -34,7 +34,7 @@ public class QueryableIndexIndexableAdapterTest { - private final static IndexMerger INDEX_MERGER = TestHelper.getTestIndexMerger(); + private final static IndexMerger INDEX_MERGER = TestHelper.getTestIndexMergerV9(); private final static IndexIO INDEX_IO = TestHelper.getTestIndexIO(); private static final IndexSpec INDEX_SPEC = IndexMergerTest.makeIndexSpec( new ConciseBitmapSerdeFactory(), diff --git a/processing/src/test/java/io/druid/segment/SchemalessIndexTest.java b/processing/src/test/java/io/druid/segment/SchemalessIndexTest.java index 724a609b6501..f048d7b682e1 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessIndexTest.java +++ b/processing/src/test/java/io/druid/segment/SchemalessIndexTest.java @@ -86,7 +86,7 @@ public class SchemalessIndexTest private static final Map> mergedIndexes = Maps.newHashMap(); private static final List rowPersistedIndexes = Lists.newArrayList(); - private static final IndexMerger INDEX_MERGER = TestHelper.getTestIndexMerger(); + private static final IndexMerger INDEX_MERGER = TestHelper.getTestIndexMergerV9(); private static final IndexIO INDEX_IO = TestHelper.getTestIndexIO(); private static IncrementalIndex index = null; diff --git a/processing/src/test/java/io/druid/segment/TestHelper.java b/processing/src/test/java/io/druid/segment/TestHelper.java index cba996ef14f2..16259313b660 100644 --- a/processing/src/test/java/io/druid/segment/TestHelper.java +++ b/processing/src/test/java/io/druid/segment/TestHelper.java @@ -41,7 +41,6 @@ */ public class TestHelper { - private static final IndexMerger INDEX_MERGER; private static final IndexMergerV9 INDEX_MERGER_V9; private static final IndexIO INDEX_IO; @@ -58,15 +57,9 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER = new IndexMerger(jsonMapper, INDEX_IO); INDEX_MERGER_V9 = new IndexMergerV9(jsonMapper, INDEX_IO); } - public static IndexMerger getTestIndexMerger() - { - return INDEX_MERGER; - } - public static IndexMergerV9 getTestIndexMergerV9() { return INDEX_MERGER_V9; diff --git a/processing/src/test/java/io/druid/segment/TestIndex.java b/processing/src/test/java/io/druid/segment/TestIndex.java index 9fd84655a9e4..566d2f7174a7 100644 --- a/processing/src/test/java/io/druid/segment/TestIndex.java +++ b/processing/src/test/java/io/druid/segment/TestIndex.java @@ -125,7 +125,7 @@ public class TestIndex }; private static final IndexSpec indexSpec = new IndexSpec(); - private static final IndexMerger INDEX_MERGER = TestHelper.getTestIndexMerger(); + private static final IndexMerger INDEX_MERGER = TestHelper.getTestIndexMergerV9(); private static final IndexIO INDEX_IO = TestHelper.getTestIndexIO(); static { diff --git a/processing/src/test/java/io/druid/segment/filter/BaseFilterTest.java b/processing/src/test/java/io/druid/segment/filter/BaseFilterTest.java index 8d18672b8168..7a85b76d901f 100644 --- a/processing/src/test/java/io/druid/segment/filter/BaseFilterTest.java +++ b/processing/src/test/java/io/druid/segment/filter/BaseFilterTest.java @@ -183,8 +183,7 @@ public static Collection makeConstructors() "roaring", new RoaringBitmapSerdeFactory(true) ); - final Map indexMergers = ImmutableMap.of( - "IndexMerger", TestHelper.getTestIndexMerger(), + final Map indexMergers = ImmutableMap.of( "IndexMergerV9", TestHelper.getTestIndexMergerV9() ); diff --git a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java index 8ac4fad7178f..cff578ed551b 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java @@ -82,7 +82,7 @@ public class SpatialFilterBonusTest new LongSumAggregatorFactory("val", "val") }; private static List DIMS = Lists.newArrayList("dim", "dim.geo"); - private static final IndexMerger INDEX_MERGER = TestHelper.getTestIndexMerger(); + private static final IndexMerger INDEX_MERGER = TestHelper.getTestIndexMergerV9(); private static final IndexIO INDEX_IO = TestHelper.getTestIndexIO(); private final Segment segment; diff --git a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java index f2fe2cd886e1..0c058f15127d 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java @@ -72,7 +72,7 @@ @RunWith(Parameterized.class) public class SpatialFilterTest { - private static IndexMerger INDEX_MERGER = TestHelper.getTestIndexMerger(); + private static IndexMerger INDEX_MERGER = TestHelper.getTestIndexMergerV9(); private static IndexIO INDEX_IO = TestHelper.getTestIndexIO(); public static final int NUM_POINTS = 5000; diff --git a/processing/src/test/resources/v8SegmentPersistDir/00000.smoosh b/processing/src/test/resources/v8SegmentPersistDir/00000.smoosh new file mode 100644 index 0000000000000000000000000000000000000000..8a9dcc84e45937adb29d743241cc2a4892ff7d4b GIT binary patch literal 2525 zcmeHJ&ubGw6n?u&YCK44BWgfY=1^2}NId2yZ7rcO6%)Kjk#RGPL-&`nvtkH=9(woS zN&ga0;-4b^BlUZ`8%ZcZPkLyEeEViz-muGi?|Xbi^m!`YOFvb^NvuRi42Cbd{e96C zZ)Nf<^{tM5^^*TjM8}#`HAO6q#o^Ud*!)UjjSk1V+Os+erJ3OJ8*Kv_50n|Hr_!?# zoSonCUKDydQDTO;iB*RPGI(>ZI~?}*UhnmHyHEOJb~JN|$Z;Gzw}BE+2Ce}&0bZ4n z4qC1GrtP{Me1-7L?m2gHHGU-I?iITAyZrHk`Xr0Y(Rt{ZS{MvhDT zyY$KWnJ0@|iyi>nh*e+{xDAv~i8_U?EZ8r9Ppo6D>!L=P3dj{Jb;ufJ zDozxXOfHyMzsT{}sIi2=SyDty)Il16tDy{7Rb`)uOyv7%oGZ22FP`cXI{*Lx literal 0 HcmV?d00001 diff --git a/processing/src/test/resources/v8SegmentPersistDir/meta.smoosh b/processing/src/test/resources/v8SegmentPersistDir/meta.smoosh new file mode 100644 index 000000000000..37e71c95dd2e --- /dev/null +++ b/processing/src/test/resources/v8SegmentPersistDir/meta.smoosh @@ -0,0 +1,7 @@ +v1,2147483647,1 +__time,0,1793,2219 +count,0,1386,1793 +dim0,0,0,857 +dim1,0,857,1386 +index.drd,0,2219,2338 +metadata.drd,0,2338,2525 diff --git a/processing/src/test/resources/v8SegmentPersistDir/version.bin b/processing/src/test/resources/v8SegmentPersistDir/version.bin new file mode 100644 index 0000000000000000000000000000000000000000..3dd5ace49c65e7fd143d228be939732534095e78 GIT binary patch literal 4 LcmZQzVBiD*01W^N literal 0 HcmV?d00001 diff --git a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java index 99b639da3685..5338055a4d8b 100644 --- a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java @@ -47,7 +47,6 @@ public class RealtimeTuningConfig implements TuningConfig, AppenderatorConfig private static final int defaultMaxPendingPersists = 0; private static final ShardSpec defaultShardSpec = NoneShardSpec.instance(); private static final IndexSpec defaultIndexSpec = new IndexSpec(); - private static final Boolean defaultBuildV9Directly = Boolean.TRUE; private static final Boolean defaultReportParseExceptions = Boolean.FALSE; private static final long defaultHandoffConditionTimeout = 0; private static final long defaultAlertTimeout = 0; @@ -70,7 +69,7 @@ public static RealtimeTuningConfig makeDefaultTuningConfig(final File basePersis defaultMaxPendingPersists, defaultShardSpec, defaultIndexSpec, - defaultBuildV9Directly, + true, 0, 0, defaultReportParseExceptions, @@ -88,7 +87,6 @@ public static RealtimeTuningConfig makeDefaultTuningConfig(final File basePersis private final int maxPendingPersists; private final ShardSpec shardSpec; private final IndexSpec indexSpec; - private final boolean buildV9Directly; private final int persistThreadPriority; private final int mergeThreadPriority; private final boolean reportParseExceptions; @@ -106,6 +104,7 @@ public RealtimeTuningConfig( @JsonProperty("maxPendingPersists") Integer maxPendingPersists, @JsonProperty("shardSpec") ShardSpec shardSpec, @JsonProperty("indexSpec") IndexSpec indexSpec, + // This parameter is left for compatibility when reading existing configs, to be removed in Druid 0.12. @JsonProperty("buildV9Directly") Boolean buildV9Directly, @JsonProperty("persistThreadPriority") int persistThreadPriority, @JsonProperty("mergeThreadPriority") int mergeThreadPriority, @@ -127,7 +126,6 @@ public RealtimeTuningConfig( this.maxPendingPersists = maxPendingPersists == null ? defaultMaxPendingPersists : maxPendingPersists; this.shardSpec = shardSpec == null ? defaultShardSpec : shardSpec; this.indexSpec = indexSpec == null ? defaultIndexSpec : indexSpec; - this.buildV9Directly = buildV9Directly == null ? defaultBuildV9Directly : buildV9Directly; this.mergeThreadPriority = mergeThreadPriority; this.persistThreadPriority = persistThreadPriority; this.reportParseExceptions = reportParseExceptions == null @@ -201,10 +199,14 @@ public IndexSpec getIndexSpec() return indexSpec; } + /** + * Always returns true, doesn't affect the version being built. + */ + @Deprecated @JsonProperty public Boolean getBuildV9Directly() { - return buildV9Directly; + return true; } @JsonProperty @@ -250,7 +252,7 @@ public RealtimeTuningConfig withVersioningPolicy(VersioningPolicy policy) maxPendingPersists, shardSpec, indexSpec, - buildV9Directly, + true, persistThreadPriority, mergeThreadPriority, reportParseExceptions, @@ -271,7 +273,7 @@ public RealtimeTuningConfig withBasePersistDirectory(File dir) maxPendingPersists, shardSpec, indexSpec, - buildV9Directly, + true, persistThreadPriority, mergeThreadPriority, reportParseExceptions, diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java index 9c5bd269bd45..11416650ff9a 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java @@ -30,7 +30,6 @@ import io.druid.guice.annotations.Processing; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMerger; import io.druid.segment.IndexMergerV9; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; @@ -54,7 +53,6 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool private final QueryRunnerFactoryConglomerate conglomerate; private final DataSegmentAnnouncer segmentAnnouncer; private final ExecutorService queryExecutorService; - private final IndexMerger indexMerger; private final IndexMergerV9 indexMergerV9; private final IndexIO indexIO; private final Cache cache; @@ -68,7 +66,6 @@ public FlushingPlumberSchool( @JacksonInject QueryRunnerFactoryConglomerate conglomerate, @JacksonInject DataSegmentAnnouncer segmentAnnouncer, @JacksonInject @Processing ExecutorService queryExecutorService, - @JacksonInject IndexMerger indexMerger, @JacksonInject IndexMergerV9 indexMergerV9, @JacksonInject IndexIO indexIO, @JacksonInject Cache cache, @@ -84,7 +81,6 @@ public FlushingPlumberSchool( null, null, queryExecutorService, - indexMerger, indexMergerV9, indexIO, cache, @@ -97,7 +93,6 @@ public FlushingPlumberSchool( this.conglomerate = conglomerate; this.segmentAnnouncer = segmentAnnouncer; this.queryExecutorService = queryExecutorService; - this.indexMerger = Preconditions.checkNotNull(indexMerger, "Null IndexMerger"); this.indexMergerV9 = Preconditions.checkNotNull(indexMergerV9, "Null IndexMergerV9"); this.indexIO = Preconditions.checkNotNull(indexIO, "Null IndexIO"); this.cache = cache; @@ -123,7 +118,7 @@ public Plumber findPlumber( conglomerate, segmentAnnouncer, queryExecutorService, - config.getBuildV9Directly() ? indexMergerV9 : indexMerger, + indexMergerV9, indexIO, cache, cacheConfig, diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java index 898395380623..8aaf0f2bf7a2 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java @@ -29,7 +29,6 @@ import io.druid.guice.annotations.Processing; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMerger; import io.druid.segment.IndexMergerV9; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; @@ -51,7 +50,6 @@ public class RealtimePlumberSchool implements PlumberSchool private final SegmentPublisher segmentPublisher; private final SegmentHandoffNotifierFactory handoffNotifierFactory; private final ExecutorService queryExecutorService; - private final IndexMerger indexMerger; private final IndexMergerV9 indexMergerV9; private final IndexIO indexIO; private final Cache cache; @@ -67,7 +65,6 @@ public RealtimePlumberSchool( @JacksonInject SegmentPublisher segmentPublisher, @JacksonInject SegmentHandoffNotifierFactory handoffNotifierFactory, @JacksonInject @Processing ExecutorService executorService, - @JacksonInject IndexMerger indexMerger, @JacksonInject IndexMergerV9 indexMergerV9, @JacksonInject IndexIO indexIO, @JacksonInject Cache cache, @@ -82,7 +79,6 @@ public RealtimePlumberSchool( this.segmentPublisher = segmentPublisher; this.handoffNotifierFactory = handoffNotifierFactory; this.queryExecutorService = executorService; - this.indexMerger = Preconditions.checkNotNull(indexMerger, "Null IndexMerger"); this.indexMergerV9 = Preconditions.checkNotNull(indexMergerV9, "Null IndexMergerV9"); this.indexIO = Preconditions.checkNotNull(indexIO, "Null IndexIO"); @@ -111,7 +107,7 @@ public Plumber findPlumber( dataSegmentPusher, segmentPublisher, handoffNotifierFactory.createSegmentHandoffNotifier(schema.getDataSource()), - config.getBuildV9Directly() ? indexMergerV9 : indexMerger, + indexMergerV9, indexIO, cache, cacheConfig, diff --git a/server/src/test/java/io/druid/segment/indexing/RealtimeTuningConfigTest.java b/server/src/test/java/io/druid/segment/indexing/RealtimeTuningConfigTest.java index 44bc5e327ce8..aed2d22cb345 100644 --- a/server/src/test/java/io/druid/segment/indexing/RealtimeTuningConfigTest.java +++ b/server/src/test/java/io/druid/segment/indexing/RealtimeTuningConfigTest.java @@ -65,7 +65,6 @@ public void testSerdeWithDefaults() throws Exception ); Assert.assertNotNull(config.getBasePersistDirectory()); - Assert.assertEquals(true, config.getBuildV9Directly()); Assert.assertEquals(0, config.getHandoffConditionTimeout()); Assert.assertEquals(0, config.getAlertTimeout()); Assert.assertEquals(new IndexSpec(), config.getIndexSpec()); @@ -89,7 +88,6 @@ public void testSerdeWithNonDefaults() throws Exception + " \"windowPeriod\": \"PT1H\",\n" + " \"basePersistDirectory\": \"/tmp/xxx\",\n" + " \"maxPendingPersists\": 100,\n" - + " \"buildV9Directly\": false,\n" + " \"persistThreadPriority\": 100,\n" + " \"mergeThreadPriority\": 100,\n" + " \"reportParseExceptions\": true,\n" @@ -109,7 +107,6 @@ public void testSerdeWithNonDefaults() throws Exception ); Assert.assertEquals("/tmp/xxx", config.getBasePersistDirectory().toString()); - Assert.assertEquals(false, config.getBuildV9Directly()); Assert.assertEquals(100, config.getHandoffConditionTimeout()); Assert.assertEquals(70, config.getAlertTimeout()); Assert.assertEquals(new IndexSpec(), config.getIndexSpec()); diff --git a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java index 65512a3f7c0b..91eb6fd58988 100644 --- a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java @@ -109,7 +109,6 @@ public void testSerde() throws Exception null, null, null, - TestHelper.getTestIndexMerger(), TestHelper.getTestIndexMergerV9(), TestHelper.getTestIndexIO(), MapCache.create(0), diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java index 98399784c511..71fd3714d23c 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java @@ -48,6 +48,7 @@ import io.druid.query.timeseries.TimeseriesQueryRunnerFactory; import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; +import io.druid.segment.IndexMergerV9; import io.druid.segment.column.ColumnConfig; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; @@ -153,7 +154,7 @@ public int columnCacheSizeBytes() } } ); - indexMerger = new IndexMerger(objectMapper, indexIO); + indexMerger = new IndexMergerV9(objectMapper, indexIO); emitter = new ServiceEmitter( "test", diff --git a/server/src/test/java/io/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java b/server/src/test/java/io/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java index 01a90fa8b8bc..7e59e0a377b6 100644 --- a/server/src/test/java/io/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java +++ b/server/src/test/java/io/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java @@ -93,7 +93,7 @@ public class IngestSegmentFirehoseTest public final TemporaryFolder tempFolder = new TemporaryFolder(); private IndexIO indexIO = TestHelper.getTestIndexIO(); - private IndexMerger indexMerger = TestHelper.getTestIndexMerger(); + private IndexMerger indexMerger = TestHelper.getTestIndexMergerV9(); @Test public void testReadFromIndexAndWriteAnotherIndex() throws Exception diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index 6cab395d6969..2c50bcddb406 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -82,7 +82,6 @@ public class RealtimePlumberSchoolTest { private final RejectionPolicyFactory rejectionPolicy; - private final boolean buildV9Directly; private RealtimePlumber plumber; private RealtimePlumberSchool realtimePlumberSchool; private DataSegmentAnnouncer announcer; @@ -97,26 +96,22 @@ public class RealtimePlumberSchoolTest private FireDepartmentMetrics metrics; private File tmpDir; - public RealtimePlumberSchoolTest(RejectionPolicyFactory rejectionPolicy, boolean buildV9Directly) + public RealtimePlumberSchoolTest(RejectionPolicyFactory rejectionPolicy) { this.rejectionPolicy = rejectionPolicy; - this.buildV9Directly = buildV9Directly; } - @Parameterized.Parameters(name = "rejectionPolicy = {0}, buildV9Directly = {1}") + @Parameterized.Parameters(name = "rejectionPolicy = {0}") public static Collection constructorFeeder() throws IOException { final RejectionPolicyFactory[] rejectionPolicies = new RejectionPolicyFactory[]{ new NoopRejectionPolicyFactory(), new MessageTimeRejectionPolicyFactory() }; - final boolean[] buildV9Directlies = new boolean[]{true, false}; final List constructors = Lists.newArrayList(); for (RejectionPolicyFactory rejectionPolicy : rejectionPolicies) { - for (boolean buildV9Directly : buildV9Directlies) { - constructors.add(new Object[]{rejectionPolicy, buildV9Directly}); - } + constructors.add(new Object[]{rejectionPolicy}); } return constructors; } @@ -199,7 +194,7 @@ public void setUp() throws Exception null, null, null, - buildV9Directly, + true, 0, 0, false, @@ -215,7 +210,6 @@ public void setUp() throws Exception segmentPublisher, handoffNotifierFactory, MoreExecutors.sameThreadExecutor(), - TestHelper.getTestIndexMerger(), TestHelper.getTestIndexMergerV9(), TestHelper.getTestIndexIO(), MapCache.create(0), From c4c6b09a062ba932b5460410af45cb7b3f94104b Mon Sep 17 00:00:00 2001 From: leventov Date: Fri, 16 Jun 2017 14:17:33 -0500 Subject: [PATCH 2/6] Fix IndexGeneratorJobTest --- .../src/test/java/io/druid/indexer/IndexGeneratorJobTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java index c8e079fbb86f..37c2891b1549 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java @@ -375,7 +375,7 @@ public static Collection constructFeed() final List constructors = Lists.newArrayList(); for (Object[] baseConstructor : baseConstructors) { for (int forceExtendableShardSpecs = 0; forceExtendableShardSpecs < 2 ; forceExtendableShardSpecs++) { - final Object[] fullConstructor = new Object[baseConstructor.length + 2]; + final Object[] fullConstructor = new Object[baseConstructor.length + 1]; System.arraycopy(baseConstructor, 0, fullConstructor, 0, baseConstructor.length); fullConstructor[baseConstructor.length] = forceExtendableShardSpecs == 0; constructors.add(fullConstructor); From d4658470bb313bfcc5d8529a292134c7365e33a1 Mon Sep 17 00:00:00 2001 From: leventov Date: Fri, 16 Jun 2017 18:09:45 -0500 Subject: [PATCH 3/6] Fix parameterized test name in IndexMergerTest --- processing/src/test/java/io/druid/segment/IndexMergerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/test/java/io/druid/segment/IndexMergerTest.java b/processing/src/test/java/io/druid/segment/IndexMergerTest.java index e4e3b934d7ef..d9586dba5f18 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerTest.java @@ -90,7 +90,7 @@ public class IndexMergerTest protected IndexMerger INDEX_MERGER; private final static IndexIO INDEX_IO = TestHelper.getTestIndexIO(); - @Parameterized.Parameters(name = "{index}: useV9={0}, bitmap={1}, metric compression={2}, dimension compression={3}, long encoding={4}") + @Parameterized.Parameters(name = "{index}: bitmap={0}, metric compression={1}, dimension compression={2}, long encoding={3}") public static Collection data() { return Collections2.transform( From 6f67835dcf12164f064ddd6b2476543e86da695f Mon Sep 17 00:00:00 2001 From: leventov Date: Fri, 23 Jun 2017 17:03:54 -0500 Subject: [PATCH 4/6] Remove extra legacy merging stuff --- .../common/index/YeOldePlumberSchool.java | 9 +- .../io/druid/segment/DimensionHandler.java | 22 -- .../druid/segment/DimensionMergerLegacy.java | 69 ------ .../druid/segment/FloatDimensionHandler.java | 15 -- .../segment/FloatDimensionMergerLegacy.java | 89 ------- .../segment/FloatMetricColumnSerializer.java | 91 -------- .../druid/segment/LongDimensionHandler.java | 15 -- .../segment/LongDimensionMergerLegacy.java | 91 -------- .../segment/LongMetricColumnSerializer.java | 95 -------- .../druid/segment/MetricColumnSerializer.java | 33 --- .../druid/segment/StringDimensionHandler.java | 11 - .../segment/StringDimensionMergerLegacy.java | 220 ------------------ .../serde/ComplexMetricColumnSerializer.java | 93 -------- 13 files changed, 2 insertions(+), 851 deletions(-) delete mode 100644 processing/src/main/java/io/druid/segment/DimensionMergerLegacy.java delete mode 100644 processing/src/main/java/io/druid/segment/FloatDimensionMergerLegacy.java delete mode 100644 processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java delete mode 100644 processing/src/main/java/io/druid/segment/LongDimensionMergerLegacy.java delete mode 100644 processing/src/main/java/io/druid/segment/LongMetricColumnSerializer.java delete mode 100644 processing/src/main/java/io/druid/segment/MetricColumnSerializer.java delete mode 100644 processing/src/main/java/io/druid/segment/StringDimensionMergerLegacy.java delete mode 100644 processing/src/main/java/io/druid/segment/serde/ComplexMetricColumnSerializer.java diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java index 24dde83b8f30..08b881f32d52 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java @@ -30,14 +30,12 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; - import io.druid.data.input.Committer; import io.druid.data.input.InputRow; import io.druid.java.util.common.logger.Logger; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMerger; import io.druid.segment.IndexMergerV9; import io.druid.segment.QueryableIndex; import io.druid.segment.SegmentUtils; @@ -115,9 +113,6 @@ public Plumber findPlumber( // Set of spilled segments. Will be merged at the end. final Set spilled = Sets.newHashSet(); - // IndexMerger implementation. - final IndexMerger theIndexMerger = indexMergerV9; - return new Plumber() { @Override @@ -186,7 +181,7 @@ public void finishJob() } fileToUpload = new File(tmpSegmentDir, "merged"); - theIndexMerger.mergeQueryableIndex(indexes, schema.getGranularitySpec().isRollup(), schema.getAggregators(), fileToUpload, config.getIndexSpec()); + indexMergerV9.mergeQueryableIndex(indexes, schema.getGranularitySpec().isRollup(), schema.getAggregators(), fileToUpload, config.getIndexSpec()); } // Map merged segment so we can extract dimensions @@ -231,7 +226,7 @@ private void spillIfSwappable() log.info("Spilling index[%d] with rows[%d] to: %s", indexToPersist.getCount(), rowsToPersist, dirToPersist); try { - theIndexMerger.persist( + indexMergerV9.persist( indexToPersist.getIndex(), dirToPersist, config.getIndexSpec() diff --git a/processing/src/main/java/io/druid/segment/DimensionHandler.java b/processing/src/main/java/io/druid/segment/DimensionHandler.java index 893779211080..f0c24cd80794 100644 --- a/processing/src/main/java/io/druid/segment/DimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/DimensionHandler.java @@ -102,28 +102,6 @@ DimensionMergerV9 makeMerger( ) throws IOException; - /** - * Creates a new DimensionMergerLegacy, a per-dimension object responsible for merging indexes/row data across - * segments and building the on-disk representation of a dimension. For use with IndexMerger only. - * - * See {@link DimensionMergerLegacy} interface for more information. - * - * @param indexSpec Specification object for the index merge - * @param outDir Location to store files generated by the merging process - * @param ioPeon ioPeon object passed in by IndexMerger, manages files created by the merging process - * @param capabilities The ColumnCapabilities of the dimension represented by this DimensionHandler - * @param progress ProgressIndicator used by the merging process - - * @return A new DimensionMergerLegacy object. - */ - DimensionMergerLegacy makeLegacyMerger( - IndexSpec indexSpec, - File outDir, - IOPeon ioPeon, - ColumnCapabilities capabilities, - ProgressIndicator progress - ) throws IOException; - /** * Given an key component representing a single set of row value(s) for this dimension as an Object, * return the length of the key component after appropriate type-casting. diff --git a/processing/src/main/java/io/druid/segment/DimensionMergerLegacy.java b/processing/src/main/java/io/druid/segment/DimensionMergerLegacy.java deleted file mode 100644 index 590f270e87e5..000000000000 --- a/processing/src/main/java/io/druid/segment/DimensionMergerLegacy.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.druid.segment; - -import com.google.common.io.ByteSink; -import com.google.common.io.OutputSupplier; -import io.druid.common.guava.FileOutputSupplier; - -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; - -/** - * Processing related interface - * - * DimensionMerger subclass to be used with the legacy IndexMerger. - * - * NOTE: Remove this class when the legacy IndexMerger is deprecated and removed. - */ -public interface DimensionMergerLegacy extends DimensionMergerV9 -{ - /** - * Write this dimension's value metadata to a file. - * - * @param valueEncodingFile Destination file - * @throws IOException - */ - void writeValueMetadataToFile(FileOutputSupplier valueEncodingFile) throws IOException; - - - /** - * Write this dimension's sequence of row values to a file. - * @param rowValueOut Destination file - * @throws IOException - */ - void writeRowValuesToFile(FileOutputSupplier rowValueOut) throws IOException; - - - /** - * Write this dimension's bitmap and spatial indexes to a file. - * @param invertedOut Destination file for bitmap indexes - * @param spatialOut Destination file for spatial indexes - * @throws IOException - */ - void writeIndexesToFiles( - ByteSink invertedOut, - OutputSupplier spatialOut - ) throws IOException; - - - File makeDimFile() throws IOException; -} diff --git a/processing/src/main/java/io/druid/segment/FloatDimensionHandler.java b/processing/src/main/java/io/druid/segment/FloatDimensionHandler.java index e683bf251461..69318e7c2770 100644 --- a/processing/src/main/java/io/druid/segment/FloatDimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/FloatDimensionHandler.java @@ -66,21 +66,6 @@ public DimensionMergerV9 makeMerger( ); } - @Override - public DimensionMergerLegacy makeLegacyMerger( - IndexSpec indexSpec, File outDir, IOPeon ioPeon, ColumnCapabilities capabilities, ProgressIndicator progress - ) throws IOException - { - return new FloatDimensionMergerLegacy( - dimensionName, - indexSpec, - outDir, - ioPeon, - capabilities, - progress - ); - } - @Override public int getLengthOfEncodedKeyComponent(Float dimVals) { diff --git a/processing/src/main/java/io/druid/segment/FloatDimensionMergerLegacy.java b/processing/src/main/java/io/druid/segment/FloatDimensionMergerLegacy.java deleted file mode 100644 index 02ff4cd8ef98..000000000000 --- a/processing/src/main/java/io/druid/segment/FloatDimensionMergerLegacy.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.druid.segment; - -import com.google.common.io.ByteSink; -import com.google.common.io.OutputSupplier; -import io.druid.common.guava.FileOutputSupplier; -import io.druid.segment.column.ColumnCapabilities; -import io.druid.segment.data.CompressedObjectStrategy; -import io.druid.segment.data.IOPeon; - -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; - -public class FloatDimensionMergerLegacy extends FloatDimensionMergerV9 implements DimensionMergerLegacy -{ - private FloatMetricColumnSerializer serializerV8; - - public FloatDimensionMergerLegacy( - String dimensionName, - IndexSpec indexSpec, - File outDir, - IOPeon ioPeon, - ColumnCapabilities capabilities, - ProgressIndicator progress - ) - { - super(dimensionName, indexSpec, outDir, ioPeon, capabilities, progress); - } - - @Override - protected void setupEncodedValueWriter() throws IOException - { - final CompressedObjectStrategy.CompressionStrategy metCompression = indexSpec.getMetricCompression(); - serializerV8 = new FloatMetricColumnSerializer(dimensionName, outDir, ioPeon, metCompression); - serializerV8.open(); - } - - @Override - public void processMergedRow(Float rowValues) throws IOException - { - serializerV8.serialize(rowValues); - } - - @Override - public void writeValueMetadataToFile(FileOutputSupplier valueEncodingFile) throws IOException - { - // floats have no metadata to write - } - - @Override - public void writeRowValuesToFile(FileOutputSupplier rowValueOut) throws IOException - { - // closing the serializer writes its data to the file - serializerV8.closeFile(rowValueOut.getFile()); - } - - @Override - public void writeIndexesToFiles( - ByteSink invertedOut, OutputSupplier spatialOut - ) throws IOException - { - // floats have no indices to write - } - - @Override - public File makeDimFile() throws IOException - { - return IndexIO.makeNumericDimFile(outDir, dimensionName, IndexIO.BYTE_ORDER); - } -} diff --git a/processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java b/processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java deleted file mode 100644 index 674e805d4548..000000000000 --- a/processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.druid.segment; - -import com.google.common.io.FileWriteMode; -import com.google.common.io.Files; -import io.druid.segment.data.CompressedObjectStrategy; -import io.druid.segment.data.CompressionFactory; -import io.druid.segment.data.FloatSupplierSerializer; -import io.druid.segment.data.IOPeon; - -import java.io.File; -import java.io.IOException; - -/** - */ -public class FloatMetricColumnSerializer implements MetricColumnSerializer -{ - private final String metricName; - private final IOPeon ioPeon; - private final File outDir; - private final CompressedObjectStrategy.CompressionStrategy compression; - - private FloatSupplierSerializer writer; - - public FloatMetricColumnSerializer( - String metricName, - File outDir, - IOPeon ioPeon, - CompressedObjectStrategy.CompressionStrategy compression - ) - { - this.metricName = metricName; - this.ioPeon = ioPeon; - this.outDir = outDir; - this.compression = compression; - } - - @Override - public void open() throws IOException - { - writer = CompressionFactory.getFloatSerializer( - ioPeon, String.format("%s_little", metricName), IndexIO.BYTE_ORDER, compression - ); - - writer.open(); - } - - @Override - public void serialize(Object obj) throws IOException - { - float val = (obj == null) ? 0 : ((Number) obj).floatValue(); - writer.add(val); - } - - @Override - public void close() throws IOException - { - final File outFile = IndexIO.makeMetricFile(outDir, metricName, IndexIO.BYTE_ORDER); - closeFile(outFile); - } - - @Override - public void closeFile(final File outFile) throws IOException - { - outFile.delete(); - MetricHolder.writeFloatMetric( - Files.asByteSink(outFile, FileWriteMode.APPEND), metricName, writer - ); - IndexIO.checkFileSize(outFile); - - writer = null; - } -} diff --git a/processing/src/main/java/io/druid/segment/LongDimensionHandler.java b/processing/src/main/java/io/druid/segment/LongDimensionHandler.java index 7b3ff7685a10..68875b4b13b8 100644 --- a/processing/src/main/java/io/druid/segment/LongDimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/LongDimensionHandler.java @@ -66,21 +66,6 @@ public DimensionMergerV9 makeMerger( ); } - @Override - public DimensionMergerLegacy makeLegacyMerger( - IndexSpec indexSpec, File outDir, IOPeon ioPeon, ColumnCapabilities capabilities, ProgressIndicator progress - ) throws IOException - { - return new LongDimensionMergerLegacy( - dimensionName, - indexSpec, - outDir, - ioPeon, - capabilities, - progress - ); - } - @Override public int getLengthOfEncodedKeyComponent(Long dimVals) { diff --git a/processing/src/main/java/io/druid/segment/LongDimensionMergerLegacy.java b/processing/src/main/java/io/druid/segment/LongDimensionMergerLegacy.java deleted file mode 100644 index 046d774c5f75..000000000000 --- a/processing/src/main/java/io/druid/segment/LongDimensionMergerLegacy.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.druid.segment; - -import com.google.common.io.ByteSink; -import com.google.common.io.OutputSupplier; -import io.druid.common.guava.FileOutputSupplier; -import io.druid.segment.column.ColumnCapabilities; -import io.druid.segment.data.CompressedObjectStrategy; -import io.druid.segment.data.CompressionFactory; -import io.druid.segment.data.IOPeon; - -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; - -public class LongDimensionMergerLegacy extends LongDimensionMergerV9 implements DimensionMergerLegacy -{ - private LongMetricColumnSerializer serializerV8; - - public LongDimensionMergerLegacy( - String dimensionName, - IndexSpec indexSpec, - File outDir, - IOPeon ioPeon, - ColumnCapabilities capabilities, - ProgressIndicator progress - ) - { - super(dimensionName, indexSpec, outDir, ioPeon, capabilities, progress); - } - - @Override - protected void setupEncodedValueWriter() throws IOException - { - final CompressedObjectStrategy.CompressionStrategy metCompression = indexSpec.getMetricCompression(); - final CompressionFactory.LongEncodingStrategy longEncoding = indexSpec.getLongEncoding(); - serializerV8 = new LongMetricColumnSerializer(dimensionName, outDir, ioPeon, metCompression, longEncoding); - serializerV8.open(); - } - - @Override - public void processMergedRow(Long rowValues) throws IOException - { - serializerV8.serialize(rowValues); - } - - @Override - public void writeValueMetadataToFile(FileOutputSupplier valueEncodingFile) throws IOException - { - // longs have no metadata to write - } - - @Override - public void writeRowValuesToFile(FileOutputSupplier rowValueOut) throws IOException - { - // closing the serializer writes its data to the file - serializerV8.closeFile(rowValueOut.getFile()); - } - - @Override - public void writeIndexesToFiles( - ByteSink invertedOut, OutputSupplier spatialOut - ) throws IOException - { - // longs have no indices to write - } - - @Override - public File makeDimFile() throws IOException - { - return IndexIO.makeNumericDimFile(outDir, dimensionName, IndexIO.BYTE_ORDER); - } -} diff --git a/processing/src/main/java/io/druid/segment/LongMetricColumnSerializer.java b/processing/src/main/java/io/druid/segment/LongMetricColumnSerializer.java deleted file mode 100644 index 8fe336ba1f54..000000000000 --- a/processing/src/main/java/io/druid/segment/LongMetricColumnSerializer.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.druid.segment; - -import com.google.common.io.FileWriteMode; -import com.google.common.io.Files; -import io.druid.segment.data.CompressedObjectStrategy; -import io.druid.segment.data.CompressionFactory; -import io.druid.segment.data.IOPeon; -import io.druid.segment.data.LongSupplierSerializer; - -import java.io.File; -import java.io.IOException; - -/** - * Unsafe for concurrent use from multiple threads. - */ -public class LongMetricColumnSerializer implements MetricColumnSerializer -{ - private final String metricName; - private final IOPeon ioPeon; - private final File outDir; - private final CompressedObjectStrategy.CompressionStrategy compression; - private final CompressionFactory.LongEncodingStrategy encoding; - - private LongSupplierSerializer writer; - - public LongMetricColumnSerializer( - String metricName, - File outDir, - IOPeon ioPeon, - CompressedObjectStrategy.CompressionStrategy compression, - CompressionFactory.LongEncodingStrategy encoding - ) - { - this.metricName = metricName; - this.ioPeon = ioPeon; - this.outDir = outDir; - this.compression = compression; - this.encoding = encoding; - } - - @Override - public void open() throws IOException - { - writer = CompressionFactory.getLongSerializer( - ioPeon, String.format("%s_little", metricName), IndexIO.BYTE_ORDER, encoding, compression - ); - - writer.open(); - } - - @Override - public void serialize(Object obj) throws IOException - { - long val = (obj == null) ? 0 : ((Number) obj).longValue(); - writer.add(val); - } - - @Override - public void close() throws IOException - { - final File outFile = IndexIO.makeMetricFile(outDir, metricName, IndexIO.BYTE_ORDER); - closeFile(outFile); - } - - @Override - public void closeFile(final File outFile) throws IOException - { - outFile.delete(); - MetricHolder.writeLongMetric( - Files.asByteSink(outFile, FileWriteMode.APPEND), metricName, writer - ); - IndexIO.checkFileSize(outFile); - - writer = null; - } -} diff --git a/processing/src/main/java/io/druid/segment/MetricColumnSerializer.java b/processing/src/main/java/io/druid/segment/MetricColumnSerializer.java deleted file mode 100644 index 6433b342ca15..000000000000 --- a/processing/src/main/java/io/druid/segment/MetricColumnSerializer.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.druid.segment; - -import java.io.File; -import java.io.IOException; - -/** - */ -public interface MetricColumnSerializer -{ - public void open() throws IOException; - public void serialize(Object aggs) throws IOException; - public void close() throws IOException; - public void closeFile(File outFile) throws IOException; -} diff --git a/processing/src/main/java/io/druid/segment/StringDimensionHandler.java b/processing/src/main/java/io/druid/segment/StringDimensionHandler.java index 14a42b0f74cb..bfe510c6f781 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionHandler.java @@ -202,15 +202,4 @@ public DimensionMergerV9 makeMerger( return new StringDimensionMergerV9(dimensionName, indexSpec, outDir, ioPeon, capabilities, progress); } - @Override - public DimensionMergerLegacy makeLegacyMerger( - IndexSpec indexSpec, - File outDir, - IOPeon ioPeon, - ColumnCapabilities capabilities, - ProgressIndicator progress - ) - { - return new StringDimensionMergerLegacy(dimensionName, indexSpec, outDir, ioPeon, capabilities, progress); - } } diff --git a/processing/src/main/java/io/druid/segment/StringDimensionMergerLegacy.java b/processing/src/main/java/io/druid/segment/StringDimensionMergerLegacy.java deleted file mode 100644 index 57a3fd1dfd3e..000000000000 --- a/processing/src/main/java/io/druid/segment/StringDimensionMergerLegacy.java +++ /dev/null @@ -1,220 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.druid.segment; - -import com.google.common.io.ByteSink; -import com.google.common.io.ByteStreams; -import com.google.common.io.Files; -import com.google.common.io.OutputSupplier; -import com.google.common.primitives.Ints; -import io.druid.collections.bitmap.BitmapFactory; -import io.druid.collections.spatial.ImmutableRTree; -import io.druid.collections.spatial.RTree; -import io.druid.collections.spatial.split.LinearGutmanSplitStrategy; -import io.druid.common.guava.FileOutputSupplier; -import io.druid.common.utils.SerializerUtils; -import io.druid.java.util.common.io.Closer; -import io.druid.java.util.common.ByteBufferUtils; -import io.druid.java.util.common.ISE; -import io.druid.java.util.common.logger.Logger; -import io.druid.segment.column.ColumnCapabilities; -import io.druid.segment.data.BitmapSerdeFactory; -import io.druid.segment.data.ByteBufferWriter; -import io.druid.segment.data.GenericIndexed; -import io.druid.segment.data.GenericIndexedWriter; -import io.druid.segment.data.IOPeon; -import io.druid.segment.data.Indexed; -import io.druid.segment.data.IndexedRTree; -import io.druid.segment.data.VSizeIndexedWriter; - -import java.io.Closeable; -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.OutputStream; -import java.nio.IntBuffer; -import java.nio.MappedByteBuffer; -import java.util.List; - -public class StringDimensionMergerLegacy extends StringDimensionMergerV9 implements DimensionMergerLegacy -{ - private static final Logger log = new Logger(StringDimensionMergerLegacy.class); - - private VSizeIndexedWriter encodedValueWriterV8; - private File dictionaryFile; - - public StringDimensionMergerLegacy( - String dimensionName, - IndexSpec indexSpec, - File outDir, - IOPeon ioPeon, - ColumnCapabilities capabilities, - ProgressIndicator progress - ) - { - super(dimensionName, indexSpec, outDir, ioPeon, capabilities, progress); - } - - @Override - protected void setupEncodedValueWriter() throws IOException - { - encodedValueWriterV8 = new VSizeIndexedWriter(ioPeon, dimensionName, cardinality); - encodedValueWriterV8.open(); - } - - @Override - protected void processMergedRowHelper(int[] vals) throws IOException - { - List listToWrite = (vals == null) - ? null - : Ints.asList(vals); - encodedValueWriterV8.add(listToWrite); - } - - @Override - public void writeIndexes(List segmentRowNumConversions, Closer closer) throws IOException - { - final SerializerUtils serializerUtils = new SerializerUtils(); - long dimStartTime = System.currentTimeMillis(); - final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory(); - - String bmpFilename = String.format("%s.inverted", dimensionName); - bitmapWriter = new GenericIndexedWriter<>( - ioPeon, - bmpFilename, - bitmapSerdeFactory.getObjectStrategy() - ); - bitmapWriter.open(); - - final MappedByteBuffer dimValsMapped = Files.map(dictionaryFile); - closer.register(new Closeable() - { - @Override - public void close() throws IOException - { - ByteBufferUtils.unmap(dimValsMapped); - } - }); - - if (!dimensionName.equals(serializerUtils.readString(dimValsMapped))) { - throw new ISE("dimensions[%s] didn't equate!? This is a major WTF moment.", dimensionName); - } - Indexed dimVals = GenericIndexed.read(dimValsMapped, GenericIndexed.STRING_STRATEGY); - log.info("Starting dimension[%s] with cardinality[%,d]", dimensionName, dimVals.size()); - - - final BitmapFactory bmpFactory = bitmapSerdeFactory.getBitmapFactory(); - - RTree tree = null; - spatialWriter = null; - boolean hasSpatial = capabilities.hasSpatialIndexes(); - if (hasSpatial) { - String spatialFilename = String.format("%s.spatial", dimensionName); - spatialWriter = new ByteBufferWriter<>( - ioPeon, spatialFilename, new IndexedRTree.ImmutableRTreeObjectStrategy(bmpFactory) - ); - spatialWriter.open(); - tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50, bmpFactory), bmpFactory); - } - - IndexSeeker[] dictIdSeeker = toIndexSeekers(adapters, dimConversions, dimensionName); - - //Iterate all dim values's dictionary id in ascending order which in line with dim values's compare result. - for (int dictId = 0; dictId < dimVals.size(); dictId++) { - progress.progress(); - mergeBitmaps( - segmentRowNumConversions, - dimVals, - bmpFactory, - tree, - hasSpatial, - dictIdSeeker, - dictId, - adapters, - dimensionName, - nullRowsBitmap, - bitmapWriter - ); - } - - log.info("Completed dimension[%s] in %,d millis.", dimensionName, System.currentTimeMillis() - dimStartTime); - - if (hasSpatial) { - spatialWriter.write(ImmutableRTree.newImmutableFromMutable(tree)); - - } - } - - @Override - public void writeValueMetadataToFile(final FileOutputSupplier valueEncodingFile) throws IOException - { - final SerializerUtils serializerUtils = new SerializerUtils(); - - dictionaryWriter.close(); - serializerUtils.writeString(valueEncodingFile, dimensionName); - ByteStreams.copy(dictionaryWriter.combineStreams(), valueEncodingFile); - - // save this File reference, we will read from it later when building bitmap/spatial indexes - dictionaryFile = valueEncodingFile.getFile(); - } - - @Override - public void writeRowValuesToFile(FileOutputSupplier rowValueFile) throws IOException - { - encodedValueWriterV8.close(); - ByteStreams.copy(encodedValueWriterV8.combineStreams(), rowValueFile); - } - - @Override - public void writeIndexesToFiles( - final ByteSink invertedIndexFile, - final OutputSupplier spatialIndexFile - ) throws IOException - { - final SerializerUtils serializerUtils = new SerializerUtils(); - final OutputSupplier invertedIndexOutputSupplier = new OutputSupplier() - { - @Override - public OutputStream getOutput() throws IOException - { - return invertedIndexFile.openStream(); - } - }; - - bitmapWriter.close(); - serializerUtils.writeString(invertedIndexOutputSupplier, dimensionName); - ByteStreams.copy(bitmapWriter.combineStreams(), invertedIndexOutputSupplier); - - - if (capabilities.hasSpatialIndexes()) { - spatialWriter.close(); - serializerUtils.writeString(spatialIndexFile, dimensionName); - ByteStreams.copy(spatialWriter.combineStreams(), spatialIndexFile); - } - } - - @Override - public File makeDimFile() throws IOException - { - return IndexIO.makeDimFile(outDir, dimensionName); - } -} - - diff --git a/processing/src/main/java/io/druid/segment/serde/ComplexMetricColumnSerializer.java b/processing/src/main/java/io/druid/segment/serde/ComplexMetricColumnSerializer.java deleted file mode 100644 index f67ead1e79d9..000000000000 --- a/processing/src/main/java/io/druid/segment/serde/ComplexMetricColumnSerializer.java +++ /dev/null @@ -1,93 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.druid.segment.serde; - -import com.google.common.io.Files; -import io.druid.segment.IndexIO; -import io.druid.segment.MetricColumnSerializer; -import io.druid.segment.MetricHolder; -import io.druid.segment.data.GenericIndexedWriter; -import io.druid.segment.data.IOPeon; - -import java.io.File; -import java.io.IOException; - -/** - */ -public class ComplexMetricColumnSerializer implements MetricColumnSerializer -{ - private final String metricName; - private final ComplexMetricSerde serde; - private final IOPeon ioPeon; - private final File outDir; - - private GenericIndexedWriter writer; - - public ComplexMetricColumnSerializer( - String metricName, - File outDir, - IOPeon ioPeon, - ComplexMetricSerde serde - ) - { - this.metricName = metricName; - this.serde = serde; - this.ioPeon = ioPeon; - this.outDir = outDir; - } - - @SuppressWarnings(value = "unchecked") - @Override - public void open() throws IOException - { - writer = new GenericIndexedWriter( - ioPeon, String.format("%s_%s", metricName, outDir.getName()), serde.getObjectStrategy() - ); - - writer.open(); - } - - @Override - public void serialize(Object agg) throws IOException - { - writer.write(agg); - } - - @Override - public void close() throws IOException - { - writer.close(); - - final File outFile = IndexIO.makeMetricFile(outDir, metricName, IndexIO.BYTE_ORDER); - closeFile(outFile); - } - - @Override - public void closeFile(final File outFile) throws IOException - { - outFile.delete(); - MetricHolder.writeComplexMetric( - Files.newOutputStreamSupplier(outFile, true), metricName, serde.getTypeName(), writer - ); - IndexIO.checkFileSize(outFile); - - writer = null; - } -} From 0c07d0d17069665e9ddf3715297a1d51bdf1264a Mon Sep 17 00:00:00 2001 From: leventov Date: Fri, 23 Jun 2017 17:14:43 -0500 Subject: [PATCH 5/6] Remove legacy serializer builders --- .../segment/serde/ComplexColumnPartSerde.java | 44 ----- .../DictionaryEncodedColumnPartSerde.java | 158 ------------------ .../serde/FloatGenericColumnPartSerde.java | 43 ----- .../serde/LongGenericColumnPartSerde.java | 43 ----- 4 files changed, 288 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSerde.java index 88dbcea8918d..01447cb83bd3 100644 --- a/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSerde.java @@ -25,7 +25,6 @@ import io.druid.segment.GenericColumnSerializer; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnConfig; -import io.druid.segment.data.GenericIndexed; import java.io.IOException; import java.nio.ByteBuffer; @@ -58,11 +57,6 @@ public static SerializerBuilder serializerBuilder() return new SerializerBuilder(); } - public static LegacySerializerBuilder legacySerializerBuilder() - { - return new LegacySerializerBuilder(); - } - @JsonProperty public String getTypeName() { @@ -127,42 +121,4 @@ public void write(WritableByteChannel channel, FileSmoosher smoosher) throws IOE ); } } - - public static class LegacySerializerBuilder - { - private String typeName = null; - private GenericIndexed delegate = null; - - public LegacySerializerBuilder withTypeName(final String typeName) - { - this.typeName = typeName; - return this; - } - - public LegacySerializerBuilder withDelegate(final GenericIndexed delegate) - { - this.delegate = delegate; - return this; - } - - public ComplexColumnPartSerde build() - { - return new ComplexColumnPartSerde( - typeName, new Serializer() - { - @Override - public long numBytes() - { - return delegate.getSerializedSize(); - } - - @Override - public void write(WritableByteChannel channel, FileSmoosher smoosher) throws IOException - { - delegate.writeToChannel(channel); - } - } - ); - } - } } diff --git a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java index f3eff916ce97..72b031c107ca 100644 --- a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java @@ -251,164 +251,6 @@ public void write(WritableByteChannel channel, FileSmoosher smoosher) throws IOE } } - public static LegacySerializerBuilder legacySerializerBuilder() - { - return new LegacySerializerBuilder(); - } - - public static class LegacySerializerBuilder - { - private VERSION version = null; - private int flags = NO_FLAGS; - private GenericIndexed dictionary = null; - private WritableSupplier singleValuedColumn = null; - private WritableSupplier> multiValuedColumn = null; - private BitmapSerdeFactory bitmapSerdeFactory = null; - private GenericIndexed bitmaps = null; - private ImmutableRTree spatialIndex = null; - private ByteOrder byteOrder = null; - - private LegacySerializerBuilder() - { - } - - public LegacySerializerBuilder withDictionary(GenericIndexed dictionary) - { - this.dictionary = dictionary; - return this; - } - - public LegacySerializerBuilder withBitmapSerdeFactory(BitmapSerdeFactory bitmapSerdeFactory) - { - this.bitmapSerdeFactory = bitmapSerdeFactory; - return this; - } - - public LegacySerializerBuilder withBitmaps(GenericIndexed bitmaps) - { - this.bitmaps = bitmaps; - return this; - } - - public LegacySerializerBuilder withSpatialIndex(ImmutableRTree spatialIndex) - { - this.spatialIndex = spatialIndex; - return this; - } - - public LegacySerializerBuilder withByteOrder(ByteOrder byteOrder) - { - this.byteOrder = byteOrder; - return this; - } - - public LegacySerializerBuilder withSingleValuedColumn(VSizeIndexedInts singleValuedColumn) - { - Preconditions.checkState(multiValuedColumn == null, "Cannot set both singleValuedColumn and multiValuedColumn"); - this.version = VERSION.UNCOMPRESSED_SINGLE_VALUE; - this.singleValuedColumn = singleValuedColumn.asWritableSupplier(); - return this; - } - - public LegacySerializerBuilder withSingleValuedColumn(CompressedVSizeIntsIndexedSupplier singleValuedColumn) - { - Preconditions.checkState(multiValuedColumn == null, "Cannot set both singleValuedColumn and multiValuedColumn"); - this.version = VERSION.COMPRESSED; - this.singleValuedColumn = singleValuedColumn; - return this; - } - - public LegacySerializerBuilder withMultiValuedColumn(VSizeIndexed multiValuedColumn) - { - Preconditions.checkState(singleValuedColumn == null, "Cannot set both multiValuedColumn and singleValuedColumn"); - this.version = VERSION.UNCOMPRESSED_MULTI_VALUE; - this.flags |= Feature.MULTI_VALUE.getMask(); - this.multiValuedColumn = multiValuedColumn.asWritableSupplier(); - return this; - } - - public LegacySerializerBuilder withMultiValuedColumn(CompressedVSizeIndexedSupplier multiValuedColumn) - { - Preconditions.checkState(singleValuedColumn == null, "Cannot set both singleValuedColumn and multiValuedColumn"); - this.version = VERSION.COMPRESSED; - this.flags |= Feature.MULTI_VALUE.getMask(); - this.multiValuedColumn = multiValuedColumn; - return this; - } - - public DictionaryEncodedColumnPartSerde build() - { - Preconditions.checkArgument( - singleValuedColumn != null ^ multiValuedColumn != null, - "Exactly one of singleValCol[%s] or multiValCol[%s] must be set", - singleValuedColumn, multiValuedColumn - ); - - return new DictionaryEncodedColumnPartSerde( - byteOrder, - bitmapSerdeFactory, - new Serializer() - { - @Override - public long numBytes() - { - long size = 1 + // version - (version.compareTo(VERSION.COMPRESSED) >= 0 ? Ints.BYTES : 0);// flag if version >= compressed - - size += dictionary.getSerializedSize(); - - if (Feature.MULTI_VALUE.isSet(flags)) { - size += multiValuedColumn.getSerializedSize(); - } else { - size += singleValuedColumn.getSerializedSize(); - } - - size += bitmaps.getSerializedSize(); - if (spatialIndex != null) { - size += spatialIndex.size() + Ints.BYTES; - } - return size; - } - - @Override - public void write(WritableByteChannel channel, FileSmoosher smoosher) throws IOException - { - channel.write(ByteBuffer.wrap(new byte[]{version.asByte()})); - if (version.compareTo(VERSION.COMPRESSED) >= 0) { - channel.write(ByteBuffer.wrap(Ints.toByteArray(flags))); - } - - if (dictionary != null) { - dictionary.writeToChannel(channel); - } - - if (Feature.MULTI_VALUE.isSet(flags)) { - if (multiValuedColumn != null) { - multiValuedColumn.writeToChannel(channel); - } - } else { - if (singleValuedColumn != null) { - singleValuedColumn.writeToChannel(channel); - } - } - - if (bitmaps != null) { - bitmaps.writeToChannel(channel); - } - - if (spatialIndex != null) { - ByteBufferSerializer.writeToChannel( - spatialIndex, - new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapSerdeFactory.getBitmapFactory()), - channel - ); - } - } - } - ); - } - } - @Override public Serializer getSerializer() { diff --git a/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java index 4e693dc671e8..fe2ca9705fe4 100644 --- a/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java @@ -104,49 +104,6 @@ public void write(WritableByteChannel channel, FileSmoosher fileSmoosher) throws } } - public static LegacySerializerBuilder legacySerializerBuilder() - { - return new LegacySerializerBuilder(); - } - - public static class LegacySerializerBuilder - { - private ByteOrder byteOrder = null; - private CompressedFloatsIndexedSupplier delegate = null; - - public LegacySerializerBuilder withByteOrder(final ByteOrder byteOrder) - { - this.byteOrder = byteOrder; - return this; - } - - public LegacySerializerBuilder withDelegate(final CompressedFloatsIndexedSupplier delegate) - { - this.delegate = delegate; - return this; - } - - public FloatGenericColumnPartSerde build() - { - return new FloatGenericColumnPartSerde( - byteOrder, new Serializer() - { - @Override - public long numBytes() - { - return delegate.getSerializedSize(); - } - - @Override - public void write(WritableByteChannel channel, FileSmoosher smoosher) throws IOException - { - delegate.writeToChannel(channel); - } - } - ); - } - } - @Override public Serializer getSerializer() { diff --git a/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java index d8eeff9ae738..e5f943be0bb5 100644 --- a/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java @@ -104,49 +104,6 @@ public void write(WritableByteChannel channel, FileSmoosher smoosher) throws IOE } } - public static LegacySerializerBuilder legacySerializerBuilder() - { - return new LegacySerializerBuilder(); - } - - public static class LegacySerializerBuilder - { - private ByteOrder byteOrder = null; - private CompressedLongsIndexedSupplier delegate = null; - - public LegacySerializerBuilder withByteOrder(final ByteOrder byteOrder) - { - this.byteOrder = byteOrder; - return this; - } - - public LegacySerializerBuilder withDelegate(final CompressedLongsIndexedSupplier delegate) - { - this.delegate = delegate; - return this; - } - - public LongGenericColumnPartSerde build() - { - return new LongGenericColumnPartSerde( - byteOrder, new Serializer() - { - @Override - public long numBytes() - { - return delegate.getSerializedSize(); - } - - @Override - public void write(WritableByteChannel channel, FileSmoosher smoosher) throws IOException - { - delegate.writeToChannel(channel); - } - } - ); - } - } - @Override public Serializer getSerializer() { From c954509ca8b9e53c64d5667e84200871e197e773 Mon Sep 17 00:00:00 2001 From: leventov Date: Mon, 26 Jun 2017 12:06:31 -0500 Subject: [PATCH 6/6] Remove ConciseBitmapIndexMergerTest and RoaringBitmapIndexMergerTest --- .../segment/ConciseBitmapIndexMergerTest.java | 40 ------------------- .../segment/RoaringBitmapIndexMergerTest.java | 40 ------------------- 2 files changed, 80 deletions(-) delete mode 100644 processing/src/test/java/io/druid/segment/ConciseBitmapIndexMergerTest.java delete mode 100644 processing/src/test/java/io/druid/segment/RoaringBitmapIndexMergerTest.java diff --git a/processing/src/test/java/io/druid/segment/ConciseBitmapIndexMergerTest.java b/processing/src/test/java/io/druid/segment/ConciseBitmapIndexMergerTest.java deleted file mode 100644 index 538ef69953d4..000000000000 --- a/processing/src/test/java/io/druid/segment/ConciseBitmapIndexMergerTest.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.druid.segment; - -import io.druid.segment.data.CompressedObjectStrategy.CompressionStrategy; -import io.druid.segment.data.CompressionFactory.LongEncodingStrategy; -import io.druid.segment.data.ConciseBitmapSerdeFactory; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class ConciseBitmapIndexMergerTest extends IndexMergerTestBase -{ - public ConciseBitmapIndexMergerTest( - CompressionStrategy compressionStrategy, - CompressionStrategy dimCompressionStrategy, - LongEncodingStrategy longEncodingStrategy - ) - { - super(new ConciseBitmapSerdeFactory(), compressionStrategy, dimCompressionStrategy, longEncodingStrategy); - indexMerger = TestHelper.getTestIndexMergerV9(); - } -} diff --git a/processing/src/test/java/io/druid/segment/RoaringBitmapIndexMergerTest.java b/processing/src/test/java/io/druid/segment/RoaringBitmapIndexMergerTest.java deleted file mode 100644 index e58601f45098..000000000000 --- a/processing/src/test/java/io/druid/segment/RoaringBitmapIndexMergerTest.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.druid.segment; - -import io.druid.segment.data.CompressedObjectStrategy.CompressionStrategy; -import io.druid.segment.data.CompressionFactory.LongEncodingStrategy; -import io.druid.segment.data.RoaringBitmapSerdeFactory; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class RoaringBitmapIndexMergerTest extends IndexMergerTestBase -{ - public RoaringBitmapIndexMergerTest( - CompressionStrategy compressionStrategy, - CompressionStrategy dimCompressionStrategy, - LongEncodingStrategy longEncodingStrategy - ) - { - super(new RoaringBitmapSerdeFactory(null), compressionStrategy, dimCompressionStrategy, longEncodingStrategy); - indexMerger = TestHelper.getTestIndexMergerV9(); - } -}