Merge pull request #2138 from KurtYoung/feature-build-v9

build v9 directly
This commit is contained in:
Fangjin Yang 2016-01-16 13:35:46 -06:00
commit f6a1a4ae20
80 changed files with 6377 additions and 537 deletions

View File

@ -104,7 +104,8 @@ The spec\_file is a path to a file that contains JSON and an example looks like:
"ignoreInvalidRows" : false,
"jobProperties" : { },
"combineText" : false,
"rowFlushBoundary" : 300000
"rowFlushBoundary" : 300000,
"buildV9Directly" : false
}
}
```
@ -205,6 +206,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon
|ignoreInvalidRows|Boolean|Ignore rows found to have problems.|no (default == false)|
|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.|no (default == null)|
|buildV9Directly|Boolean|Whether to build v9 index directly instead of building v8 index and convert it to v9 format|no (default = false)|
### Partitioning specification

View File

@ -142,6 +142,7 @@ 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 v9 index directly instead of building v8 index and convert it to v9 format|no (default = false)|
#### Rejection Policy

View File

@ -25,8 +25,6 @@ import io.druid.data.input.InputRow;
import io.druid.segment.column.ColumnBuilder;
import io.druid.segment.data.GenericIndexed;
import io.druid.segment.data.ObjectStrategy;
import io.druid.segment.serde.ColumnPartSerde;
import io.druid.segment.serde.ComplexColumnPartSerde;
import io.druid.segment.serde.ComplexColumnPartSupplier;
import io.druid.segment.serde.ComplexMetricExtractor;
import io.druid.segment.serde.ComplexMetricSerde;
@ -67,11 +65,10 @@ public class SketchMergeComplexMetricSerde extends ComplexMetricSerde
}
@Override
public ColumnPartSerde deserializeColumn(ByteBuffer buffer, ColumnBuilder builder)
public void deserializeColumn(ByteBuffer buffer, ColumnBuilder builder)
{
GenericIndexed<Sketch> ge = GenericIndexed.read(buffer, strategy);
builder.setComplexColumn(new ComplexColumnPartSupplier(getTypeName(), ge));
return new ComplexColumnPartSerde(ge, getTypeName());
}
@Override

View File

@ -24,8 +24,6 @@ import io.druid.data.input.InputRow;
import io.druid.segment.column.ColumnBuilder;
import io.druid.segment.data.GenericIndexed;
import io.druid.segment.data.ObjectStrategy;
import io.druid.segment.serde.ColumnPartSerde;
import io.druid.segment.serde.ComplexColumnPartSerde;
import io.druid.segment.serde.ComplexColumnPartSupplier;
import io.druid.segment.serde.ComplexMetricExtractor;
import io.druid.segment.serde.ComplexMetricSerde;
@ -92,15 +90,12 @@ public class ApproximateHistogramFoldingSerde extends ComplexMetricSerde
}
@Override
public ColumnPartSerde deserializeColumn(
public void deserializeColumn(
ByteBuffer byteBuffer, ColumnBuilder columnBuilder
)
{
final GenericIndexed column = GenericIndexed.read(byteBuffer, getObjectStrategy());
columnBuilder.setComplexColumn(new ComplexColumnPartSupplier(getTypeName(), column));
return new ComplexColumnPartSerde(column, getTypeName());
}
public ObjectStrategy getObjectStrategy()

View File

@ -51,6 +51,7 @@ import io.druid.indexer.path.PathSpec;
import io.druid.initialization.Initialization;
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;
import io.druid.server.DruidNode;
@ -89,6 +90,7 @@ public class HadoopDruidIndexerConfig
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;
private static final String DEFAULT_WORKING_PATH = "/tmp/druid-indexing";
@ -112,6 +114,7 @@ public class HadoopDruidIndexerConfig
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);
}
public static enum IndexJobCounters
@ -351,6 +354,11 @@ public class HadoopDruidIndexerConfig
return schema.getTuningConfig().getShardSpecs().get(bucket.time).get(bucket.partitionNum);
}
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

View File

@ -42,6 +42,7 @@ public class HadoopTuningConfig implements TuningConfig
private static final IndexSpec DEFAULT_INDEX_SPEC = new IndexSpec();
private static final int DEFAULT_ROW_FLUSH_BOUNDARY = 80000;
private static final boolean DEFAULT_USE_COMBINER = false;
private static final Boolean DEFAULT_BUILD_V9_DIRECTLY = Boolean.FALSE;
public static HadoopTuningConfig makeDefaultTuningConfig()
{
@ -59,7 +60,8 @@ public class HadoopTuningConfig implements TuningConfig
null,
false,
false,
null
null,
DEFAULT_BUILD_V9_DIRECTLY
);
}
@ -76,6 +78,7 @@ public class HadoopTuningConfig implements TuningConfig
private final Map<String, String> jobProperties;
private final boolean combineText;
private final boolean useCombiner;
private final Boolean buildV9Directly;
@JsonCreator
public HadoopTuningConfig(
@ -93,7 +96,8 @@ public class HadoopTuningConfig implements TuningConfig
final @JsonProperty("combineText") boolean combineText,
final @JsonProperty("useCombiner") Boolean useCombiner,
// See https://github.com/druid-io/druid/pull/1922
final @JsonProperty("rowFlushBoundary") Integer maxRowsInMemoryCOMPAT
final @JsonProperty("rowFlushBoundary") Integer maxRowsInMemoryCOMPAT,
final @JsonProperty("buildV9Directly") Boolean buildV9Directly
)
{
this.workingPath = workingPath;
@ -111,6 +115,7 @@ public class HadoopTuningConfig implements TuningConfig
: ImmutableMap.copyOf(jobProperties));
this.combineText = combineText;
this.useCombiner = useCombiner == null ? DEFAULT_USE_COMBINER : useCombiner.booleanValue();
this.buildV9Directly = buildV9Directly == null ? DEFAULT_BUILD_V9_DIRECTLY : buildV9Directly;
}
@JsonProperty
@ -191,6 +196,11 @@ public class HadoopTuningConfig implements TuningConfig
return useCombiner;
}
@JsonProperty
public Boolean getBuildV9Directly() {
return buildV9Directly;
}
public HadoopTuningConfig withWorkingPath(String path)
{
return new HadoopTuningConfig(
@ -207,7 +217,8 @@ public class HadoopTuningConfig implements TuningConfig
jobProperties,
combineText,
useCombiner,
null
null,
buildV9Directly
);
}
@ -227,7 +238,8 @@ public class HadoopTuningConfig implements TuningConfig
jobProperties,
combineText,
useCombiner,
null
null,
buildV9Directly
);
}
@ -247,7 +259,8 @@ public class HadoopTuningConfig implements TuningConfig
jobProperties,
combineText,
useCombiner,
null
null,
buildV9Directly
);
}
}

View File

@ -454,6 +454,7 @@ public class IndexGeneratorJob implements Jobby
@Override
public void progress()
{
super.progress();
context.progress();
}
};
@ -466,9 +467,15 @@ public class IndexGeneratorJob implements Jobby
final ProgressIndicator progressIndicator
) throws IOException
{
return HadoopDruidIndexerConfig.INDEX_MERGER.persist(
index, interval, file, null, config.getIndexSpec(), progressIndicator
);
if (config.isBuildV9Directly()) {
return HadoopDruidIndexerConfig.INDEX_MERGER_V9.persist(
index, interval, file, null, config.getIndexSpec(), progressIndicator
);
} else {
return HadoopDruidIndexerConfig.INDEX_MERGER.persist(
index, interval, file, null, config.getIndexSpec(), progressIndicator
);
}
}
protected File mergeQueryableIndex(
@ -478,9 +485,15 @@ public class IndexGeneratorJob implements Jobby
ProgressIndicator progressIndicator
) throws IOException
{
return HadoopDruidIndexerConfig.INDEX_MERGER.mergeQueryableIndex(
indexes, aggs, file, config.getIndexSpec(), progressIndicator
);
if (config.isBuildV9Directly()) {
return HadoopDruidIndexerConfig.INDEX_MERGER_V9.mergeQueryableIndex(
indexes, aggs, file, config.getIndexSpec(), progressIndicator
);
} else {
return HadoopDruidIndexerConfig.INDEX_MERGER.mergeQueryableIndex(
indexes, aggs, file, config.getIndexSpec(), progressIndicator
);
}
}
@Override
@ -586,7 +599,7 @@ public class IndexGeneratorJob implements Jobby
indexes.add(HadoopDruidIndexerConfig.INDEX_IO.loadIndex(file));
}
mergedBase = mergeQueryableIndex(
indexes, aggregators, new File(baseFlushFile, "merged"), progressIndicator
indexes, aggregators, new File(baseFlushFile, "merged"), progressIndicator
);
}
final FileSystem outputFS = new Path(config.getSchema().getIOConfig().getSegmentOutputPath())

View File

@ -381,6 +381,7 @@ public class BatchDeltaIngestionTest
null,
false,
false,
null,
null
)
)

View File

@ -160,6 +160,7 @@ public class DetermineHashedPartitionsJobTest
null,
false,
false,
null,
null
)
);

View File

@ -264,6 +264,7 @@ public class DeterminePartitionsJobTest
null,
false,
false,
null,
null
)
)

View File

@ -207,6 +207,7 @@ public class HadoopDruidIndexerConfigTest
null,
false,
false,
null,
null
)
);

View File

@ -53,6 +53,7 @@ public class HadoopTuningConfigTest
null,
true,
true,
null,
null
);

View File

@ -395,6 +395,7 @@ public class IndexGeneratorJobTest
ImmutableMap.of(JobContext.NUM_REDUCES, "0"), //verifies that set num reducers is ignored
false,
useCombiner,
null,
null
)
)

View File

@ -115,6 +115,7 @@ public class JobHelperTest
),
false,
false,
null,
null
)
)

View File

@ -120,7 +120,7 @@ public class GranularityPathSpecTest
jsonMapper
),
new HadoopIOConfig(null, null, null),
new HadoopTuningConfig(null, null, null, null, null, null, false, false, false, false, null, false, false, null)
new HadoopTuningConfig(null, null, null, null, null, null, false, false, false, false, null, false, false, null, null)
);
granularityPathSpec.setDataGranularity(Granularity.HOUR);

View File

@ -201,6 +201,7 @@ public class HadoopConverterJobTest
null,
false,
false,
null,
null
)
)

View File

@ -37,6 +37,7 @@ 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;
import io.druid.segment.loading.DataSegmentMover;
@ -80,7 +81,7 @@ public class TaskToolbox
private final IndexIO indexIO;
private final Cache cache;
private final CacheConfig cacheConfig;
private final IndexMergerV9 indexMergerV9;
public TaskToolbox(
TaskConfig config,
@ -102,7 +103,8 @@ public class TaskToolbox
IndexMerger indexMerger,
IndexIO indexIO,
Cache cache,
CacheConfig cacheConfig
CacheConfig cacheConfig,
IndexMergerV9 indexMergerV9
)
{
this.config = config;
@ -125,6 +127,7 @@ public class TaskToolbox
this.indexIO = Preconditions.checkNotNull(indexIO, "Null IndexIO");
this.cache = cache;
this.cacheConfig = cacheConfig;
this.indexMergerV9 = Preconditions.checkNotNull(indexMergerV9, "Null IndexMergerV9");
}
public TaskConfig getConfig()
@ -247,4 +250,8 @@ public class TaskToolbox
{
return cacheConfig;
}
public IndexMergerV9 getIndexMergerV9() {
return indexMergerV9;
}
}

View File

@ -34,6 +34,7 @@ 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;
import io.druid.segment.loading.DataSegmentMover;
@ -67,6 +68,7 @@ public class TaskToolboxFactory
private final IndexIO indexIO;
private final Cache cache;
private final CacheConfig cacheConfig;
private final IndexMergerV9 indexMergerV9;
@Inject
public TaskToolboxFactory(
@ -87,7 +89,8 @@ public class TaskToolboxFactory
IndexMerger indexMerger,
IndexIO indexIO,
Cache cache,
CacheConfig cacheConfig
CacheConfig cacheConfig,
IndexMergerV9 indexMergerV9
)
{
this.config = config;
@ -108,6 +111,7 @@ public class TaskToolboxFactory
this.indexIO = Preconditions.checkNotNull(indexIO, "Null IndexIO");
this.cache = cache;
this.cacheConfig = cacheConfig;
this.indexMergerV9 = indexMergerV9;
}
public TaskToolbox build(Task task)
@ -133,7 +137,8 @@ public class TaskToolboxFactory
indexMerger,
indexIO,
cache,
cacheConfig
cacheConfig,
indexMergerV9
);
}
}

View File

@ -48,6 +48,7 @@ import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.index.YeOldePlumberSchool;
import io.druid.query.aggregation.hyperloglog.HyperLogLogCollector;
import io.druid.segment.IndexMerger;
import io.druid.segment.IndexSpec;
import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.IOConfig;
@ -134,7 +135,8 @@ public class IndexTask extends AbstractFixedIntervalTask
null,
null,
shardSpec,
indexSpec
indexSpec,
null
);
}
@ -353,12 +355,15 @@ public class IndexTask extends AbstractFixedIntervalTask
final FireDepartmentMetrics metrics = new FireDepartmentMetrics();
final Firehose firehose = firehoseFactory.connect(ingestionSchema.getDataSchema().getParser());
final Supplier<Committer> committerSupplier = Committers.supplierFromFirehose(firehose);
final IndexMerger indexMerger = ingestionSchema.getTuningConfig().getBuildV9Directly()
? toolbox.getIndexMergerV9()
: toolbox.getIndexMerger();
final Plumber plumber = new YeOldePlumberSchool(
interval,
version,
wrappedDataSegmentPusher,
tmpDir,
toolbox.getIndexMerger(),
indexMerger,
toolbox.getIndexIO()
).findPlumber(
schema,
@ -434,7 +439,7 @@ public class IndexTask extends AbstractFixedIntervalTask
this.dataSchema = dataSchema;
this.ioConfig = ioConfig;
this.tuningConfig = tuningConfig == null ? new IndexTuningConfig(0, 0, null, null) : tuningConfig;
this.tuningConfig = tuningConfig == null ? new IndexTuningConfig(0, 0, null, null, null) : tuningConfig;
}
@Override
@ -485,18 +490,21 @@ public class IndexTask extends AbstractFixedIntervalTask
private static final int DEFAULT_TARGET_PARTITION_SIZE = 5000000;
private static final int DEFAULT_ROW_FLUSH_BOUNDARY = 500000;
private static final IndexSpec DEFAULT_INDEX_SPEC = new IndexSpec();
private static final Boolean DEFAULT_BUILD_V9_DIRECTLY = Boolean.FALSE;
private final int targetPartitionSize;
private final int rowFlushBoundary;
private final int numShards;
private final IndexSpec indexSpec;
private final Boolean buildV9Directly;
@JsonCreator
public IndexTuningConfig(
@JsonProperty("targetPartitionSize") int targetPartitionSize,
@JsonProperty("rowFlushBoundary") int rowFlushBoundary,
@JsonProperty("numShards") @Nullable Integer numShards,
@JsonProperty("indexSpec") @Nullable IndexSpec indexSpec
@JsonProperty("indexSpec") @Nullable IndexSpec indexSpec,
@JsonProperty("buildV9Directly") Boolean buildV9Directly
)
{
this.targetPartitionSize = targetPartitionSize == 0 ? DEFAULT_TARGET_PARTITION_SIZE : targetPartitionSize;
@ -508,6 +516,7 @@ public class IndexTask extends AbstractFixedIntervalTask
this.targetPartitionSize == -1 || this.numShards == -1,
"targetPartitionsSize and shardCount both cannot be set"
);
this.buildV9Directly = buildV9Directly == null ? DEFAULT_BUILD_V9_DIRECTLY : buildV9Directly;
}
@JsonProperty
@ -533,5 +542,11 @@ public class IndexTask extends AbstractFixedIntervalTask
{
return indexSpec;
}
@JsonProperty
public Boolean getBuildV9Directly()
{
return buildV9Directly;
}
}
}

View File

@ -47,6 +47,7 @@ import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryRunnerFactoryConglomerate;
import io.druid.query.QueryToolChest;
import io.druid.segment.IndexMerger;
import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.RealtimeIOConfig;
import io.druid.segment.indexing.RealtimeTuningConfig;
@ -286,6 +287,9 @@ public class RealtimeIndexTask extends AbstractTask
);
this.queryRunnerFactoryConglomerate = toolbox.getQueryRunnerFactoryConglomerate();
IndexMerger indexMerger = spec.getTuningConfig().getBuildV9Directly()
? toolbox.getIndexMergerV9()
: toolbox.getIndexMerger();
// NOTE: This pusher selects path based purely on global configuration and the DataSegment, which means
// NOTE: that redundant realtime tasks will upload to the same location. This can cause index.zip and
// NOTE: descriptor.json to mismatch, or it can cause historical nodes to load different instances of the
@ -298,7 +302,7 @@ public class RealtimeIndexTask extends AbstractTask
segmentPublisher,
toolbox.getSegmentHandoffNotifierFactory(),
toolbox.getQueryExecutorService(),
toolbox.getIndexMerger(),
indexMerger,
toolbox.getIndexIO(),
toolbox.getCache(),
toolbox.getCacheConfig(),

View File

@ -31,6 +31,7 @@ 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;
import io.druid.segment.loading.DataSegmentMover;
@ -77,6 +78,7 @@ public class TaskToolboxTest
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);
private CacheConfig mockCacheConfig = EasyMock.createMock(CacheConfig.class);
@ -109,7 +111,8 @@ public class TaskToolboxTest
mockIndexMerger,
mockIndexIO,
mockCache,
mockCacheConfig
mockCacheConfig,
mockIndexMergerV9
);
}

View File

@ -28,6 +28,7 @@ import io.druid.guice.ServerModule;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMerger;
import io.druid.segment.IndexMergerV9;
import io.druid.segment.column.ColumnConfig;
import java.util.List;
@ -39,6 +40,7 @@ public class TestUtils
{
private final ObjectMapper jsonMapper;
private final IndexMerger indexMerger;
private final IndexMergerV9 indexMergerV9;
private final IndexIO indexIO;
public TestUtils()
@ -56,6 +58,7 @@ public class TestUtils
}
);
indexMerger = new IndexMerger(jsonMapper, indexIO);
indexMergerV9 = new IndexMergerV9(jsonMapper, indexIO);
final List<? extends Module> list = new ServerModule().getJacksonModules();
for (Module module : list) {
@ -80,6 +83,10 @@ public class TestUtils
return indexMerger;
}
public IndexMergerV9 getTestIndexMergerV9() {
return indexMergerV9;
}
public IndexIO getTestIndexIO()
{
return indexIO;

View File

@ -39,6 +39,7 @@ 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;
import io.druid.segment.indexing.RealtimeTuningConfig;
@ -70,6 +71,7 @@ public class IndexTaskTest
private final IndexSpec indexSpec;
private final ObjectMapper jsonMapper;
private IndexMerger indexMerger;
private IndexMergerV9 indexMergerV9;
private IndexIO indexIO;
public IndexTaskTest()
@ -78,6 +80,7 @@ public class IndexTaskTest
TestUtils testUtils = new TestUtils();
jsonMapper = testUtils.getTestObjectMapper();
indexMerger = testUtils.getTestIndexMerger();
indexMergerV9 = testUtils.getTestIndexMergerV9();
indexIO = testUtils.getTestIndexIO();
}
@ -140,7 +143,8 @@ public class IndexTaskTest
2,
0,
null,
indexSpec
indexSpec,
null
)
),
jsonMapper,
@ -252,7 +256,7 @@ public class IndexTaskTest
return segment;
}
}, null, null, null, null, null, null, null, null, null, null, temporaryFolder.newFolder(),
indexMerger, indexIO, null, null
indexMerger, indexIO, null, null, indexMergerV9
)
);
@ -332,7 +336,8 @@ public class IndexTaskTest
100,
1000,
null,
new IndexSpec()
new IndexSpec(),
null
);
RealtimeTuningConfig realtimeTuningConfig = IndexTask.convertTuningConfig(
spec,

View File

@ -571,6 +571,7 @@ public class RealtimeIndexTaskTest
null,
null,
null,
null,
null
);
return new RealtimeIndexTask(
@ -715,7 +716,8 @@ public class RealtimeIndexTaskTest
testUtils.getTestIndexMerger(),
testUtils.getTestIndexIO(),
MapCache.create(1024),
new CacheConfig()
new CacheConfig(),
testUtils.getTestIndexMergerV9()
);
return toolboxFactory.build(task);

View File

@ -91,7 +91,7 @@ public class TaskSerdeTest
jsonMapper
),
new IndexTask.IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null)),
new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec)
new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec, null)
),
jsonMapper,
null
@ -132,7 +132,7 @@ public class TaskSerdeTest
jsonMapper
),
new IndexTask.IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null)),
new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec)
new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec, null)
),
jsonMapper,
null
@ -332,7 +332,8 @@ public class TaskSerdeTest
null,
1,
new NoneShardSpec(),
indexSpec
indexSpec,
null
)
),
null

View File

@ -64,6 +64,7 @@ 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;
import io.druid.segment.incremental.OnheapIncrementalIndex;
@ -109,12 +110,14 @@ 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();
}
@ -274,7 +277,8 @@ public class IngestSegmentFirehoseFactoryTest
INDEX_MERGER,
INDEX_IO,
null,
null
null,
INDEX_MERGER_V9
);
Collection<Object[]> values = new LinkedList<>();
for (InputRowParser parser : Arrays.<InputRowParser>asList(

View File

@ -55,6 +55,7 @@ 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;
import io.druid.segment.incremental.IndexSizeExceededException;
@ -111,12 +112,14 @@ public class IngestSegmentFirehoseFactoryTimelineTest
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();
}
public IngestSegmentFirehoseFactoryTimelineTest(
@ -334,7 +337,8 @@ public class IngestSegmentFirehoseFactoryTimelineTest
INDEX_MERGER,
INDEX_IO,
null,
null
null,
INDEX_MERGER_V9
);
final Injector injector = Guice.createInjector(
new Module()

View File

@ -82,6 +82,7 @@ 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;
import io.druid.segment.indexing.RealtimeIOConfig;
@ -132,12 +133,14 @@ 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;
static {
TestUtils testUtils = new TestUtils();
MAPPER = testUtils.getTestObjectMapper();
INDEX_MERGER = testUtils.getTestIndexMerger();
INDEX_MERGER_V9 = testUtils.getTestIndexMergerV9();
INDEX_IO = testUtils.getTestIndexIO();
}
@ -532,7 +535,8 @@ public class TaskLifecycleTest
INDEX_MERGER,
INDEX_IO,
MapCache.create(0),
FireDepartmentTest.NO_CACHE_CONFIG
FireDepartmentTest.NO_CACHE_CONFIG,
INDEX_MERGER_V9
);
tr = new ThreadPoolTaskRunner(tb, taskConfig, emitter);
tq = new TaskQueue(tqc, ts, tr, tac, tl, emitter);
@ -565,7 +569,7 @@ public class TaskLifecycleTest
mapper
),
new IndexTask.IndexIOConfig(new MockFirehoseFactory(false)),
new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec)
new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec, null)
),
mapper,
null
@ -623,7 +627,7 @@ public class TaskLifecycleTest
mapper
),
new IndexTask.IndexIOConfig(new MockExceptionalFirehoseFactory()),
new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec)
new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec, null)
),
mapper,
null
@ -969,7 +973,7 @@ public class TaskLifecycleTest
mapper
),
new IndexTask.IndexIOConfig(new MockFirehoseFactory(false)),
new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec)
new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec, null)
),
mapper,
null
@ -1080,6 +1084,7 @@ public class TaskLifecycleTest
null,
null,
null,
null,
null
);
FireDepartment fireDepartment = new FireDepartment(dataSchema, realtimeIOConfig, realtimeTuningConfig);

View File

@ -41,6 +41,7 @@ import io.druid.indexing.overlord.ThreadPoolTaskRunner;
import io.druid.indexing.worker.config.WorkerConfig;
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;
import io.druid.segment.loading.StorageLocationConfig;
@ -80,6 +81,7 @@ public class WorkerTaskMonitorTest
private Worker worker;
private ObjectMapper jsonMapper;
private IndexMerger indexMerger;
private IndexMergerV9 indexMergerV9;
private IndexIO indexIO;
public WorkerTaskMonitorTest()
@ -87,6 +89,7 @@ public class WorkerTaskMonitorTest
TestUtils testUtils = new TestUtils();
jsonMapper = testUtils.getTestObjectMapper();
indexMerger = testUtils.getTestIndexMerger();
indexMergerV9 = testUtils.getTestIndexMergerV9();
indexIO = testUtils.getTestIndexIO();
}
@ -183,7 +186,8 @@ public class WorkerTaskMonitorTest
indexMerger,
indexIO,
null,
null
null,
indexMergerV9
),
taskConfig,
new NoopServiceEmitter()

View File

@ -26,8 +26,6 @@ import io.druid.data.input.InputRow;
import io.druid.segment.column.ColumnBuilder;
import io.druid.segment.data.GenericIndexed;
import io.druid.segment.data.ObjectStrategy;
import io.druid.segment.serde.ColumnPartSerde;
import io.druid.segment.serde.ComplexColumnPartSerde;
import io.druid.segment.serde.ComplexColumnPartSupplier;
import io.druid.segment.serde.ComplexMetricExtractor;
import io.druid.segment.serde.ComplexMetricSerde;
@ -103,15 +101,12 @@ public class HyperUniquesSerde extends ComplexMetricSerde
}
@Override
public ColumnPartSerde deserializeColumn(
public void deserializeColumn(
ByteBuffer byteBuffer, ColumnBuilder columnBuilder
)
{
final GenericIndexed column = GenericIndexed.read(byteBuffer, getObjectStrategy());
columnBuilder.setComplexColumn(new ComplexColumnPartSupplier(getTypeName(), column));
return new ComplexColumnPartSerde(column, getTypeName());
}
@Override

View File

@ -0,0 +1,138 @@
/*
* 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.metamx.common.IAE;
import io.druid.segment.data.CompressedIntsIndexedSupplier;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.CompressedVSizeIntsIndexedSupplier;
import io.druid.segment.data.IndexedInts;
import io.druid.segment.data.IndexedMultivalue;
import io.druid.segment.data.WritableSupplier;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.nio.channels.WritableByteChannel;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
/**
* The format is mostly the same with CompressedVSizeIndexedSupplier(which has version 0x2, so we call it V2),
* the only difference is V3's offsets is not VSize encoded, it's just compressed.
* The reason we provide this is we can streams the data out in the binary format with CompressedVSizeIndexedV3Writer.
* If we want to streams VSizeInts, we must know the max value in the value sets. It's easy to know the max id of
* values(like dimension cardinality while encoding dimension), but difficult to known the max id of offsets.
*/
public class CompressedVSizeIndexedV3Supplier implements WritableSupplier<IndexedMultivalue<IndexedInts>>
{
public static final byte VERSION = 0x3;
private final CompressedIntsIndexedSupplier offsetSupplier;
private final CompressedVSizeIntsIndexedSupplier valueSupplier;
CompressedVSizeIndexedV3Supplier(
CompressedIntsIndexedSupplier offsetSupplier,
CompressedVSizeIntsIndexedSupplier valueSupplier
)
{
this.offsetSupplier = offsetSupplier;
this.valueSupplier = valueSupplier;
}
public static CompressedVSizeIndexedV3Supplier fromByteBuffer(ByteBuffer buffer, ByteOrder order)
{
byte versionFromBuffer = buffer.get();
if (versionFromBuffer == VERSION) {
CompressedIntsIndexedSupplier offsetSupplier = CompressedIntsIndexedSupplier.fromByteBuffer(
buffer,
order
);
CompressedVSizeIntsIndexedSupplier valueSupplier = CompressedVSizeIntsIndexedSupplier.fromByteBuffer(
buffer,
order
);
return new CompressedVSizeIndexedV3Supplier(offsetSupplier, valueSupplier);
}
throw new IAE("Unknown version[%s]", versionFromBuffer);
}
// for test
public static CompressedVSizeIndexedV3Supplier fromIterable(
Iterable<IndexedInts> objectsIterable,
int offsetChunkFactor,
int maxValue,
final ByteOrder byteOrder,
CompressedObjectStrategy.CompressionStrategy compression
)
{
Iterator<IndexedInts> objects = objectsIterable.iterator();
List<Integer> offsetList = new ArrayList<>();
List<Integer> values = new ArrayList<>();
int offset = 0;
while (objects.hasNext()) {
IndexedInts next = objects.next();
offsetList.add(offset);
for (int i = 0; i < next.size(); i++) {
values.add(next.get(i));
}
offset += next.size();
}
offsetList.add(offset);
CompressedIntsIndexedSupplier headerSupplier = CompressedIntsIndexedSupplier.fromList(
offsetList,
offsetChunkFactor,
byteOrder,
compression
);
CompressedVSizeIntsIndexedSupplier valuesSupplier = CompressedVSizeIntsIndexedSupplier.fromList(
values,
maxValue,
CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(maxValue),
byteOrder,
compression
);
return new CompressedVSizeIndexedV3Supplier(headerSupplier, valuesSupplier);
}
@Override
public long getSerializedSize()
{
return 1 + offsetSupplier.getSerializedSize() + valueSupplier.getSerializedSize();
}
@Override
public void writeToChannel(WritableByteChannel channel) throws IOException
{
channel.write(ByteBuffer.wrap(new byte[]{VERSION}));
offsetSupplier.writeToChannel(channel);
valueSupplier.writeToChannel(channel);
}
@Override
public IndexedMultivalue<IndexedInts> get()
{
return new CompressedVSizeIndexedSupplier.CompressedVSizeIndexed(offsetSupplier.get(), valueSupplier.get());
}
}

View File

@ -0,0 +1,97 @@
/*
* 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.CompressedFloatsSupplierSerializer;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.IOPeon;
import java.io.IOException;
import java.nio.ByteOrder;
import java.nio.channels.WritableByteChannel;
public class FloatColumnSerializer implements GenericColumnSerializer
{
public static FloatColumnSerializer create(
IOPeon ioPeon,
String filenameBase,
CompressedObjectStrategy.CompressionStrategy compression
)
{
return new FloatColumnSerializer(ioPeon, filenameBase, IndexIO.BYTE_ORDER, compression);
}
private final IOPeon ioPeon;
private final String filenameBase;
private final ByteOrder byteOrder;
private final CompressedObjectStrategy.CompressionStrategy compression;
private CompressedFloatsSupplierSerializer writer;
public FloatColumnSerializer(
IOPeon ioPeon,
String filenameBase,
ByteOrder byteOrder,
CompressedObjectStrategy.CompressionStrategy compression
)
{
this.ioPeon = ioPeon;
this.filenameBase = filenameBase;
this.byteOrder = byteOrder;
this.compression = compression;
}
@Override
public void open() throws IOException
{
writer = CompressedFloatsSupplierSerializer.create(
ioPeon,
String.format("%s.float_column", filenameBase),
byteOrder,
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
{
writer.close();
}
@Override
public long getSerializedSize()
{
return writer.getSerializedSize();
}
@Override
public void writeToChannel(WritableByteChannel channel) throws IOException
{
writer.writeToChannel(channel);
}
}

View File

@ -0,0 +1,35 @@
/*
* 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.Closeable;
import java.io.IOException;
import java.nio.channels.WritableByteChannel;
public interface GenericColumnSerializer extends Closeable
{
public void open() throws IOException;
public void serialize(Object obj) throws IOException;
public long getSerializedSize();
public void writeToChannel(WritableByteChannel channel) throws IOException;
}

View File

@ -255,8 +255,7 @@ public class IndexIO
IndexSpec indexSpec,
boolean forceIfCurrent,
boolean validate
)
throws IOException
) throws IOException
{
final int version = SegmentUtils.getVersionFromDir(toConvert);
switch (version) {
@ -695,8 +694,8 @@ public class IndexIO
final CompressedObjectStrategy.CompressionStrategy compressionStrategy = indexSpec.getDimensionCompressionStrategy();
final DictionaryEncodedColumnPartSerde.Builder columnPartBuilder = DictionaryEncodedColumnPartSerde
.builder()
final DictionaryEncodedColumnPartSerde.LegacySerializerBuilder columnPartBuilder = DictionaryEncodedColumnPartSerde
.legacySerializerBuilder()
.withDictionary(dictionary)
.withBitmapSerdeFactory(bitmapSerdeFactory)
.withBitmaps(bitmaps)
@ -758,11 +757,21 @@ public class IndexIO
switch (holder.getType()) {
case LONG:
builder.setValueType(ValueType.LONG);
builder.addSerde(new LongGenericColumnPartSerde(holder.longType, BYTE_ORDER));
builder.addSerde(
LongGenericColumnPartSerde.legacySerializerBuilder()
.withByteOrder(BYTE_ORDER)
.withDelegate(holder.longType)
.build()
);
break;
case FLOAT:
builder.setValueType(ValueType.FLOAT);
builder.addSerde(new FloatGenericColumnPartSerde(holder.floatType, BYTE_ORDER));
builder.addSerde(
FloatGenericColumnPartSerde.legacySerializerBuilder()
.withByteOrder(BYTE_ORDER)
.withDelegate(holder.floatType)
.build()
);
break;
case COMPLEX:
if (!(holder.complexType instanceof GenericIndexed)) {
@ -770,9 +779,12 @@ public class IndexIO
}
final GenericIndexed column = (GenericIndexed) holder.complexType;
final String complexType = holder.getTypeName();
builder.setValueType(ValueType.COMPLEX);
builder.addSerde(new ComplexColumnPartSerde(column, complexType));
builder.addSerde(
ComplexColumnPartSerde.legacySerializerBuilder()
.withTypeName(complexType)
.withDelegate(column).build()
);
break;
default:
throw new ISE("Unknown type[%s]", holder.getType());
@ -797,8 +809,12 @@ public class IndexIO
final ColumnDescriptor.Builder builder = ColumnDescriptor.builder();
builder.setValueType(ValueType.LONG);
builder.addSerde(new LongGenericColumnPartSerde(timestamps, BYTE_ORDER));
builder.addSerde(
LongGenericColumnPartSerde.legacySerializerBuilder()
.withByteOrder(BYTE_ORDER)
.withDelegate(timestamps)
.build()
);
final ColumnDescriptor serdeficator = builder.build();
ByteArrayOutputStream baos = new ByteArrayOutputStream();

View File

@ -104,12 +104,12 @@ public class IndexMerger
{
private static final Logger log = new Logger(IndexMerger.class);
private static final SerializerUtils serializerUtils = new SerializerUtils();
private static final int INVALID_ROW = -1;
private static final Splitter SPLITTER = Splitter.on(",");
protected static final SerializerUtils serializerUtils = new SerializerUtils();
protected static final int INVALID_ROW = -1;
protected static final Splitter SPLITTER = Splitter.on(",");
private final ObjectMapper mapper;
private final IndexIO indexIO;
protected final ObjectMapper mapper;
protected final IndexIO indexIO;
@Inject
public IndexMerger(
@ -255,7 +255,7 @@ public class IndexMerger
return merge(indexes, metricAggs, outDir, segmentMetadata, indexSpec, new BaseProgressIndicator());
}
private List<String> getLexicographicMergedDimensions(List<IndexableAdapter> indexes)
private static List<String> getLexicographicMergedDimensions(List<IndexableAdapter> indexes)
{
return mergeIndexed(
Lists.transform(
@ -272,7 +272,7 @@ public class IndexMerger
);
}
private List<String> getMergedDimensions(List<IndexableAdapter> indexes)
public static List<String> getMergedDimensions(List<IndexableAdapter> indexes)
{
if (indexes.size() == 0) {
return ImmutableList.of();
@ -280,7 +280,7 @@ public class IndexMerger
Indexed<String> dimOrder = indexes.get(0).getDimensionNames();
for (IndexableAdapter index : indexes) {
Indexed<String> dimOrder2 = index.getDimensionNames();
if(!Iterators.elementsEqual(dimOrder.iterator(), dimOrder2.iterator())) {
if (!Iterators.elementsEqual(dimOrder.iterator(), dimOrder2.iterator())) {
return getLexicographicMergedDimensions(indexes);
}
}
@ -473,7 +473,7 @@ public class IndexMerger
return makeIndexFiles(indexes, outDir, progress, mergedDimensions, mergedMetrics, null, rowMergerFn, indexSpec);
}
private File makeIndexFiles(
protected File makeIndexFiles(
final List<IndexableAdapter> indexes,
final File outDir,
final ProgressIndicator progress,
@ -964,7 +964,7 @@ public class IndexMerger
return outDir;
}
private <T extends Comparable> ArrayList<T> mergeIndexed(final List<Iterable<T>> indexedLists)
public static <T extends Comparable> ArrayList<T> mergeIndexed(final List<Iterable<T>> indexedLists)
{
Set<T> retVal = Sets.newTreeSet(Ordering.<T>natural().nullsFirst());
@ -1003,7 +1003,7 @@ public class IndexMerger
IndexIO.checkFileSize(indexFile);
}
private static class DimValueConverter
public static class DimValueConverter
{
private final Indexed<String> dimSet;
private final IntBuffer conversionBuf;
@ -1074,7 +1074,7 @@ public class IndexMerger
}
}
private static class ConvertingIndexedInts implements Iterable<Integer>
public static class ConvertingIndexedInts implements Iterable<Integer>
{
private final IndexedInts baseIndex;
private final IntBuffer conversionBuffer;
@ -1115,7 +1115,7 @@ public class IndexMerger
}
}
private static class MMappedIndexRowIterable implements Iterable<Rowboat>
public static class MMappedIndexRowIterable implements Iterable<Rowboat>
{
private final Iterable<Rowboat> index;
private final List<String> convertedDims;
@ -1140,21 +1140,6 @@ public class IndexMerger
return index;
}
public List<String> getConvertedDims()
{
return convertedDims;
}
public Map<String, IntBuffer> getConverters()
{
return converters;
}
public int getIndexNumber()
{
return indexNumber;
}
@Override
public Iterator<Rowboat> iterator()
{
@ -1174,8 +1159,6 @@ public class IndexMerger
index.iterator(),
new Function<Rowboat, Rowboat>()
{
int rowCount = 0;
@Override
public Rowboat apply(@Nullable Rowboat input)
{
@ -1218,7 +1201,7 @@ public class IndexMerger
}
}
private static class AggFactoryStringIndexed implements Indexed<String>
public static class AggFactoryStringIndexed implements Indexed<String>
{
private final AggregatorFactory[] metricAggs;
@ -1255,7 +1238,7 @@ public class IndexMerger
}
}
private static class RowboatMergeFunction implements BinaryFn<Rowboat, Rowboat, Rowboat>
public static class RowboatMergeFunction implements BinaryFn<Rowboat, Rowboat, Rowboat>
{
private final AggregatorFactory[] metricAggs;
@ -1301,7 +1284,7 @@ public class IndexMerger
}
}
static boolean isNullColumn(Iterable<String> dimValues)
public static boolean isNullColumn(Iterable<String> dimValues)
{
if (dimValues == null) {
return true;

View File

@ -0,0 +1,990 @@
/*
* 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.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.base.Splitter;
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.inject.Inject;
import com.metamx.collections.bitmap.BitmapFactory;
import com.metamx.collections.bitmap.ImmutableBitmap;
import com.metamx.collections.bitmap.MutableBitmap;
import com.metamx.collections.spatial.ImmutableRTree;
import com.metamx.collections.spatial.RTree;
import com.metamx.collections.spatial.split.LinearGutmanSplitStrategy;
import com.metamx.common.IAE;
import com.metamx.common.ISE;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.common.guava.MergeIterable;
import com.metamx.common.io.smoosh.FileSmoosher;
import com.metamx.common.io.smoosh.SmooshedWriter;
import com.metamx.common.logger.Logger;
import io.druid.collections.CombiningIterable;
import io.druid.common.utils.JodaUtils;
import io.druid.common.utils.SerializerUtils;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.column.BitmapIndexSeeker;
import io.druid.segment.column.Column;
import io.druid.segment.column.ColumnCapabilities;
import io.druid.segment.column.ColumnCapabilitiesImpl;
import io.druid.segment.column.ColumnDescriptor;
import io.druid.segment.column.ValueType;
import io.druid.segment.data.ArrayIndexed;
import io.druid.segment.data.BitmapSerdeFactory;
import io.druid.segment.data.ByteBufferWriter;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.CompressedVSizeIndexedV3Writer;
import io.druid.segment.data.CompressedVSizeIntsIndexedWriter;
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.IndexedIntsWriter;
import io.druid.segment.data.IndexedIterable;
import io.druid.segment.data.IndexedRTree;
import io.druid.segment.data.TmpFileIOPeon;
import io.druid.segment.data.VSizeIndexedIntsWriter;
import io.druid.segment.data.VSizeIndexedWriter;
import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.incremental.IncrementalIndexAdapter;
import io.druid.segment.serde.ComplexColumnPartSerde;
import io.druid.segment.serde.ComplexColumnSerializer;
import io.druid.segment.serde.ComplexMetricSerde;
import io.druid.segment.serde.ComplexMetrics;
import io.druid.segment.serde.DictionaryEncodedColumnPartSerde;
import io.druid.segment.serde.FloatGenericColumnPartSerde;
import io.druid.segment.serde.LongGenericColumnPartSerde;
import org.apache.commons.io.FileUtils;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.IntBuffer;
import java.nio.MappedByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
public class IndexMergerV9 extends IndexMerger
{
private static final Logger log = new Logger(IndexMergerV9.class);
@Inject
public IndexMergerV9(
ObjectMapper mapper,
IndexIO indexIO
)
{
super(mapper, indexIO);
}
@Override
protected File makeIndexFiles(
final List<IndexableAdapter> adapters,
final File outDir,
final ProgressIndicator progress,
final List<String> mergedDimensions,
final List<String> mergedMetrics,
final Map<String, Object> segmentMetadata,
final Function<ArrayList<Iterable<Rowboat>>, Iterable<Rowboat>> rowMergerFn,
final IndexSpec indexSpec
) throws IOException
{
progress.start();
progress.progress();
final IOPeon ioPeon = new TmpFileIOPeon(false);
final FileSmoosher v9Smoosher = new FileSmoosher(outDir);
final File v9TmpDir = new File(outDir, "v9-tmp");
v9TmpDir.mkdirs();
log.info("Start making v9 index files, outDir:%s", outDir);
long startTime = System.currentTimeMillis();
ByteStreams.write(
Ints.toByteArray(IndexIO.V9_VERSION),
Files.newOutputStreamSupplier(new File(outDir, "version.bin"))
);
log.info("Completed version.bin in %,d millis.", System.currentTimeMillis() - startTime);
progress.progress();
final Map<String, ValueType> metricsValueTypes = Maps.newTreeMap(Ordering.<String>natural().nullsFirst());
final Map<String, String> metricTypeNames = Maps.newTreeMap(Ordering.<String>natural().nullsFirst());
final List<ColumnCapabilitiesImpl> dimCapabilities = Lists.newArrayListWithCapacity(mergedDimensions.size());
mergeCapabilities(adapters, mergedDimensions, metricsValueTypes, metricTypeNames, dimCapabilities);
/************* Setup Dim Conversions **************/
progress.progress();
startTime = System.currentTimeMillis();
final Map<String, Integer> dimCardinalities = Maps.newHashMap();
final ArrayList<GenericIndexedWriter<String>> dimValueWriters = setupDimValueWriters(ioPeon, mergedDimensions);
final ArrayList<Map<String, IntBuffer>> dimConversions = Lists.newArrayListWithCapacity(adapters.size());
final ArrayList<Boolean> dimensionSkipFlag = Lists.newArrayListWithCapacity(mergedDimensions.size());
writeDimValueAndSetupDimConversion(
adapters, progress, mergedDimensions, dimCardinalities, dimValueWriters, dimensionSkipFlag, dimConversions
);
log.info("Completed dim conversions in %,d millis.", System.currentTimeMillis() - startTime);
/************* Walk through data sets, merge them, and write merged columns *************/
progress.progress();
final Iterable<Rowboat> theRows = makeRowIterable(
adapters, mergedDimensions, mergedMetrics, dimConversions, rowMergerFn
);
final LongColumnSerializer timeWriter = setupTimeWriter(ioPeon);
final ArrayList<IndexedIntsWriter> dimWriters = setupDimensionWriters(
ioPeon, mergedDimensions, dimCapabilities, dimCardinalities, indexSpec
);
final ArrayList<GenericColumnSerializer> metWriters = setupMetricsWriters(
ioPeon, mergedMetrics, metricsValueTypes, metricTypeNames, indexSpec
);
final List<IntBuffer> rowNumConversions = Lists.newArrayListWithCapacity(adapters.size());
final ArrayList<MutableBitmap> nullRowsList = Lists.newArrayListWithCapacity(mergedDimensions.size());
for (int i = 0; i < mergedDimensions.size(); ++i) {
nullRowsList.add(indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap());
}
mergeIndexesAndWriteColumns(
adapters, progress, theRows, timeWriter, dimWriters, metWriters,
dimensionSkipFlag, rowNumConversions, nullRowsList
);
/************ Create Inverted Indexes *************/
progress.progress();
final ArrayList<GenericIndexedWriter<ImmutableBitmap>> bitmapIndexWriters = setupBitmapIndexWriters(
ioPeon, mergedDimensions, indexSpec
);
final ArrayList<ByteBufferWriter<ImmutableRTree>> spatialIndexWriters = setupSpatialIndexWriters(
ioPeon, mergedDimensions, indexSpec, dimCapabilities
);
makeInvertedIndexes(
adapters, progress, mergedDimensions, indexSpec, v9TmpDir, rowNumConversions,
nullRowsList, dimValueWriters, bitmapIndexWriters, spatialIndexWriters
);
/************ Finalize Build Columns *************/
progress.progress();
makeTimeColumn(v9Smoosher, progress, timeWriter);
makeMetricsColumns(v9Smoosher, progress, mergedMetrics, metricsValueTypes, metricTypeNames, metWriters);
makeDimensionColumns(
v9Smoosher, progress, indexSpec, mergedDimensions, dimensionSkipFlag, dimCapabilities,
dimValueWriters, dimWriters, bitmapIndexWriters, spatialIndexWriters
);
/************* Make index.drd & metadata.drd files **************/
progress.progress();
makeIndexBinary(
v9Smoosher, adapters, outDir, mergedDimensions, dimensionSkipFlag, mergedMetrics, progress, indexSpec
);
makeMetadataBinary(v9Smoosher, progress, segmentMetadata);
v9Smoosher.close();
ioPeon.cleanup();
FileUtils.deleteDirectory(v9TmpDir);
progress.stop();
return outDir;
}
private void makeMetadataBinary(
final FileSmoosher v9Smoosher,
final ProgressIndicator progress,
final Map<String, Object> segmentMetadata
) throws IOException
{
if (segmentMetadata != null && !segmentMetadata.isEmpty()) {
progress.startSection("make metadata.drd");
v9Smoosher.add("metadata.drd", ByteBuffer.wrap(mapper.writeValueAsBytes(segmentMetadata)));
progress.stopSection("make metadata.drd");
}
}
private void makeIndexBinary(
final FileSmoosher v9Smoosher,
final List<IndexableAdapter> adapters,
final File outDir,
final List<String> mergedDimensions,
final ArrayList<Boolean> dimensionSkipFlag,
final List<String> mergedMetrics,
final ProgressIndicator progress,
final IndexSpec indexSpec
) throws IOException
{
final String section = "make index.drd";
progress.startSection(section);
long startTime = System.currentTimeMillis();
final Set<String> finalColumns = Sets.newTreeSet();
final Set<String> finalDimensions = Sets.newTreeSet();
finalColumns.addAll(mergedMetrics);
for (int i = 0; i < mergedDimensions.size(); ++i) {
if (dimensionSkipFlag.get(i)) {
continue;
}
finalColumns.add(mergedDimensions.get(i));
finalDimensions.add(mergedDimensions.get(i));
}
GenericIndexed<String> cols = GenericIndexed.fromIterable(finalColumns, GenericIndexed.STRING_STRATEGY);
GenericIndexed<String> dims = GenericIndexed.fromIterable(finalDimensions, GenericIndexed.STRING_STRATEGY);
final String bitmapSerdeFactoryType = mapper.writeValueAsString(indexSpec.getBitmapSerdeFactory());
final long numBytes = cols.getSerializedSize()
+ dims.getSerializedSize()
+ 16
+ serializerUtils.getSerializedStringByteSize(bitmapSerdeFactoryType);
final SmooshedWriter writer = v9Smoosher.addWithSmooshedWriter("index.drd", numBytes);
cols.writeToChannel(writer);
dims.writeToChannel(writer);
DateTime minTime = new DateTime(JodaUtils.MAX_INSTANT);
DateTime maxTime = new DateTime(JodaUtils.MIN_INSTANT);
for (IndexableAdapter index : adapters) {
minTime = JodaUtils.minDateTime(minTime, index.getDataInterval().getStart());
maxTime = JodaUtils.maxDateTime(maxTime, index.getDataInterval().getEnd());
}
final Interval dataInterval = new Interval(minTime, maxTime);
serializerUtils.writeLong(writer, dataInterval.getStartMillis());
serializerUtils.writeLong(writer, dataInterval.getEndMillis());
serializerUtils.writeString(
writer, bitmapSerdeFactoryType
);
writer.close();
IndexIO.checkFileSize(new File(outDir, "index.drd"));
log.info("Completed index.drd in %,d millis.", System.currentTimeMillis() - startTime);
progress.stopSection(section);
}
private void makeDimensionColumns(
final FileSmoosher v9Smoosher,
final ProgressIndicator progress,
final IndexSpec indexSpec,
final List<String> mergedDimensions,
final ArrayList<Boolean> dimensionSkipFlag,
final List<ColumnCapabilitiesImpl> dimCapabilities,
final ArrayList<GenericIndexedWriter<String>> dimValueWriters,
final ArrayList<IndexedIntsWriter> dimWriters,
final ArrayList<GenericIndexedWriter<ImmutableBitmap>> bitmapIndexWriters,
final ArrayList<ByteBufferWriter<ImmutableRTree>> spatialIndexWriters
) throws IOException
{
final String section = "make dimension columns";
progress.startSection(section);
long startTime = System.currentTimeMillis();
final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory();
final CompressedObjectStrategy.CompressionStrategy compressionStrategy = indexSpec.getDimensionCompressionStrategy();
for (int i = 0; i < mergedDimensions.size(); ++i) {
long dimStartTime = System.currentTimeMillis();
final String dim = mergedDimensions.get(i);
final IndexedIntsWriter dimWriter = dimWriters.get(i);
final GenericIndexedWriter<ImmutableBitmap> bitmapIndexWriter = bitmapIndexWriters.get(i);
final ByteBufferWriter<ImmutableRTree> spatialIndexWriter = spatialIndexWriters.get(i);
dimWriter.close();
bitmapIndexWriter.close();
if (spatialIndexWriter != null) {
spatialIndexWriter.close();
}
if (dimensionSkipFlag.get(i)) {
continue;
}
boolean hasMultiValue = dimCapabilities.get(i).hasMultipleValues();
final ColumnDescriptor.Builder builder = ColumnDescriptor.builder();
builder.setValueType(ValueType.STRING);
builder.setHasMultipleValues(hasMultiValue);
final DictionaryEncodedColumnPartSerde.SerializerBuilder partBuilder = DictionaryEncodedColumnPartSerde
.serializerBuilder()
.withDictionary(dimValueWriters.get(i))
.withValue(dimWriters.get(i), hasMultiValue, compressionStrategy != null)
.withBitmapSerdeFactory(bitmapSerdeFactory)
.withBitmapIndex(bitmapIndexWriters.get(i))
.withSpatialIndex(spatialIndexWriters.get(i))
.withByteOrder(IndexIO.BYTE_ORDER);
final ColumnDescriptor serdeficator = builder
.addSerde(partBuilder.build())
.build();
makeColumn(v9Smoosher, dim, serdeficator);
log.info("Completed dimension column[%s] in %,d millis.", dim, System.currentTimeMillis() - dimStartTime);
}
log.info("Completed dimension columns in %,d millis.", System.currentTimeMillis() - startTime);
progress.stopSection(section);
}
private void makeMetricsColumns(
final FileSmoosher v9Smoosher,
final ProgressIndicator progress,
final List<String> mergedMetrics,
final Map<String, ValueType> metricsValueTypes,
final Map<String, String> metricTypeNames,
final List<GenericColumnSerializer> metWriters
) throws IOException
{
final String section = "make metric columns";
progress.startSection(section);
long startTime = System.currentTimeMillis();
for (int i = 0; i < mergedMetrics.size(); ++i) {
String metric = mergedMetrics.get(i);
long metricStartTime = System.currentTimeMillis();
GenericColumnSerializer writer = metWriters.get(i);
writer.close();
final ColumnDescriptor.Builder builder = ColumnDescriptor.builder();
ValueType type = metricsValueTypes.get(metric);
switch (type) {
case LONG:
builder.setValueType(ValueType.LONG);
builder.addSerde(
LongGenericColumnPartSerde.serializerBuilder()
.withByteOrder(IndexIO.BYTE_ORDER)
.withDelegate((LongColumnSerializer) writer)
.build()
);
break;
case FLOAT:
builder.setValueType(ValueType.FLOAT);
builder.addSerde(
FloatGenericColumnPartSerde.serializerBuilder()
.withByteOrder(IndexIO.BYTE_ORDER)
.withDelegate((FloatColumnSerializer) writer)
.build()
);
break;
case COMPLEX:
final String typeName = metricTypeNames.get(metric);
builder.setValueType(ValueType.COMPLEX);
builder.addSerde(
ComplexColumnPartSerde.serializerBuilder().withTypeName(typeName)
.withDelegate((ComplexColumnSerializer) writer)
.build()
);
break;
default:
throw new ISE("Unknown type[%s]", type);
}
makeColumn(v9Smoosher, metric, builder.build());
log.info("Completed metric column[%s] in %,d millis.", metric, System.currentTimeMillis() - metricStartTime);
}
log.info("Completed metric columns in %,d millis.", System.currentTimeMillis() - startTime);
progress.stopSection(section);
}
private void makeTimeColumn(
final FileSmoosher v9Smoosher,
final ProgressIndicator progress,
final LongColumnSerializer timeWriter
) throws IOException
{
final String section = "make time column";
progress.startSection(section);
long startTime = System.currentTimeMillis();
timeWriter.close();
final ColumnDescriptor serdeficator = ColumnDescriptor
.builder()
.setValueType(ValueType.LONG)
.addSerde(
LongGenericColumnPartSerde.serializerBuilder()
.withByteOrder(IndexIO.BYTE_ORDER)
.withDelegate(timeWriter)
.build()
)
.build();
makeColumn(v9Smoosher, Column.TIME_COLUMN_NAME, serdeficator);
log.info("Completed time column in %,d millis.", System.currentTimeMillis() - startTime);
progress.stopSection(section);
}
private void makeColumn(
final FileSmoosher v9Smoosher,
final String columnName,
final ColumnDescriptor serdeficator
) throws IOException
{
ByteArrayOutputStream baos = new ByteArrayOutputStream();
serializerUtils.writeString(baos, mapper.writeValueAsString(serdeficator));
byte[] specBytes = baos.toByteArray();
final SmooshedWriter channel = v9Smoosher.addWithSmooshedWriter(
columnName, serdeficator.numBytes() + specBytes.length
);
try {
channel.write(ByteBuffer.wrap(specBytes));
serdeficator.write(channel);
}
finally {
channel.close();
}
}
private void makeInvertedIndexes(
final List<IndexableAdapter> adapters,
final ProgressIndicator progress,
final List<String> mergedDimensions,
final IndexSpec indexSpec,
final File v9OutDir,
final List<IntBuffer> rowNumConversions,
final ArrayList<MutableBitmap> nullRowsList,
final ArrayList<GenericIndexedWriter<String>> dimValueWriters,
final ArrayList<GenericIndexedWriter<ImmutableBitmap>> bitmapIndexWriters,
final ArrayList<ByteBufferWriter<ImmutableRTree>> spatialIndexWriters
) throws IOException
{
final String section = "build inverted index";
progress.startSection(section);
long startTime = System.currentTimeMillis();
final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory();
for (int dimIndex = 0; dimIndex < mergedDimensions.size(); ++dimIndex) {
String dimension = mergedDimensions.get(dimIndex);
long dimStartTime = System.currentTimeMillis();
// write dim values to one single file because we need to read it
File dimValueFile = IndexIO.makeDimFile(v9OutDir, dimension);
FileOutputStream fos = new FileOutputStream(dimValueFile);
ByteStreams.copy(dimValueWriters.get(dimIndex).combineStreams(), fos);
fos.close();
final MappedByteBuffer dimValsMapped = Files.map(dimValueFile);
Indexed<String> dimVals = GenericIndexed.read(dimValsMapped, GenericIndexed.STRING_STRATEGY);
ByteBufferWriter<ImmutableRTree> spatialIndexWriter = spatialIndexWriters.get(dimIndex);
RTree tree = null;
if (spatialIndexWriter != null) {
BitmapFactory bitmapFactory = bitmapSerdeFactory.getBitmapFactory();
tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50, bitmapFactory), bitmapFactory);
}
BitmapIndexSeeker[] bitmapIndexSeeker = new BitmapIndexSeeker[adapters.size()];
for (int j = 0; j < adapters.size(); j++) {
bitmapIndexSeeker[j] = adapters.get(j).getBitmapIndexSeeker(dimension);
}
ImmutableBitmap nullRowBitmap = bitmapSerdeFactory.getBitmapFactory().makeImmutableBitmap(
nullRowsList.get(dimIndex)
);
if (Iterables.getFirst(dimVals, "") != null && !nullRowsList.get(dimIndex).isEmpty()) {
bitmapIndexWriters.get(dimIndex).write(nullRowBitmap);
}
for (String dimVal : IndexedIterable.create(dimVals)) {
progress.progress();
List<Iterable<Integer>> convertedInverteds = Lists.newArrayListWithCapacity(adapters.size());
for (int j = 0; j < adapters.size(); ++j) {
convertedInverteds.add(
new ConvertingIndexedInts(
bitmapIndexSeeker[j].seek(dimVal), rowNumConversions.get(j)
)
);
}
MutableBitmap bitset = bitmapSerdeFactory.getBitmapFactory().makeEmptyMutableBitmap();
for (Integer row : CombiningIterable.createSplatted(
convertedInverteds,
Ordering.<Integer>natural().nullsFirst()
)) {
if (row != INVALID_ROW) {
bitset.add(row);
}
}
ImmutableBitmap bitmapToWrite = bitmapSerdeFactory.getBitmapFactory().makeImmutableBitmap(bitset);
if (dimVal == null) {
bitmapIndexWriters.get(dimIndex).write(nullRowBitmap.union(bitmapToWrite));
} else {
bitmapIndexWriters.get(dimIndex).write(bitmapToWrite);
}
if (spatialIndexWriter != null && dimVal != null) {
List<String> stringCoords = Lists.newArrayList(SPLITTER.split(dimVal));
float[] coords = new float[stringCoords.size()];
for (int j = 0; j < coords.length; j++) {
coords[j] = Float.valueOf(stringCoords.get(j));
}
tree.insert(coords, bitset);
}
}
if (spatialIndexWriter != null) {
spatialIndexWriter.write(ImmutableRTree.newImmutableFromMutable(tree));
}
log.info(
"Completed dim[%s] inverted with cardinality[%,d] in %,d millis.",
dimension,
dimVals.size(),
System.currentTimeMillis() - dimStartTime
);
}
log.info("Completed inverted index in %,d millis.", System.currentTimeMillis() - startTime);
progress.stopSection(section);
}
private ArrayList<GenericIndexedWriter<ImmutableBitmap>> setupBitmapIndexWriters(
final IOPeon ioPeon,
final List<String> mergedDimensions,
final IndexSpec indexSpec
) throws IOException
{
ArrayList<GenericIndexedWriter<ImmutableBitmap>> writers = Lists.newArrayListWithCapacity(mergedDimensions.size());
final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory();
for (String dimension : mergedDimensions) {
GenericIndexedWriter<ImmutableBitmap> writer = new GenericIndexedWriter<>(
ioPeon, String.format("%s.inverted", dimension), bitmapSerdeFactory.getObjectStrategy()
);
writer.open();
writers.add(writer);
}
return writers;
}
private ArrayList<ByteBufferWriter<ImmutableRTree>> setupSpatialIndexWriters(
final IOPeon ioPeon,
final List<String> mergedDimensions,
final IndexSpec indexSpec,
final List<ColumnCapabilitiesImpl> dimCapabilities
) throws IOException
{
ArrayList<ByteBufferWriter<ImmutableRTree>> writers = Lists.newArrayListWithCapacity(mergedDimensions.size());
final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory();
for (int dimIndex = 0; dimIndex < mergedDimensions.size(); ++dimIndex) {
if (dimCapabilities.get(dimIndex).hasSpatialIndexes()) {
BitmapFactory bitmapFactory = bitmapSerdeFactory.getBitmapFactory();
ByteBufferWriter<ImmutableRTree> writer = new ByteBufferWriter<>(
ioPeon,
String.format("%s.spatial", mergedDimensions.get(dimIndex)),
new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapFactory)
);
writer.open();
writers.add(writer);
} else {
writers.add(null);
}
}
return writers;
}
private void mergeIndexesAndWriteColumns(
final List<IndexableAdapter> adapters,
final ProgressIndicator progress,
final Iterable<Rowboat> theRows,
final LongColumnSerializer timeWriter,
final ArrayList<IndexedIntsWriter> dimWriters,
final ArrayList<GenericColumnSerializer> metWriters,
final ArrayList<Boolean> dimensionSkipFlag,
final List<IntBuffer> rowNumConversions,
final ArrayList<MutableBitmap> nullRowsList
) throws IOException
{
final String section = "walk through and merge rows";
progress.startSection(section);
long startTime = System.currentTimeMillis();
int rowCount = 0;
for (IndexableAdapter adapter : adapters) {
int[] arr = new int[adapter.getNumRows()];
Arrays.fill(arr, INVALID_ROW);
rowNumConversions.add(IntBuffer.wrap(arr));
}
long time = System.currentTimeMillis();
for (Rowboat theRow : theRows) {
progress.progress();
timeWriter.serialize(theRow.getTimestamp());
final Object[] metrics = theRow.getMetrics();
for (int i = 0; i < metrics.length; ++i) {
metWriters.get(i).serialize(metrics[i]);
}
int[][] dims = theRow.getDims();
for (int i = 0; i < dims.length; ++i) {
if (dimensionSkipFlag.get(i)) {
continue;
}
if (dims[i] == null || dims[i].length == 0) {
nullRowsList.get(i).add(rowCount);
}
dimWriters.get(i).add(dims[i]);
}
for (Map.Entry<Integer, TreeSet<Integer>> comprisedRow : theRow.getComprisedRows().entrySet()) {
final IntBuffer conversionBuffer = rowNumConversions.get(comprisedRow.getKey());
for (Integer rowNum : comprisedRow.getValue()) {
while (conversionBuffer.position() < rowNum) {
conversionBuffer.put(INVALID_ROW);
}
conversionBuffer.put(rowCount);
}
}
if ((++rowCount % 500000) == 0) {
log.info("walked 500,000/%d rows in %,d millis.", rowCount, System.currentTimeMillis() - time);
time = System.currentTimeMillis();
}
}
for (IntBuffer rowNumConversion : rowNumConversions) {
rowNumConversion.rewind();
}
log.info("completed walk through of %,d rows in %,d millis.", rowCount, System.currentTimeMillis() - startTime);
progress.stopSection(section);
}
private LongColumnSerializer setupTimeWriter(final IOPeon ioPeon) throws IOException
{
LongColumnSerializer timeWriter = LongColumnSerializer.create(
ioPeon, "little_end_time", CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY
);
// we will close this writer after we added all the timestamps
timeWriter.open();
return timeWriter;
}
private ArrayList<GenericColumnSerializer> setupMetricsWriters(
final IOPeon ioPeon,
final List<String> mergedMetrics,
final Map<String, ValueType> metricsValueTypes,
final Map<String, String> metricTypeNames,
final IndexSpec indexSpec
) throws IOException
{
ArrayList<GenericColumnSerializer> metWriters = Lists.newArrayListWithCapacity(mergedMetrics.size());
final CompressedObjectStrategy.CompressionStrategy metCompression = indexSpec.getMetricCompressionStrategy();
for (String metric : mergedMetrics) {
ValueType type = metricsValueTypes.get(metric);
GenericColumnSerializer writer;
switch (type) {
case LONG:
writer = LongColumnSerializer.create(ioPeon, metric, metCompression);
break;
case FLOAT:
writer = FloatColumnSerializer.create(ioPeon, metric, 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);
}
writer = ComplexColumnSerializer.create(ioPeon, metric, serde);
break;
default:
throw new ISE("Unknown type[%s]", type);
}
writer.open();
// we will close these writers in another method after we added all the metrics
metWriters.add(writer);
}
return metWriters;
}
private ArrayList<IndexedIntsWriter> setupDimensionWriters(
final IOPeon ioPeon,
final List<String> mergedDimensions,
final List<ColumnCapabilitiesImpl> dimCapabilities,
final Map<String, Integer> dimCardinalities,
final IndexSpec indexSpec
) throws IOException
{
ArrayList<IndexedIntsWriter> dimWriters = Lists.newArrayListWithCapacity(mergedDimensions.size());
final CompressedObjectStrategy.CompressionStrategy dimCompression = indexSpec.getDimensionCompressionStrategy();
for (int dimIndex = 0; dimIndex < mergedDimensions.size(); ++dimIndex) {
String dim = mergedDimensions.get(dimIndex);
int cardinality = dimCardinalities.get(dim);
ColumnCapabilitiesImpl capabilities = dimCapabilities.get(dimIndex);
String filenameBase = String.format("%s.forward_dim", dim);
IndexedIntsWriter writer;
if (capabilities.hasMultipleValues()) {
writer = (dimCompression != null)
? CompressedVSizeIndexedV3Writer.create(ioPeon, filenameBase, cardinality, dimCompression)
: new VSizeIndexedWriter(ioPeon, filenameBase, cardinality);
} else {
writer = (dimCompression != null)
? CompressedVSizeIntsIndexedWriter.create(ioPeon, filenameBase, cardinality, dimCompression)
: new VSizeIndexedIntsWriter(ioPeon, filenameBase, cardinality);
}
writer.open();
// we will close these writers in another method after we added all the values
dimWriters.add(writer);
}
return dimWriters;
}
private Iterable<Rowboat> makeRowIterable(
final List<IndexableAdapter> adapters,
final List<String> mergedDimensions,
final List<String> mergedMetrics,
final ArrayList<Map<String, IntBuffer>> dimConversions,
final Function<ArrayList<Iterable<Rowboat>>, Iterable<Rowboat>> rowMergerFn
)
{
ArrayList<Iterable<Rowboat>> boats = Lists.newArrayListWithCapacity(adapters.size());
for (int i = 0; i < adapters.size(); ++i) {
final IndexableAdapter adapter = adapters.get(i);
final int[] dimLookup = new int[mergedDimensions.size()];
int count = 0;
for (String dim : adapter.getDimensionNames()) {
dimLookup[count] = mergedDimensions.indexOf(dim);
count++;
}
final int[] metricLookup = new int[mergedMetrics.size()];
count = 0;
for (String metric : adapter.getMetricNames()) {
metricLookup[count] = mergedMetrics.indexOf(metric);
count++;
}
boats.add(
new MMappedIndexRowIterable(
Iterables.transform(
adapters.get(i).getRows(),
new Function<Rowboat, Rowboat>()
{
@Override
public Rowboat apply(Rowboat input)
{
int[][] newDims = new int[mergedDimensions.size()][];
int j = 0;
for (int[] dim : input.getDims()) {
newDims[dimLookup[j]] = dim;
j++;
}
Object[] newMetrics = new Object[mergedMetrics.size()];
j = 0;
for (Object met : input.getMetrics()) {
newMetrics[metricLookup[j]] = met;
j++;
}
return new Rowboat(
input.getTimestamp(),
newDims,
newMetrics,
input.getRowNum()
);
}
}
),
mergedDimensions,
dimConversions.get(i),
i
)
);
}
return rowMergerFn.apply(boats);
}
private ArrayList<GenericIndexedWriter<String>> setupDimValueWriters(
final IOPeon ioPeon,
final List<String> mergedDimensions
)
throws IOException
{
ArrayList<GenericIndexedWriter<String>> dimValueWriters = Lists.newArrayListWithCapacity(mergedDimensions.size());
for (String dimension : mergedDimensions) {
final GenericIndexedWriter<String> writer = new GenericIndexedWriter<>(
ioPeon, String.format("%s.dim_values", dimension), GenericIndexed.STRING_STRATEGY
);
writer.open();
dimValueWriters.add(writer);
}
return dimValueWriters;
}
private void writeDimValueAndSetupDimConversion(
final List<IndexableAdapter> adapters,
final ProgressIndicator progress,
final List<String> mergedDimensions,
final Map<String, Integer> dimensionCardinalities,
final ArrayList<GenericIndexedWriter<String>> dimValueWriters,
final ArrayList<Boolean> dimensionSkipFlag,
final List<Map<String, IntBuffer>> dimConversions
) throws IOException
{
final String section = "setup dimension conversions";
progress.startSection(section);
for (int i = 0; i < adapters.size(); ++i) {
dimConversions.add(Maps.<String, IntBuffer>newHashMap());
}
for (int dimIndex = 0; dimIndex < mergedDimensions.size(); ++dimIndex) {
long dimStartTime = System.currentTimeMillis();
String dimension = mergedDimensions.get(dimIndex);
// lookups for all dimension values of this dimension
List<Indexed<String>> dimValueLookups = Lists.newArrayListWithCapacity(adapters.size());
// each converter converts dim values of this dimension to global dictionary
DimValueConverter[] converters = new DimValueConverter[adapters.size()];
boolean existNullColumn = false;
for (int i = 0; i < adapters.size(); i++) {
Indexed<String> dimValues = adapters.get(i).getDimValueLookup(dimension);
if (!isNullColumn(dimValues)) {
dimValueLookups.add(dimValues);
converters[i] = new DimValueConverter(dimValues);
} else {
existNullColumn = true;
}
}
Iterable<Indexed<String>> bumpedDimValueLookups;
if (!dimValueLookups.isEmpty() && existNullColumn) {
log.info("dim[%s] are null in some indexes, append null value to dim values", dimension);
bumpedDimValueLookups = Iterables.concat(
Arrays.asList(new ArrayIndexed<>(new String[]{null}, String.class)),
dimValueLookups
);
} else {
bumpedDimValueLookups = dimValueLookups;
}
// sort all dimension values and treat all null values as empty strings
Iterable<String> dimensionValues = CombiningIterable.createSplatted(
Iterables.transform(
bumpedDimValueLookups,
new Function<Indexed<String>, Iterable<String>>()
{
@Override
public Iterable<String> apply(@Nullable Indexed<String> indexed)
{
return Iterables.transform(
indexed,
new Function<String, String>()
{
@Override
public String apply(@Nullable String input)
{
return (input == null) ? "" : input;
}
}
);
}
}
), Ordering.<String>natural().nullsFirst()
);
GenericIndexedWriter<String> writer = dimValueWriters.get(dimIndex);
int cardinality = 0;
for (String value : dimensionValues) {
value = value == null ? "" : value;
writer.write(value);
for (int i = 0; i < adapters.size(); i++) {
DimValueConverter converter = converters[i];
if (converter != null) {
converter.convert(value, cardinality);
}
}
++cardinality;
}
log.info(
"Completed dim[%s] conversions with cardinality[%,d] in %,d millis.",
dimension,
cardinality,
System.currentTimeMillis() - dimStartTime
);
dimensionCardinalities.put(dimension, cardinality);
writer.close();
if (cardinality == 0) {
log.info(String.format("Skipping [%s], it is empty!", dimension));
dimensionSkipFlag.add(true);
continue;
}
dimensionSkipFlag.add(false);
// make the conversion
for (int i = 0; i < adapters.size(); ++i) {
DimValueConverter converter = converters[i];
if (converter != null) {
dimConversions.get(i).put(dimension, converters[i].getConversionBuffer());
}
}
}
progress.stopSection(section);
}
private void mergeCapabilities(
final List<IndexableAdapter> adapters,
final List<String> mergedDimensions,
final Map<String, ValueType> metricsValueTypes,
final Map<String, String> metricTypeNames,
final List<ColumnCapabilitiesImpl> dimCapabilities
)
{
final Map<String, ColumnCapabilitiesImpl> capabilitiesMap = Maps.newHashMap();
for (IndexableAdapter adapter : adapters) {
for (String dimension : adapter.getDimensionNames()) {
ColumnCapabilitiesImpl mergedCapabilities = capabilitiesMap.get(dimension);
if (mergedCapabilities == null) {
mergedCapabilities = new ColumnCapabilitiesImpl();
mergedCapabilities.setType(ValueType.STRING);
}
capabilitiesMap.put(dimension, mergedCapabilities.merge(adapter.getCapabilities(dimension)));
}
for (String metric : adapter.getMetricNames()) {
ColumnCapabilitiesImpl mergedCapabilities = capabilitiesMap.get(metric);
ColumnCapabilities capabilities = adapter.getCapabilities(metric);
if (mergedCapabilities == null) {
mergedCapabilities = new ColumnCapabilitiesImpl();
}
capabilitiesMap.put(metric, mergedCapabilities.merge(capabilities));
metricsValueTypes.put(metric, capabilities.getType());
metricTypeNames.put(metric, adapter.getMetricType(metric));
}
}
for (String dim : mergedDimensions) {
dimCapabilities.add(capabilitiesMap.get(dim));
}
}
}

View File

@ -0,0 +1,97 @@
/*
* 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.CompressedLongsSupplierSerializer;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.IOPeon;
import java.io.IOException;
import java.nio.ByteOrder;
import java.nio.channels.WritableByteChannel;
public class LongColumnSerializer implements GenericColumnSerializer
{
public static LongColumnSerializer create(
IOPeon ioPeon,
String filenameBase,
CompressedObjectStrategy.CompressionStrategy compression
)
{
return new LongColumnSerializer(ioPeon, filenameBase, IndexIO.BYTE_ORDER, compression);
}
private final IOPeon ioPeon;
private final String filenameBase;
private final ByteOrder byteOrder;
private final CompressedObjectStrategy.CompressionStrategy compression;
private CompressedLongsSupplierSerializer writer;
public LongColumnSerializer(
IOPeon ioPeon,
String filenameBase,
ByteOrder byteOrder,
CompressedObjectStrategy.CompressionStrategy compression
)
{
this.ioPeon = ioPeon;
this.filenameBase = filenameBase;
this.byteOrder = byteOrder;
this.compression = compression;
}
@Override
public void open() throws IOException
{
writer = CompressedLongsSupplierSerializer.create(
ioPeon,
String.format("%s.long_column", filenameBase),
byteOrder,
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
{
writer.close();
}
@Override
public long getSerializedSize()
{
return writer.getSerializedSize();
}
@Override
public void writeToChannel(WritableByteChannel channel) throws IOException
{
writer.writeToChannel(channel);
}
}

View File

@ -79,7 +79,7 @@ public class ColumnDescriptor
long retVal = 0;
for (ColumnPartSerde part : parts) {
retVal += part.numBytes();
retVal += part.getSerializer().numBytes();
}
return retVal;
@ -88,7 +88,7 @@ public class ColumnDescriptor
public void write(WritableByteChannel channel) throws IOException
{
for (ColumnPartSerde part : parts) {
part.write(channel);
part.getSerializer().write(channel);
}
}
@ -99,7 +99,7 @@ public class ColumnDescriptor
.setHasMultipleValues(hasMultipleValues);
for (ColumnPartSerde part : parts) {
part.read(buffer, builder, columnConfig);
part.getDeserializer().read(buffer, builder, columnConfig);
}
return builder.build();

View File

@ -30,6 +30,9 @@ import com.google.common.primitives.Ints;
import java.io.Closeable;
import java.io.IOException;
import java.io.InputStream;
import java.nio.channels.Channels;
import java.nio.channels.ReadableByteChannel;
import java.nio.channels.WritableByteChannel;
import java.util.Arrays;
/**
@ -85,6 +88,11 @@ public class ByteBufferWriter<T> implements Closeable
);
}
public long getSerializedSize()
{
return headerOut.getCount() + valueOut.getCount();
}
public InputSupplier<InputStream> combineStreams()
{
return ByteStreams.join(
@ -108,4 +116,10 @@ public class ByteBufferWriter<T> implements Closeable
)
);
}
public void writeToChannel(WritableByteChannel channel) throws IOException
{
final ReadableByteChannel from = Channels.newChannel(combineStreams().getInput());
ByteStreams.copy(from, channel);
}
}

View File

@ -27,22 +27,33 @@ import io.druid.collections.StupidResourceHolder;
import java.io.IOException;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.nio.FloatBuffer;
import java.nio.channels.Channels;
import java.nio.channels.ReadableByteChannel;
import java.nio.channels.WritableByteChannel;
/**
*/
public class CompressedFloatsSupplierSerializer
{
public static CompressedFloatsSupplierSerializer create(
IOPeon ioPeon, final String filenameBase, final ByteOrder order, final CompressedObjectStrategy.CompressionStrategy compression
IOPeon ioPeon,
final String filenameBase,
final ByteOrder order,
final CompressedObjectStrategy.CompressionStrategy compression
) throws IOException
{
return create(ioPeon, filenameBase, CompressedFloatsIndexedSupplier.MAX_FLOATS_IN_BUFFER, order, compression);
}
public static CompressedFloatsSupplierSerializer create(
IOPeon ioPeon, final String filenameBase, final int sizePer, final ByteOrder order, final CompressedObjectStrategy.CompressionStrategy compression
IOPeon ioPeon,
final String filenameBase,
final int sizePer,
final ByteOrder order,
final CompressedObjectStrategy.CompressionStrategy compression
) throws IOException
{
final CompressedFloatsSupplierSerializer retVal = new CompressedFloatsSupplierSerializer(
@ -89,7 +100,7 @@ public class CompressedFloatsSupplierSerializer
public void add(float value) throws IOException
{
if (! endBuffer.hasRemaining()) {
if (!endBuffer.hasRemaining()) {
endBuffer.rewind();
flattener.write(StupidResourceHolder.create(endBuffer));
endBuffer = FloatBuffer.allocate(sizePer);
@ -102,13 +113,7 @@ public class CompressedFloatsSupplierSerializer
public void closeAndConsolidate(OutputSupplier<? extends OutputStream> consolidatedOut) throws IOException
{
endBuffer.limit(endBuffer.position());
endBuffer.rewind();
flattener.write(StupidResourceHolder.create(endBuffer));
endBuffer = null;
flattener.close();
close();
try (OutputStream out = consolidatedOut.getOutput()) {
out.write(CompressedFloatsIndexedSupplier.version);
out.write(Ints.toByteArray(numInserted));
@ -117,4 +122,32 @@ public class CompressedFloatsSupplierSerializer
ByteStreams.copy(flattener.combineStreams(), out);
}
}
public void close() throws IOException
{
endBuffer.limit(endBuffer.position());
endBuffer.rewind();
flattener.write(StupidResourceHolder.create(endBuffer));
endBuffer = null;
flattener.close();
}
public long getSerializedSize()
{
return 1 + // version
Ints.BYTES + // elements num
Ints.BYTES + // sizePer
1 + // compression id
flattener.getSerializedSize();
}
public void writeToChannel(WritableByteChannel channel) throws IOException
{
channel.write(ByteBuffer.wrap(new byte[]{CompressedFloatsIndexedSupplier.version}));
channel.write(ByteBuffer.wrap(Ints.toByteArray(numInserted)));
channel.write(ByteBuffer.wrap(Ints.toByteArray(sizePer)));
channel.write(ByteBuffer.wrap(new byte[]{compression.getId()}));
final ReadableByteChannel from = Channels.newChannel(flattener.combineStreams().getInput());
ByteStreams.copy(from, channel);
}
}

View File

@ -38,7 +38,7 @@ import java.util.List;
public class CompressedIntsIndexedSupplier implements WritableSupplier<IndexedInts>
{
public static final byte version = 0x2;
public static final byte VERSION = 0x2;
public static final int MAX_INTS_IN_BUFFER = CompressedPools.BUFFER_SIZE / Ints.BYTES;
@ -103,7 +103,7 @@ public class CompressedIntsIndexedSupplier implements WritableSupplier<IndexedIn
public void writeToChannel(WritableByteChannel channel) throws IOException
{
channel.write(ByteBuffer.wrap(new byte[]{version}));
channel.write(ByteBuffer.wrap(new byte[]{VERSION}));
channel.write(ByteBuffer.wrap(Ints.toByteArray(totalSize)));
channel.write(ByteBuffer.wrap(Ints.toByteArray(sizePer)));
channel.write(ByteBuffer.wrap(new byte[]{compression.getId()}));
@ -132,7 +132,7 @@ public class CompressedIntsIndexedSupplier implements WritableSupplier<IndexedIn
{
byte versionFromBuffer = buffer.get();
if (versionFromBuffer == version) {
if (versionFromBuffer == VERSION) {
final int totalSize = buffer.getInt();
final int sizePer = buffer.getInt();
final CompressedObjectStrategy.CompressionStrategy compression = CompressedObjectStrategy.CompressionStrategy.forId(buffer.get());

View File

@ -0,0 +1,135 @@
/*
* 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.data;
import com.google.common.io.ByteStreams;
import com.google.common.primitives.Ints;
import io.druid.collections.ResourceHolder;
import io.druid.collections.StupidResourceHolder;
import io.druid.segment.IndexIO;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.nio.IntBuffer;
import java.nio.channels.Channels;
import java.nio.channels.ReadableByteChannel;
import java.nio.channels.WritableByteChannel;
/**
* Streams array of integers out in the binary format described by CompressedIntsIndexedSupplier
*/
public class CompressedIntsIndexedWriter extends SingleValueIndexedIntsWriter
{
private static final byte VERSION = CompressedIntsIndexedSupplier.VERSION;
public static CompressedIntsIndexedWriter create(
final IOPeon ioPeon,
final String filenameBase,
final CompressedObjectStrategy.CompressionStrategy compression
)
{
return new CompressedIntsIndexedWriter(
ioPeon,
filenameBase,
CompressedIntsIndexedSupplier.MAX_INTS_IN_BUFFER,
IndexIO.BYTE_ORDER,
compression
);
}
private final int chunkFactor;
private final CompressedObjectStrategy.CompressionStrategy compression;
private final GenericIndexedWriter<ResourceHolder<IntBuffer>> flattener;
private IntBuffer endBuffer;
private int numInserted;
public CompressedIntsIndexedWriter(
final IOPeon ioPeon,
final String filenameBase,
final int chunkFactor,
final ByteOrder byteOrder,
final CompressedObjectStrategy.CompressionStrategy compression
)
{
this.chunkFactor = chunkFactor;
this.compression = compression;
this.flattener = new GenericIndexedWriter<>(
ioPeon, filenameBase, CompressedIntBufferObjectStrategy.getBufferForOrder(byteOrder, compression, chunkFactor)
);
this.endBuffer = IntBuffer.allocate(chunkFactor);
this.numInserted = 0;
}
@Override
public void open() throws IOException
{
flattener.open();
}
@Override
protected void addValue(int val) throws IOException
{
if (!endBuffer.hasRemaining()) {
endBuffer.rewind();
flattener.write(StupidResourceHolder.create(endBuffer));
endBuffer = IntBuffer.allocate(chunkFactor);
}
endBuffer.put(val);
numInserted++;
}
@Override
public void close() throws IOException
{
try {
if (numInserted > 0) {
endBuffer.limit(endBuffer.position());
endBuffer.rewind();
flattener.write(StupidResourceHolder.create(endBuffer));
}
endBuffer = null;
}
finally {
flattener.close();
}
}
@Override
public long getSerializedSize()
{
return 1 + // version
Ints.BYTES + // numInserted
Ints.BYTES + // chunkFactor
1 + // compression id
flattener.getSerializedSize();
}
@Override
public void writeToChannel(WritableByteChannel channel) throws IOException
{
channel.write(ByteBuffer.wrap(new byte[]{VERSION}));
channel.write(ByteBuffer.wrap(Ints.toByteArray(numInserted)));
channel.write(ByteBuffer.wrap(Ints.toByteArray(chunkFactor)));
channel.write(ByteBuffer.wrap(new byte[]{compression.getId()}));
final ReadableByteChannel from = Channels.newChannel(flattener.combineStreams().getInput());
ByteStreams.copy(from, channel);
}
}

View File

@ -27,21 +27,34 @@ import io.druid.collections.StupidResourceHolder;
import java.io.IOException;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.nio.LongBuffer;
import java.nio.channels.Channels;
import java.nio.channels.ReadableByteChannel;
import java.nio.channels.WritableByteChannel;
/**
*/
public class CompressedLongsSupplierSerializer
{
public static CompressedLongsSupplierSerializer create(
IOPeon ioPeon, final String filenameBase, final ByteOrder order, final CompressedObjectStrategy.CompressionStrategy compression
final IOPeon ioPeon,
final String filenameBase,
final ByteOrder order,
final CompressedObjectStrategy.CompressionStrategy compression
) throws IOException
{
final CompressedLongsSupplierSerializer retVal = new CompressedLongsSupplierSerializer(
CompressedLongsIndexedSupplier.MAX_LONGS_IN_BUFFER,
new GenericIndexedWriter<ResourceHolder<LongBuffer>>(
ioPeon, filenameBase, CompressedLongBufferObjectStrategy.getBufferForOrder(order, compression, CompressedLongsIndexedSupplier.MAX_LONGS_IN_BUFFER)
ioPeon,
filenameBase,
CompressedLongBufferObjectStrategy.getBufferForOrder(
order,
compression,
CompressedLongsIndexedSupplier.MAX_LONGS_IN_BUFFER
)
),
compression
);
@ -82,7 +95,7 @@ public class CompressedLongsSupplierSerializer
public void add(long value) throws IOException
{
if (! endBuffer.hasRemaining()) {
if (!endBuffer.hasRemaining()) {
endBuffer.rewind();
flattener.write(StupidResourceHolder.create(endBuffer));
endBuffer = LongBuffer.allocate(sizePer);
@ -95,13 +108,7 @@ public class CompressedLongsSupplierSerializer
public void closeAndConsolidate(OutputSupplier<? extends OutputStream> consolidatedOut) throws IOException
{
endBuffer.limit(endBuffer.position());
endBuffer.rewind();
flattener.write(StupidResourceHolder.create(endBuffer));
endBuffer = null;
flattener.close();
close();
try (OutputStream out = consolidatedOut.getOutput()) {
out.write(CompressedLongsIndexedSupplier.version);
out.write(Ints.toByteArray(numInserted));
@ -110,4 +117,31 @@ public class CompressedLongsSupplierSerializer
ByteStreams.copy(flattener.combineStreams(), out);
}
}
public void close() throws IOException {
endBuffer.limit(endBuffer.position());
endBuffer.rewind();
flattener.write(StupidResourceHolder.create(endBuffer));
endBuffer = null;
flattener.close();
}
public long getSerializedSize()
{
return 1 + // version
Ints.BYTES + // elements num
Ints.BYTES + // sizePer
1 + // compression id
flattener.getSerializedSize();
}
public void writeToChannel(WritableByteChannel channel) throws IOException
{
channel.write(ByteBuffer.wrap(new byte[]{CompressedFloatsIndexedSupplier.version}));
channel.write(ByteBuffer.wrap(Ints.toByteArray(numInserted)));
channel.write(ByteBuffer.wrap(Ints.toByteArray(sizePer)));
channel.write(ByteBuffer.wrap(new byte[]{compression.getId()}));
final ReadableByteChannel from = Channels.newChannel(flattener.combineStreams().getInput());
ByteStreams.copy(from, channel);
}
}

View File

@ -0,0 +1,127 @@
/*
* 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.
*/
/**
* Streams array of integers out in the binary format described by CompressedVSizeIndexedV3Supplier
*/
package io.druid.segment.data;
import io.druid.segment.CompressedVSizeIndexedV3Supplier;
import io.druid.segment.IndexIO;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.WritableByteChannel;
import java.util.ArrayList;
import java.util.List;
public class CompressedVSizeIndexedV3Writer extends MultiValueIndexedIntsWriter
{
private static final byte VERSION = CompressedVSizeIndexedV3Supplier.VERSION;
private static final List<Integer> EMPTY_LIST = new ArrayList<>();
public static CompressedVSizeIndexedV3Writer create(
final IOPeon ioPeon,
final String filenameBase,
final int maxValue,
final CompressedObjectStrategy.CompressionStrategy compression
)
{
return new CompressedVSizeIndexedV3Writer(
new CompressedIntsIndexedWriter(
ioPeon,
String.format("%s.offsets", filenameBase),
CompressedIntsIndexedSupplier.MAX_INTS_IN_BUFFER,
IndexIO.BYTE_ORDER,
compression
),
new CompressedVSizeIntsIndexedWriter(
ioPeon,
String.format("%s.values", filenameBase),
maxValue,
CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(maxValue),
IndexIO.BYTE_ORDER,
compression
)
);
}
private final CompressedIntsIndexedWriter offsetWriter;
private final CompressedVSizeIntsIndexedWriter valueWriter;
private int offset;
public CompressedVSizeIndexedV3Writer(
CompressedIntsIndexedWriter offsetWriter,
CompressedVSizeIntsIndexedWriter valueWriter
)
{
this.offsetWriter = offsetWriter;
this.valueWriter = valueWriter;
this.offset = 0;
}
@Override
public void open() throws IOException
{
offsetWriter.open();
valueWriter.open();
}
@Override
protected void addValues(List<Integer> vals) throws IOException
{
if (vals == null) {
vals = EMPTY_LIST;
}
offsetWriter.add(offset);
for (Integer val : vals) {
valueWriter.add(val);
}
offset += vals.size();
}
@Override
public void close() throws IOException
{
try {
offsetWriter.add(offset);
}
finally {
offsetWriter.close();
valueWriter.close();
}
}
@Override
public long getSerializedSize()
{
return 1 + // version
offsetWriter.getSerializedSize() +
valueWriter.getSerializedSize();
}
@Override
public void writeToChannel(WritableByteChannel channel) throws IOException
{
channel.write(ByteBuffer.wrap(new byte[]{VERSION}));
offsetWriter.writeToChannel(channel);
valueWriter.writeToChannel(channel);
}
}

View File

@ -40,7 +40,7 @@ import java.util.List;
public class CompressedVSizeIntsIndexedSupplier implements WritableSupplier<IndexedInts>
{
public static final byte version = 0x2;
public static final byte VERSION = 0x2;
private final int totalSize;
private final int sizePer;
@ -79,7 +79,7 @@ public class CompressedVSizeIntsIndexedSupplier implements WritableSupplier<Inde
return 1 << (Integer.SIZE - 1 - Integer.numberOfLeadingZeros(maxSizePer));
}
private static int bufferPadding(int numBytes)
public static int bufferPadding(int numBytes)
{
// when numBytes == 3 we need to pad the buffer to allow reading an extra byte
// beyond the end of the last value, since we use buffer.getInt() to read values.
@ -132,7 +132,7 @@ public class CompressedVSizeIntsIndexedSupplier implements WritableSupplier<Inde
public void writeToChannel(WritableByteChannel channel) throws IOException
{
channel.write(ByteBuffer.wrap(new byte[]{version, (byte) numBytes}));
channel.write(ByteBuffer.wrap(new byte[]{VERSION, (byte) numBytes}));
channel.write(ByteBuffer.wrap(Ints.toByteArray(totalSize)));
channel.write(ByteBuffer.wrap(Ints.toByteArray(sizePer)));
channel.write(ByteBuffer.wrap(new byte[]{compression.getId()}));
@ -151,7 +151,7 @@ public class CompressedVSizeIntsIndexedSupplier implements WritableSupplier<Inde
{
byte versionFromBuffer = buffer.get();
if (versionFromBuffer == version) {
if (versionFromBuffer == VERSION) {
final int numBytes = buffer.get();
final int totalSize = buffer.getInt();
final int sizePer = buffer.getInt();

View File

@ -0,0 +1,150 @@
/*
* 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.data;
import com.google.common.io.ByteStreams;
import com.google.common.primitives.Ints;
import io.druid.collections.ResourceHolder;
import io.druid.collections.StupidResourceHolder;
import io.druid.segment.IndexIO;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.nio.channels.Channels;
import java.nio.channels.ReadableByteChannel;
import java.nio.channels.WritableByteChannel;
/**
* Streams array of integers out in the binary format described by CompressedVSizeIntsIndexedSupplier
*/
public class CompressedVSizeIntsIndexedWriter extends SingleValueIndexedIntsWriter
{
private static final byte VERSION = CompressedVSizeIntsIndexedSupplier.VERSION;
public static CompressedVSizeIntsIndexedWriter create(
final IOPeon ioPeon,
final String filenameBase,
final int maxValue,
final CompressedObjectStrategy.CompressionStrategy compression
)
{
return new CompressedVSizeIntsIndexedWriter(
ioPeon, filenameBase, maxValue,
CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(maxValue),
IndexIO.BYTE_ORDER, compression
);
}
private final int numBytes;
private final int chunkFactor;
private final int chunkBytes;
private final ByteOrder byteOrder;
private final CompressedObjectStrategy.CompressionStrategy compression;
private final GenericIndexedWriter<ResourceHolder<ByteBuffer>> flattener;
private final ByteBuffer intBuffer;
private ByteBuffer endBuffer;
private int numInserted;
public CompressedVSizeIntsIndexedWriter(
final IOPeon ioPeon,
final String filenameBase,
final int maxValue,
final int chunkFactor,
final ByteOrder byteOrder,
final CompressedObjectStrategy.CompressionStrategy compression
)
{
this.numBytes = VSizeIndexedInts.getNumBytesForMax(maxValue);
this.chunkFactor = chunkFactor;
this.chunkBytes = chunkFactor * numBytes + CompressedVSizeIntsIndexedSupplier.bufferPadding(numBytes);
this.byteOrder = byteOrder;
this.compression = compression;
this.flattener = new GenericIndexedWriter<>(
ioPeon, filenameBase, CompressedByteBufferObjectStrategy.getBufferForOrder(byteOrder, compression, chunkBytes)
);
this.intBuffer = ByteBuffer.allocate(Ints.BYTES).order(byteOrder);
this.endBuffer = ByteBuffer.allocate(chunkBytes).order(byteOrder);
this.endBuffer.limit(numBytes * chunkFactor);
this.numInserted = 0;
}
@Override
public void open() throws IOException
{
flattener.open();
}
@Override
protected void addValue(int val) throws IOException
{
if (!endBuffer.hasRemaining()) {
endBuffer.rewind();
flattener.write(StupidResourceHolder.create(endBuffer));
endBuffer = ByteBuffer.allocate(chunkBytes).order(byteOrder);
endBuffer.limit(numBytes * chunkFactor);
}
intBuffer.putInt(0, val);
if (byteOrder.equals(ByteOrder.BIG_ENDIAN)) {
endBuffer.put(intBuffer.array(), Ints.BYTES - numBytes, numBytes);
} else {
endBuffer.put(intBuffer.array(), 0, numBytes);
}
numInserted++;
}
@Override
public void close() throws IOException
{
try {
if (numInserted > 0) {
endBuffer.limit(endBuffer.position());
endBuffer.rewind();
flattener.write(StupidResourceHolder.create(endBuffer));
}
endBuffer = null;
}
finally {
flattener.close();
}
}
@Override
public long getSerializedSize()
{
return 1 + // version
1 + // numBytes
Ints.BYTES + // numInserted
Ints.BYTES + // chunkFactor
1 + // compression id
flattener.getSerializedSize();
}
@Override
public void writeToChannel(WritableByteChannel channel) throws IOException
{
channel.write(ByteBuffer.wrap(new byte[]{VERSION, (byte) numBytes}));
channel.write(ByteBuffer.wrap(Ints.toByteArray(numInserted)));
channel.write(ByteBuffer.wrap(Ints.toByteArray(chunkFactor)));
channel.write(ByteBuffer.wrap(new byte[]{compression.getId()}));
final ReadableByteChannel from = Channels.newChannel(flattener.combineStreams().getInput());
ByteStreams.copy(from, channel);
}
}

View File

@ -31,6 +31,9 @@ import java.io.Closeable;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.channels.Channels;
import java.nio.channels.ReadableByteChannel;
import java.nio.channels.WritableByteChannel;
import java.util.Arrays;
/**
@ -120,6 +123,15 @@ public class GenericIndexedWriter<T> implements Closeable
}
}
public long getSerializedSize()
{
return 2 + // version and sorted flag
Ints.BYTES + // numBytesWritten
Ints.BYTES + // numElements
headerOut.getCount() + // header length
valuesOut.getCount(); // value length
}
public InputSupplier<InputStream> combineStreams()
{
return ByteStreams.join(
@ -143,4 +155,10 @@ public class GenericIndexedWriter<T> implements Closeable
)
);
}
public void writeToChannel(WritableByteChannel channel) throws IOException
{
final ReadableByteChannel from = Channels.newChannel(combineStreams().getInput());
ByteStreams.copy(from, channel);
}
}

View File

@ -0,0 +1,35 @@
/*
* 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.data;
import java.io.Closeable;
import java.io.IOException;
import java.nio.channels.WritableByteChannel;
public interface IndexedIntsWriter extends Closeable
{
public void open() throws IOException;
public void add(Object obj) throws IOException;
public long getSerializedSize();
public void writeToChannel(WritableByteChannel channel) throws IOException;
}

View File

@ -0,0 +1,45 @@
/*
* 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.data;
import com.google.common.primitives.Ints;
import com.metamx.common.IAE;
import java.io.IOException;
import java.util.List;
public abstract class MultiValueIndexedIntsWriter implements IndexedIntsWriter
{
@Override
public void add(Object obj) throws IOException
{
if (obj == null) {
addValues(null);
} else if (obj instanceof int[]) {
addValues(Ints.asList((int[]) obj));
} else if (obj instanceof List) {
addValues((List<Integer>) obj);
} else {
throw new IAE("unsupported multi-value type: " + obj.getClass());
}
}
protected abstract void addValues(List<Integer> vals) throws IOException;
}

View File

@ -0,0 +1,48 @@
/*
* 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.data;
import com.metamx.common.IAE;
import java.io.IOException;
public abstract class SingleValueIndexedIntsWriter implements IndexedIntsWriter
{
@Override
public void add(Object obj) throws IOException
{
if (obj == null) {
addValue(0);
} else if (obj instanceof Integer) {
addValue(((Number) obj).intValue());
} else if (obj instanceof int[]) {
int[] vals = (int[]) obj;
if (vals.length == 0) {
addValue(0);
} else {
addValue(vals[0]);
}
} else {
throw new IAE("Unsupported single value type: " + obj.getClass());
}
}
protected abstract void addValue(int val) throws IOException;
}

View File

@ -34,8 +34,19 @@ import java.util.Map;
*/
public class TmpFileIOPeon implements IOPeon
{
private final boolean allowOverwrite;
Map<String, File> createdFiles = Maps.newLinkedHashMap();
public TmpFileIOPeon()
{
this(true);
}
public TmpFileIOPeon(boolean allowOverwrite)
{
this.allowOverwrite = allowOverwrite;
}
@Override
public OutputStream makeOutputStream(String filename) throws IOException
{
@ -44,8 +55,12 @@ public class TmpFileIOPeon implements IOPeon
retFile = File.createTempFile("filePeon", filename);
retFile.deleteOnExit();
createdFiles.put(filename, retFile);
return new BufferedOutputStream(new FileOutputStream(retFile));
} else if (allowOverwrite) {
return new BufferedOutputStream(new FileOutputStream(retFile));
} else {
throw new IOException("tmp file conflicts, file[" + filename + "] already exist!");
}
return new BufferedOutputStream(new FileOutputStream(retFile));
}
@Override
@ -64,4 +79,9 @@ public class TmpFileIOPeon implements IOPeon
}
createdFiles.clear();
}
public boolean isOverwriteAllowed()
{
return allowOverwrite;
}
}

View File

@ -33,7 +33,7 @@ import java.util.List;
*/
public class VSizeIndexedInts implements IndexedInts, Comparable<VSizeIndexedInts>
{
private static final byte version = 0x0;
public static final byte VERSION = 0x0;
public static VSizeIndexedInts fromArray(int[] array)
{
@ -185,7 +185,7 @@ public class VSizeIndexedInts implements IndexedInts, Comparable<VSizeIndexedInt
public void writeToChannel(WritableByteChannel channel) throws IOException
{
channel.write(ByteBuffer.wrap(new byte[]{version, (byte) numBytes}));
channel.write(ByteBuffer.wrap(new byte[]{VERSION, (byte) numBytes}));
channel.write(ByteBuffer.wrap(Ints.toByteArray(buffer.remaining())));
channel.write(buffer.asReadOnlyBuffer());
}
@ -194,7 +194,7 @@ public class VSizeIndexedInts implements IndexedInts, Comparable<VSizeIndexedInt
{
byte versionFromBuffer = buffer.get();
if (version == versionFromBuffer) {
if (VERSION == versionFromBuffer) {
int numBytes = buffer.get();
int size = buffer.getInt();
ByteBuffer bufferToUse = buffer.asReadOnlyBuffer();

View File

@ -0,0 +1,94 @@
/*
* 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.data;
import com.google.common.io.ByteStreams;
import com.google.common.io.CountingOutputStream;
import com.google.common.primitives.Ints;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.Channels;
import java.nio.channels.ReadableByteChannel;
import java.nio.channels.WritableByteChannel;
/**
* Streams integers out in the binary format described by VSizeIndexedInts
*/
public class VSizeIndexedIntsWriter extends SingleValueIndexedIntsWriter
{
private static final byte VERSION = VSizeIndexedInts.VERSION;
private final IOPeon ioPeon;
private final String valueFileName;
private final int numBytes;
private CountingOutputStream valuesOut = null;
public VSizeIndexedIntsWriter(
final IOPeon ioPeon,
final String filenameBase,
final int maxValue
)
{
this.ioPeon = ioPeon;
this.valueFileName = String.format("%s.values", filenameBase);
this.numBytes = VSizeIndexedInts.getNumBytesForMax(maxValue);
}
@Override
public void open() throws IOException
{
valuesOut = new CountingOutputStream(ioPeon.makeOutputStream(valueFileName));
}
@Override
protected void addValue(int val) throws IOException
{
byte[] intAsBytes = Ints.toByteArray(val);
valuesOut.write(intAsBytes, intAsBytes.length - numBytes, numBytes);
}
@Override
public void close() throws IOException
{
byte[] bufPadding = new byte[4 - numBytes];
valuesOut.write(bufPadding);
valuesOut.close();
}
@Override
public long getSerializedSize()
{
return 2 + // version and numBytes
4 + // dataLen
valuesOut.getCount();
}
@Override
public void writeToChannel(WritableByteChannel channel) throws IOException
{
long numBytesWritten = valuesOut.getCount();
channel.write(ByteBuffer.wrap(new byte[]{VERSION, (byte) numBytes}));
channel.write(ByteBuffer.wrap(Ints.toByteArray((int) numBytesWritten)));
final ReadableByteChannel from = Channels.newChannel(ioPeon.makeInputStream(valueFileName));
ByteStreams.copy(from, channel);
}
}

View File

@ -32,15 +32,18 @@ import java.io.Closeable;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.channels.Channels;
import java.nio.channels.ReadableByteChannel;
import java.nio.channels.WritableByteChannel;
import java.util.Arrays;
import java.util.List;
/**
* Streams arrays of objects out in the binary format described by VSizeIndexed
*/
public class VSizeIndexedWriter implements Closeable
public class VSizeIndexedWriter extends MultiValueIndexedIntsWriter implements Closeable
{
private static final byte version = 0x1;
private static final byte VERSION = 0x1;
private static final byte[] EMPTY_ARRAY = new byte[]{};
private final int maxId;
@ -49,7 +52,9 @@ public class VSizeIndexedWriter implements Closeable
private CountingOutputStream valuesOut = null;
int numWritten = 0;
private final IOPeon ioPeon;
private final String filenameBase;
private final String metaFileName;
private final String headerFileName;
private final String valuesFileName;
public VSizeIndexedWriter(
IOPeon ioPeon,
@ -58,14 +63,22 @@ public class VSizeIndexedWriter implements Closeable
)
{
this.ioPeon = ioPeon;
this.filenameBase = filenameBase;
this.metaFileName = String.format("%s.meta", filenameBase);
this.headerFileName = String.format("%s.header", filenameBase);
this.valuesFileName = String.format("%s.values", filenameBase);
this.maxId = maxId;
}
public void open() throws IOException
{
headerOut = new CountingOutputStream(ioPeon.makeOutputStream(makeFilename("header")));
valuesOut = new CountingOutputStream(ioPeon.makeOutputStream(makeFilename("values")));
headerOut = new CountingOutputStream(ioPeon.makeOutputStream(headerFileName));
valuesOut = new CountingOutputStream(ioPeon.makeOutputStream(valuesFileName));
}
@Override
protected void addValues(List<Integer> val) throws IOException
{
write(val);
}
public void write(List<Integer> ints) throws IOException
@ -79,11 +92,6 @@ public class VSizeIndexedWriter implements Closeable
++numWritten;
}
private String makeFilename(String suffix)
{
return String.format("%s.%s", filenameBase, suffix);
}
@Override
public void close() throws IOException
{
@ -107,10 +115,10 @@ public class VSizeIndexedWriter implements Closeable
numBytesWritten < Integer.MAX_VALUE, "Wrote[%s] bytes, which is too many.", numBytesWritten
);
OutputStream metaOut = ioPeon.makeOutputStream(makeFilename("meta"));
OutputStream metaOut = ioPeon.makeOutputStream(metaFileName);
try {
metaOut.write(new byte[]{version, numBytesForMax});
metaOut.write(new byte[]{VERSION, numBytesForMax});
metaOut.write(Ints.toByteArray((int) numBytesWritten + 4));
metaOut.write(Ints.toByteArray(numWritten));
}
@ -123,7 +131,7 @@ public class VSizeIndexedWriter implements Closeable
{
return ByteStreams.join(
Iterables.transform(
Arrays.asList("meta", "header", "values"),
Arrays.asList(metaFileName, headerFileName, valuesFileName),
new Function<String,InputSupplier<InputStream>>() {
@Override
@ -134,7 +142,7 @@ public class VSizeIndexedWriter implements Closeable
@Override
public InputStream getInput() throws IOException
{
return ioPeon.makeInputStream(makeFilename(input));
return ioPeon.makeInputStream(input);
}
};
}
@ -142,4 +150,22 @@ public class VSizeIndexedWriter implements Closeable
)
);
}
@Override
public long getSerializedSize()
{
return 1 + // version
1 + // numBytes
4 + // numBytesWritten
4 + // numElements
headerOut.getCount() +
valuesOut.getCount();
}
@Override
public void writeToChannel(WritableByteChannel channel) throws IOException
{
final ReadableByteChannel from = Channels.newChannel(combineStreams().getInput());
ByteStreams.copy(from, channel);
}
}

View File

@ -20,13 +20,14 @@
package io.druid.segment.incremental;
import com.google.common.base.Function;
import com.google.common.base.Strings;
import com.google.common.collect.Iterators;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.metamx.collections.bitmap.BitmapFactory;
import com.metamx.collections.bitmap.MutableBitmap;
import com.metamx.common.ISE;
import com.metamx.common.logger.Logger;
import io.druid.segment.IndexableAdapter;
import io.druid.segment.Rowboat;
import io.druid.segment.column.BitmapIndexSeeker;
@ -38,16 +39,15 @@ import io.druid.segment.data.Indexed;
import io.druid.segment.data.IndexedInts;
import io.druid.segment.data.IndexedIterable;
import io.druid.segment.data.ListIndexed;
import org.joda.time.Interval;
import org.roaringbitmap.IntIterator;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
*/
@ -57,6 +57,7 @@ public class IncrementalIndexAdapter implements IndexableAdapter
private final Interval dataInterval;
private final IncrementalIndex<?> index;
private final Map<String, Map<String, MutableBitmap>> invertedIndexes;
private final Set<String> hasNullValueDimensions;
public IncrementalIndexAdapter(
Interval dataInterval, IncrementalIndex<?> index, BitmapFactory bitmapFactory
@ -64,8 +65,17 @@ public class IncrementalIndexAdapter implements IndexableAdapter
{
this.dataInterval = dataInterval;
this.index = index;
this.invertedIndexes = Maps.newHashMap();
/* Sometimes it's hard to tell whether one dimension contains a null value or not.
* If one dimension had show a null or empty value explicitly, then yes, it contains
* null value. But if one dimension's values are all non-null, it still early to say
* this dimension does not contain null value. Consider a two row case, first row had
* "dimA=1" and "dimB=2", the second row only had "dimA=3". To dimB, its value are "2" and
* never showed a null or empty value. But when we combines these two rows, dimB is null
* in row 2. So we should iterate all rows to determine whether one dimension contains
* a null value.
*/
this.hasNullValueDimensions = Sets.newHashSet();
final List<IncrementalIndex.DimensionDesc> dimensions = index.getDimensions();
@ -86,8 +96,12 @@ public class IncrementalIndexAdapter implements IndexableAdapter
continue;
}
if (dimIndex >= dims.length || dims[dimIndex] == null) {
hasNullValueDimensions.add(dimension.getName());
continue;
}
if (hasNullValue(dims[dimIndex])) {
hasNullValueDimensions.add(dimension.getName());
}
for (String dimValue : dims[dimIndex]) {
MutableBitmap mutableBitmap = bitmapIndexes.get(dimValue);
@ -138,6 +152,11 @@ public class IncrementalIndexAdapter implements IndexableAdapter
public Indexed<String> getDimValueLookup(String dimension)
{
final IncrementalIndex.DimDim dimDim = index.getDimensionValues(dimension);
if (hasNullValueDimensions.contains(dimension)
&& !dimDim.contains(null))
{
dimDim.add(null);
}
dimDim.sort();
return new Indexed<String>()
@ -287,10 +306,13 @@ public class IncrementalIndexAdapter implements IndexableAdapter
@Override
public IndexedInts seek(String value)
{
if (value != null && GenericIndexed.STRING_STRATEGY.compare(value, lastVal) <= 0) {
throw new ISE("Value[%s] is less than the last value[%s] I have, cannot be.",
value, lastVal);
if (value != null && GenericIndexed.STRING_STRATEGY.compare(value, lastVal) <= 0) {
throw new ISE(
"Value[%s] is less than the last value[%s] I have, cannot be.",
value, lastVal
);
}
value = Strings.nullToEmpty(value);
lastVal = value;
final MutableBitmap bitmapIndex = dimInverted.get(value);
if (bitmapIndex == null) {
@ -301,7 +323,21 @@ public class IncrementalIndexAdapter implements IndexableAdapter
};
}
static class BitmapIndexedInts implements IndexedInts {
private boolean hasNullValue(String[] dimValues)
{
if (dimValues == null || dimValues.length == 0) {
return true;
}
for (String dimVal : dimValues) {
if (Strings.isNullOrEmpty(dimVal)) {
return true;
}
}
return false;
}
static class BitmapIndexedInts implements IndexedInts
{
private final MutableBitmap bitmapIndex;

View File

@ -39,7 +39,19 @@ import java.nio.channels.WritableByteChannel;
})
public interface ColumnPartSerde
{
public long numBytes();
public void write(WritableByteChannel channel) throws IOException;
public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig);
public Serializer getSerializer();
public Deserializer getDeserializer();
public interface Serializer
{
public long numBytes();
public void write(WritableByteChannel channel) throws IOException;
}
public interface Deserializer
{
public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig);
}
}

View File

@ -30,7 +30,7 @@ import java.nio.ByteBuffer;
import java.nio.channels.WritableByteChannel;
/**
*/
*/
public class ComplexColumnPartSerde implements ColumnPartSerde
{
@JsonCreator
@ -38,19 +38,18 @@ public class ComplexColumnPartSerde implements ColumnPartSerde
@JsonProperty("typeName") String complexType
)
{
return new ComplexColumnPartSerde(null, complexType);
return new ComplexColumnPartSerde(complexType, null);
}
private final GenericIndexed column;
private final String typeName;
private final ComplexMetricSerde serde;
private final Serializer serializer;
public ComplexColumnPartSerde(GenericIndexed column, String typeName)
private ComplexColumnPartSerde(String typeName, Serializer serializer)
{
this.column = column;
this.typeName = typeName;
serde = ComplexMetrics.getSerdeForType(typeName);
this.serde = ComplexMetrics.getSerdeForType(typeName);
this.serializer = serializer;
}
@JsonProperty
@ -59,21 +58,110 @@ public class ComplexColumnPartSerde implements ColumnPartSerde
return typeName;
}
@Override
public long numBytes()
public static SerializerBuilder serializerBuilder()
{
return column.getSerializedSize();
return new SerializerBuilder();
}
public static class SerializerBuilder
{
private String typeName = null;
private ComplexColumnSerializer delegate = null;
public SerializerBuilder withTypeName(final String typeName)
{
this.typeName = typeName;
return this;
}
public SerializerBuilder withDelegate(final ComplexColumnSerializer 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) throws IOException
{
delegate.writeToChannel(channel);
}
}
);
}
}
public static LegacySerializerBuilder legacySerializerBuilder()
{
return new LegacySerializerBuilder();
}
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) throws IOException
{
delegate.writeToChannel(channel);
}
}
);
}
}
@Override
public void write(WritableByteChannel channel) throws IOException
public Serializer getSerializer()
{
column.writeToChannel(channel);
return serializer;
}
@Override
public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig)
public Deserializer getDeserializer()
{
return serde == null ? this : serde.deserializeColumn(buffer, builder);
return new Deserializer()
{
@Override
public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig)
{
if (serde != null) {
serde.deserializeColumn(buffer, builder);
}
}
};
}
}

View File

@ -0,0 +1,90 @@
/*
* 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 io.druid.segment.GenericColumnSerializer;
import io.druid.segment.data.GenericIndexedWriter;
import io.druid.segment.data.IOPeon;
import java.io.IOException;
import java.nio.channels.WritableByteChannel;
public class ComplexColumnSerializer implements GenericColumnSerializer
{
public static ComplexColumnSerializer create(
IOPeon ioPeon,
String filenameBase,
ComplexMetricSerde serde
)
{
return new ComplexColumnSerializer(ioPeon, filenameBase, serde);
}
private final IOPeon ioPeon;
private final String filenameBase;
private final ComplexMetricSerde serde;
private GenericIndexedWriter writer;
public ComplexColumnSerializer(
IOPeon ioPeon,
String filenameBase,
ComplexMetricSerde serde
)
{
this.ioPeon = ioPeon;
this.filenameBase = filenameBase;
this.serde = serde;
}
@SuppressWarnings(value = "unchecked")
@Override
public void open() throws IOException
{
writer = new GenericIndexedWriter(
ioPeon, String.format("%s.complex_column", filenameBase), serde.getObjectStrategy()
);
writer.open();
}
@SuppressWarnings(value = "unchecked")
@Override
public void serialize(Object obj) throws IOException
{
writer.write(obj);
}
@Override
public void close() throws IOException
{
writer.close();
}
@Override
public long getSerializedSize()
{
return writer.getSerializedSize();
}
@Override
public void writeToChannel(WritableByteChannel channel) throws IOException
{
writer.writeToChannel(channel);
}
}

View File

@ -36,14 +36,10 @@ public abstract class ComplexMetricSerde
* Deserializes a ByteBuffer and adds it to the ColumnBuilder. This method allows for the ComplexMetricSerde
* to implement it's own versioning scheme to allow for changes of binary format in a forward-compatible manner.
*
* The method is also in charge of returning a ColumnPartSerde that knows how to serialize out the object it
* added to the builder.
*
* @param buffer the buffer to deserialize
* @param builder ColumnBuilder to add the column to
* @return a ColumnPartSerde that can serialize out the object that was read from the buffer to the builder
*/
public abstract ColumnPartSerde deserializeColumn(ByteBuffer buffer, ColumnBuilder builder);
public abstract void deserializeColumn(ByteBuffer buffer, ColumnBuilder builder);
/**
* This is deprecated because its usage is going to be removed from the code.

View File

@ -27,15 +27,18 @@ import com.metamx.collections.bitmap.ImmutableBitmap;
import com.metamx.collections.spatial.ImmutableRTree;
import com.metamx.common.IAE;
import io.druid.segment.CompressedVSizeIndexedSupplier;
import io.druid.segment.CompressedVSizeIndexedV3Supplier;
import io.druid.segment.column.ColumnBuilder;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.column.ValueType;
import io.druid.segment.data.BitmapSerde;
import io.druid.segment.data.BitmapSerdeFactory;
import io.druid.segment.data.ByteBufferSerializer;
import io.druid.segment.data.ByteBufferWriter;
import io.druid.segment.data.CompressedVSizeIntsIndexedSupplier;
import io.druid.segment.data.GenericIndexed;
import io.druid.segment.data.GenericIndexedWriter;
import io.druid.segment.data.IndexedInts;
import io.druid.segment.data.IndexedIntsWriter;
import io.druid.segment.data.IndexedMultivalue;
import io.druid.segment.data.IndexedRTree;
import io.druid.segment.data.VSizeIndexed;
@ -56,7 +59,8 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
enum Feature
{
MULTI_VALUE;
MULTI_VALUE,
MULTI_VALUE_V3;
public boolean isSet(int flags) { return (getMask() & flags) != 0; }
@ -83,7 +87,169 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
}
}
public static class Builder
@JsonCreator
public static DictionaryEncodedColumnPartSerde createDeserializer(
@Nullable @JsonProperty("bitmapSerdeFactory") BitmapSerdeFactory bitmapSerdeFactory,
@NotNull @JsonProperty("byteOrder") ByteOrder byteOrder
)
{
return new DictionaryEncodedColumnPartSerde(byteOrder, bitmapSerdeFactory, null);
}
private final ByteOrder byteOrder;
private final BitmapSerdeFactory bitmapSerdeFactory;
private final Serializer serializer;
private DictionaryEncodedColumnPartSerde(
ByteOrder byteOrder,
BitmapSerdeFactory bitmapSerdeFactory,
Serializer serializer
)
{
this.byteOrder = byteOrder;
this.bitmapSerdeFactory = bitmapSerdeFactory;
this.serializer = serializer;
}
@JsonProperty
public BitmapSerdeFactory getBitmapSerdeFactory()
{
return bitmapSerdeFactory;
}
@JsonProperty
public ByteOrder getByteOrder()
{
return byteOrder;
}
public static SerializerBuilder serializerBuilder()
{
return new SerializerBuilder();
}
public static class SerializerBuilder
{
private VERSION version = null;
private int flags = NO_FLAGS;
private GenericIndexedWriter<String> dictionaryWriter = null;
private IndexedIntsWriter valueWriter = null;
private BitmapSerdeFactory bitmapSerdeFactory = null;
private GenericIndexedWriter<ImmutableBitmap> bitmapIndexWriter = null;
private ByteBufferWriter<ImmutableRTree> spatialIndexWriter = null;
private ByteOrder byteOrder = null;
public SerializerBuilder withDictionary(GenericIndexedWriter<String> dictionaryWriter)
{
this.dictionaryWriter = dictionaryWriter;
return this;
}
public SerializerBuilder withBitmapSerdeFactory(BitmapSerdeFactory bitmapSerdeFactory)
{
this.bitmapSerdeFactory = bitmapSerdeFactory;
return this;
}
public SerializerBuilder withBitmapIndex(GenericIndexedWriter<ImmutableBitmap> bitmapIndexWriter)
{
this.bitmapIndexWriter = bitmapIndexWriter;
return this;
}
public SerializerBuilder withSpatialIndex(ByteBufferWriter<ImmutableRTree> spatialIndexWriter)
{
this.spatialIndexWriter = spatialIndexWriter;
return this;
}
public SerializerBuilder withByteOrder(ByteOrder byteOrder)
{
this.byteOrder = byteOrder;
return this;
}
public SerializerBuilder withValue(IndexedIntsWriter valueWriter, boolean hasMultiValue, boolean compressed)
{
this.valueWriter = valueWriter;
if (hasMultiValue) {
if (compressed) {
this.version = VERSION.COMPRESSED;
this.flags |= Feature.MULTI_VALUE_V3.getMask();
} else {
this.version = VERSION.UNCOMPRESSED_MULTI_VALUE;
this.flags |= Feature.MULTI_VALUE.getMask();
}
} else {
if (compressed) {
this.version = VERSION.COMPRESSED;
} else {
this.version = VERSION.UNCOMPRESSED_SINGLE_VALUE;
}
}
return this;
}
public DictionaryEncodedColumnPartSerde build()
{
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
if (dictionaryWriter != null) {
size += dictionaryWriter.getSerializedSize();
}
if (valueWriter != null) {
size += valueWriter.getSerializedSize();
}
if (bitmapIndexWriter != null) {
size += bitmapIndexWriter.getSerializedSize();
}
if (spatialIndexWriter != null) {
size += spatialIndexWriter.getSerializedSize();
}
return size;
}
@Override
public void write(WritableByteChannel channel) throws IOException
{
channel.write(ByteBuffer.wrap(new byte[]{version.asByte()}));
if (version.compareTo(VERSION.COMPRESSED) >= 0) {
channel.write(ByteBuffer.wrap(Ints.toByteArray(flags)));
}
if (dictionaryWriter != null) {
dictionaryWriter.writeToChannel(channel);
}
if (valueWriter != null) {
valueWriter.writeToChannel(channel);
}
if (bitmapIndexWriter != null) {
bitmapIndexWriter.writeToChannel(channel);
}
if (spatialIndexWriter != null) {
spatialIndexWriter.writeToChannel(channel);
}
}
}
);
}
}
public static LegacySerializerBuilder legacySerializerBuilder()
{
return new LegacySerializerBuilder();
}
public static class LegacySerializerBuilder
{
private VERSION version = null;
private int flags = NO_FLAGS;
@ -95,41 +261,41 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
private ImmutableRTree spatialIndex = null;
private ByteOrder byteOrder = null;
private Builder()
private LegacySerializerBuilder()
{
}
public Builder withDictionary(GenericIndexed<String> dictionary)
public LegacySerializerBuilder withDictionary(GenericIndexed<String> dictionary)
{
this.dictionary = dictionary;
return this;
}
public Builder withBitmapSerdeFactory(BitmapSerdeFactory bitmapSerdeFactory)
public LegacySerializerBuilder withBitmapSerdeFactory(BitmapSerdeFactory bitmapSerdeFactory)
{
this.bitmapSerdeFactory = bitmapSerdeFactory;
return this;
}
public Builder withBitmaps(GenericIndexed<ImmutableBitmap> bitmaps)
public LegacySerializerBuilder withBitmaps(GenericIndexed<ImmutableBitmap> bitmaps)
{
this.bitmaps = bitmaps;
return this;
}
public Builder withSpatialIndex(ImmutableRTree spatialIndex)
public LegacySerializerBuilder withSpatialIndex(ImmutableRTree spatialIndex)
{
this.spatialIndex = spatialIndex;
return this;
}
public Builder withByteOrder(ByteOrder byteOrder)
public LegacySerializerBuilder withByteOrder(ByteOrder byteOrder)
{
this.byteOrder = byteOrder;
return this;
}
public Builder withSingleValuedColumn(VSizeIndexedInts singleValuedColumn)
public LegacySerializerBuilder withSingleValuedColumn(VSizeIndexedInts singleValuedColumn)
{
Preconditions.checkState(multiValuedColumn == null, "Cannot set both singleValuedColumn and multiValuedColumn");
this.version = VERSION.UNCOMPRESSED_SINGLE_VALUE;
@ -137,7 +303,7 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
return this;
}
public Builder withSingleValuedColumn(CompressedVSizeIntsIndexedSupplier singleValuedColumn)
public LegacySerializerBuilder withSingleValuedColumn(CompressedVSizeIntsIndexedSupplier singleValuedColumn)
{
Preconditions.checkState(multiValuedColumn == null, "Cannot set both singleValuedColumn and multiValuedColumn");
this.version = VERSION.COMPRESSED;
@ -145,7 +311,7 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
return this;
}
public Builder withMultiValuedColumn(VSizeIndexed multiValuedColumn)
public LegacySerializerBuilder withMultiValuedColumn(VSizeIndexed multiValuedColumn)
{
Preconditions.checkState(singleValuedColumn == null, "Cannot set both multiValuedColumn and singleValuedColumn");
this.version = VERSION.UNCOMPRESSED_MULTI_VALUE;
@ -154,7 +320,7 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
return this;
}
public Builder withMultiValuedColumn(CompressedVSizeIndexedSupplier multiValuedColumn)
public LegacySerializerBuilder withMultiValuedColumn(CompressedVSizeIndexedSupplier multiValuedColumn)
{
Preconditions.checkState(singleValuedColumn == null, "Cannot set both singleValuedColumn and multiValuedColumn");
this.version = VERSION.COMPRESSED;
@ -172,251 +338,171 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
);
return new DictionaryEncodedColumnPartSerde(
version,
flags,
dictionary,
singleValuedColumn,
multiValuedColumn,
byteOrder,
bitmapSerdeFactory,
bitmaps,
spatialIndex,
byteOrder
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) 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
);
}
}
}
);
}
}
public static Builder builder()
{
return new Builder();
}
private final BitmapSerdeFactory bitmapSerdeFactory;
private final ByteOrder byteOrder;
private final GenericIndexed<String> dictionary;
private final WritableSupplier<IndexedInts> singleValuedColumn;
private final WritableSupplier<IndexedMultivalue<IndexedInts>> multiValuedColumn;
private final GenericIndexed<ImmutableBitmap> bitmaps;
private final ImmutableRTree spatialIndex;
private final int flags;
private final VERSION version;
private final long size;
@JsonCreator
public DictionaryEncodedColumnPartSerde(
@Nullable @JsonProperty("bitmapSerdeFactory") BitmapSerdeFactory bitmapSerdeFactory,
@NotNull @JsonProperty("byteOrder") ByteOrder byteOrder
)
{
this.bitmapSerdeFactory = bitmapSerdeFactory == null
? new BitmapSerde.LegacyBitmapSerdeFactory()
: bitmapSerdeFactory;
this.byteOrder = byteOrder;
// dummy values
this.dictionary = null;
this.singleValuedColumn = null;
this.multiValuedColumn = null;
this.bitmaps = null;
this.spatialIndex = null;
this.size = -1;
this.flags = 0;
this.version = VERSION.COMPRESSED;
}
private DictionaryEncodedColumnPartSerde(
VERSION version,
int flags,
GenericIndexed<String> dictionary,
WritableSupplier<IndexedInts> singleValuedColumn,
WritableSupplier<IndexedMultivalue<IndexedInts>> multiValuedColumn,
BitmapSerdeFactory bitmapSerdeFactory,
GenericIndexed<ImmutableBitmap> bitmaps,
ImmutableRTree spatialIndex,
ByteOrder byteOrder
)
{
Preconditions.checkArgument(version.compareTo(VERSION.COMPRESSED) <= 0, "Unsupported version[%s]", version);
this.bitmapSerdeFactory = bitmapSerdeFactory;
this.byteOrder = byteOrder;
this.version = version;
this.flags = flags;
this.dictionary = dictionary;
this.singleValuedColumn = singleValuedColumn;
this.multiValuedColumn = multiValuedColumn;
this.bitmaps = bitmaps;
this.spatialIndex = spatialIndex;
long 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;
}
this.size = size;
}
@JsonProperty
public BitmapSerdeFactory getBitmapSerdeFactory()
{
return bitmapSerdeFactory;
}
@JsonProperty
public ByteOrder getByteOrder()
{
return byteOrder;
}
@Override
public void write(WritableByteChannel channel) throws IOException
public Serializer getSerializer()
{
channel.write(ByteBuffer.wrap(new byte[]{version.asByte()}));
if (version.compareTo(VERSION.COMPRESSED) >= 0) {
channel.write(ByteBuffer.wrap(Ints.toByteArray(flags)));
}
return serializer;
}
if (dictionary != null) {
dictionary.writeToChannel(channel);
}
@Override
public Deserializer getDeserializer()
{
return new Deserializer()
{
@Override
public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig)
{
final VERSION rVersion = VERSION.fromByte(buffer.get());
final int rFlags;
if (Feature.MULTI_VALUE.isSet(flags)) {
if (multiValuedColumn != null) {
multiValuedColumn.writeToChannel(channel);
if (rVersion.compareTo(VERSION.COMPRESSED) >= 0) {
rFlags = buffer.getInt();
} else {
rFlags = rVersion.equals(VERSION.UNCOMPRESSED_MULTI_VALUE)
? Feature.MULTI_VALUE.getMask()
: NO_FLAGS;
}
final boolean hasMultipleValues = Feature.MULTI_VALUE.isSet(rFlags) || Feature.MULTI_VALUE_V3.isSet(rFlags);
final GenericIndexed<String> rDictionary = GenericIndexed.read(buffer, GenericIndexed.STRING_STRATEGY);
builder.setType(ValueType.STRING);
final WritableSupplier<IndexedInts> rSingleValuedColumn;
final WritableSupplier<IndexedMultivalue<IndexedInts>> rMultiValuedColumn;
if (hasMultipleValues) {
rMultiValuedColumn = readMultiValuedColum(rVersion, buffer, rFlags);
rSingleValuedColumn = null;
} else {
rSingleValuedColumn = readSingleValuedColumn(rVersion, buffer);
rMultiValuedColumn = null;
}
builder.setHasMultipleValues(hasMultipleValues)
.setDictionaryEncodedColumn(
new DictionaryEncodedColumnSupplier(
rDictionary,
rSingleValuedColumn,
rMultiValuedColumn,
columnConfig.columnCacheSizeBytes()
)
);
GenericIndexed<ImmutableBitmap> rBitmaps = GenericIndexed.read(
buffer, bitmapSerdeFactory.getObjectStrategy()
);
builder.setBitmapIndex(
new BitmapIndexColumnPartSupplier(
bitmapSerdeFactory.getBitmapFactory(),
rBitmaps,
rDictionary
)
);
ImmutableRTree rSpatialIndex = null;
if (buffer.hasRemaining()) {
rSpatialIndex = ByteBufferSerializer.read(
buffer, new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapSerdeFactory.getBitmapFactory())
);
builder.setSpatialIndex(new SpatialIndexColumnPartSupplier(rSpatialIndex));
}
}
} else {
if (singleValuedColumn != null) {
singleValuedColumn.writeToChannel(channel);
private WritableSupplier<IndexedInts> readSingleValuedColumn(VERSION version, ByteBuffer buffer)
{
switch (version) {
case UNCOMPRESSED_SINGLE_VALUE:
return VSizeIndexedInts.readFromByteBuffer(buffer).asWritableSupplier();
case COMPRESSED:
return CompressedVSizeIntsIndexedSupplier.fromByteBuffer(buffer, byteOrder);
}
throw new IAE("Unsupported single-value version[%s]", version);
}
}
if (bitmaps != null) {
bitmaps.writeToChannel(channel);
}
if (spatialIndex != null) {
ByteBufferSerializer.writeToChannel(
spatialIndex,
new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapSerdeFactory.getBitmapFactory()),
channel
);
}
}
@Override
public ColumnPartSerde read(
ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig
)
{
final VERSION rVersion = VERSION.fromByte(buffer.get());
final int rFlags;
if (rVersion.compareTo(VERSION.COMPRESSED) >= 0) {
rFlags = buffer.getInt();
} else {
rFlags = rVersion.equals(VERSION.UNCOMPRESSED_MULTI_VALUE) ?
Feature.MULTI_VALUE.getMask() :
NO_FLAGS;
}
final boolean hasMultipleValues = Feature.MULTI_VALUE.isSet(rFlags);
final GenericIndexed<String> rDictionary = GenericIndexed.read(buffer, GenericIndexed.STRING_STRATEGY);
builder.setType(ValueType.STRING);
final WritableSupplier<IndexedInts> rSingleValuedColumn;
final WritableSupplier<IndexedMultivalue<IndexedInts>> rMultiValuedColumn;
if (hasMultipleValues) {
rMultiValuedColumn = readMultiValuedColum(rVersion, buffer);
rSingleValuedColumn = null;
} else {
rSingleValuedColumn = readSingleValuedColumn(rVersion, buffer);
rMultiValuedColumn = null;
}
builder.setHasMultipleValues(hasMultipleValues)
.setDictionaryEncodedColumn(
new DictionaryEncodedColumnSupplier(
rDictionary,
rSingleValuedColumn,
rMultiValuedColumn,
columnConfig.columnCacheSizeBytes()
)
);
GenericIndexed<ImmutableBitmap> rBitmaps = GenericIndexed.read(
buffer, bitmapSerdeFactory.getObjectStrategy()
);
builder.setBitmapIndex(
new BitmapIndexColumnPartSupplier(
bitmapSerdeFactory.getBitmapFactory(),
rBitmaps,
rDictionary
)
);
ImmutableRTree rSpatialIndex = null;
if (buffer.hasRemaining()) {
rSpatialIndex = ByteBufferSerializer.read(
buffer, new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapSerdeFactory.getBitmapFactory())
);
builder.setSpatialIndex(new SpatialIndexColumnPartSupplier(rSpatialIndex));
}
return new DictionaryEncodedColumnPartSerde(
rVersion,
rFlags,
rDictionary,
rSingleValuedColumn,
rMultiValuedColumn,
bitmapSerdeFactory,
rBitmaps,
rSpatialIndex,
byteOrder
);
}
private WritableSupplier<IndexedInts> readSingleValuedColumn(VERSION version, ByteBuffer buffer)
{
switch (version) {
case UNCOMPRESSED_SINGLE_VALUE:
return VSizeIndexedInts.readFromByteBuffer(buffer).asWritableSupplier();
case COMPRESSED:
return CompressedVSizeIntsIndexedSupplier.fromByteBuffer(buffer, byteOrder);
}
throw new IAE("Unsupported single-value version[%s]", version);
}
private WritableSupplier<IndexedMultivalue<IndexedInts>> readMultiValuedColum(VERSION version, ByteBuffer buffer)
{
switch (version) {
case UNCOMPRESSED_MULTI_VALUE:
return VSizeIndexed.readFromByteBuffer(buffer).asWritableSupplier();
case COMPRESSED:
return CompressedVSizeIndexedSupplier.fromByteBuffer(buffer, byteOrder);
}
throw new IAE("Unsupported multi-value version[%s]", version);
}
@Override
public long numBytes()
{
return 1 + // version
(version.compareTo(VERSION.COMPRESSED) >= 0 ? Ints.BYTES : 0) + // flag if version >= compressed
size; // size of everything else (dictionary, bitmaps, column, spatialIndex)
private WritableSupplier<IndexedMultivalue<IndexedInts>> readMultiValuedColum(
VERSION version, ByteBuffer buffer, int flags
)
{
switch (version) {
case UNCOMPRESSED_MULTI_VALUE:
return VSizeIndexed.readFromByteBuffer(buffer).asWritableSupplier();
case COMPRESSED:
if (Feature.MULTI_VALUE.isSet(flags)) {
return CompressedVSizeIndexedSupplier.fromByteBuffer(buffer, byteOrder);
} else if (Feature.MULTI_VALUE_V3.isSet(flags)) {
return CompressedVSizeIndexedV3Supplier.fromByteBuffer(buffer, byteOrder);
} else {
throw new IAE("Unrecognized multi-value flag[%d]", flags);
}
}
throw new IAE("Unsupported multi-value version[%s]", version);
}
};
}
}

View File

@ -21,6 +21,7 @@ package io.druid.segment.serde;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.segment.FloatColumnSerializer;
import io.druid.segment.column.ColumnBuilder;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.column.ValueType;
@ -32,7 +33,7 @@ import java.nio.ByteOrder;
import java.nio.channels.WritableByteChannel;
/**
*/
*/
public class FloatGenericColumnPartSerde implements ColumnPartSerde
{
@JsonCreator
@ -40,16 +41,16 @@ public class FloatGenericColumnPartSerde implements ColumnPartSerde
@JsonProperty("byteOrder") ByteOrder byteOrder
)
{
return new FloatGenericColumnPartSerde(null, byteOrder);
return new FloatGenericColumnPartSerde(byteOrder, null);
}
private final CompressedFloatsIndexedSupplier compressedFloats;
private final ByteOrder byteOrder;
private Serializer serializer;
public FloatGenericColumnPartSerde(CompressedFloatsIndexedSupplier compressedFloats, ByteOrder byteOrder)
private FloatGenericColumnPartSerde(ByteOrder byteOrder, Serializer serializer)
{
this.compressedFloats = compressedFloats;
this.byteOrder = byteOrder;
this.serializer = serializer;
}
@JsonProperty
@ -58,27 +59,114 @@ public class FloatGenericColumnPartSerde implements ColumnPartSerde
return byteOrder;
}
@Override
public long numBytes()
public static SerializerBuilder serializerBuilder()
{
return compressedFloats.getSerializedSize();
return new SerializerBuilder();
}
public static class SerializerBuilder
{
private ByteOrder byteOrder = null;
private FloatColumnSerializer delegate = null;
public SerializerBuilder withByteOrder(final ByteOrder byteOrder)
{
this.byteOrder = byteOrder;
return this;
}
public SerializerBuilder withDelegate(final FloatColumnSerializer 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) throws IOException
{
delegate.writeToChannel(channel);
}
}
);
}
}
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) throws IOException
{
delegate.writeToChannel(channel);
}
}
);
}
}
@Override
public void write(WritableByteChannel channel) throws IOException
public Serializer getSerializer()
{
compressedFloats.writeToChannel(channel);
return serializer;
}
@Override
public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig)
public Deserializer getDeserializer()
{
final CompressedFloatsIndexedSupplier column = CompressedFloatsIndexedSupplier.fromByteBuffer(buffer, byteOrder);
builder.setType(ValueType.FLOAT)
.setHasMultipleValues(false)
.setGenericColumn(new FloatGenericColumnSupplier(column, byteOrder));
return new FloatGenericColumnPartSerde(column, byteOrder);
return new Deserializer()
{
@Override
public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig)
{
final CompressedFloatsIndexedSupplier column = CompressedFloatsIndexedSupplier.fromByteBuffer(
buffer,
byteOrder
);
builder.setType(ValueType.FLOAT)
.setHasMultipleValues(false)
.setGenericColumn(new FloatGenericColumnSupplier(column, byteOrder));
}
};
}
}

View File

@ -21,6 +21,7 @@ package io.druid.segment.serde;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.segment.LongColumnSerializer;
import io.druid.segment.column.ColumnBuilder;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.column.ValueType;
@ -32,7 +33,7 @@ import java.nio.ByteOrder;
import java.nio.channels.WritableByteChannel;
/**
*/
*/
public class LongGenericColumnPartSerde implements ColumnPartSerde
{
@JsonCreator
@ -40,16 +41,16 @@ public class LongGenericColumnPartSerde implements ColumnPartSerde
@JsonProperty("byteOrder") ByteOrder byteOrder
)
{
return new LongGenericColumnPartSerde(null, byteOrder);
return new LongGenericColumnPartSerde(byteOrder, null);
}
private final CompressedLongsIndexedSupplier compressedLongs;
private final ByteOrder byteOrder;
private Serializer serializer;
public LongGenericColumnPartSerde(CompressedLongsIndexedSupplier compressedLongs, ByteOrder byteOrder)
private LongGenericColumnPartSerde(ByteOrder byteOrder, Serializer serializer)
{
this.compressedLongs = compressedLongs;
this.byteOrder = byteOrder;
this.serializer = serializer;
}
@JsonProperty
@ -58,27 +59,114 @@ public class LongGenericColumnPartSerde implements ColumnPartSerde
return byteOrder;
}
@Override
public long numBytes()
public static SerializerBuilder serializerBuilder()
{
return compressedLongs.getSerializedSize();
return new SerializerBuilder();
}
public static class SerializerBuilder
{
private ByteOrder byteOrder = null;
private LongColumnSerializer delegate = null;
public SerializerBuilder withByteOrder(final ByteOrder byteOrder)
{
this.byteOrder = byteOrder;
return this;
}
public SerializerBuilder withDelegate(final LongColumnSerializer 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) throws IOException
{
delegate.writeToChannel(channel);
}
}
);
}
}
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) throws IOException
{
delegate.writeToChannel(channel);
}
}
);
}
}
@Override
public void write(WritableByteChannel channel) throws IOException
public Serializer getSerializer()
{
compressedLongs.writeToChannel(channel);
return serializer;
}
@Override
public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig)
public Deserializer getDeserializer()
{
final CompressedLongsIndexedSupplier column = CompressedLongsIndexedSupplier.fromByteBuffer(buffer, byteOrder);
builder.setType(ValueType.LONG)
.setHasMultipleValues(false)
.setGenericColumn(new LongGenericColumnSupplier(column));
return new LongGenericColumnPartSerde(column, byteOrder);
return new Deserializer()
{
@Override
public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig)
{
final CompressedLongsIndexedSupplier column = CompressedLongsIndexedSupplier.fromByteBuffer(
buffer,
byteOrder
);
builder.setType(ValueType.LONG)
.setHasMultipleValues(false)
.setGenericColumn(new LongGenericColumnSupplier(column));
}
};
}
}
}

View File

@ -0,0 +1,77 @@
/*
* 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.base.Function;
import com.google.common.collect.Iterables;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.CompressedVSizeIndexedSupplierTest;
import io.druid.segment.data.IndexedInts;
import io.druid.segment.data.IndexedMultivalue;
import io.druid.segment.data.VSizeIndexedInts;
import io.druid.segment.data.WritableSupplier;
import org.junit.After;
import org.junit.Before;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.util.Arrays;
public class CompressedVSizeIndexedV3SupplierTest extends CompressedVSizeIndexedSupplierTest
{
@Before
public void setUpSimple(){
vals = Arrays.asList(
new int[1],
new int[]{1, 2, 3, 4, 5},
new int[]{6, 7, 8, 9, 10},
new int[]{11, 12, 13, 14, 15, 16, 17, 18, 19, 20}
);
indexedSupplier = CompressedVSizeIndexedV3Supplier.fromIterable(
Iterables.transform(
vals,
new Function<int[], IndexedInts>()
{
@Override
public IndexedInts apply(int[] input)
{
return VSizeIndexedInts.fromArray(input, 20);
}
}
), 2, 20, ByteOrder.nativeOrder(),
CompressedObjectStrategy.CompressionStrategy.LZ4
);
}
@After
public void teardown(){
indexedSupplier = null;
vals = null;
}
@Override
protected WritableSupplier<IndexedMultivalue<IndexedInts>> fromByteBuffer(ByteBuffer buffer, ByteOrder order)
{
return CompressedVSizeIndexedV3Supplier.fromByteBuffer(
buffer, ByteOrder.nativeOrder()
);
}
}

View File

@ -37,6 +37,7 @@ import io.druid.segment.data.BitmapSerdeFactory;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.ConciseBitmapSerdeFactory;
import io.druid.segment.data.IncrementalIndexTest;
import io.druid.segment.data.IndexedInts;
import io.druid.segment.data.RoaringBitmapSerdeFactory;
import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.incremental.IncrementalIndexAdapter;
@ -54,6 +55,7 @@ import java.lang.reflect.Field;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@ -63,7 +65,7 @@ public class IndexMergerTest
@Rule
public final TemporaryFolder temporaryFolder = new TemporaryFolder();
private final static IndexMerger INDEX_MERGER = TestHelper.getTestIndexMerger();
protected final static IndexMerger INDEX_MERGER = TestHelper.getTestIndexMerger();
private final static IndexIO INDEX_IO = TestHelper.getTestIndexIO();
@Parameterized.Parameters(name = "{index}: bitmap={0}, metric compression={1}, dimension compression={2}")
@ -74,10 +76,12 @@ public class IndexMergerTest
ImmutableList.of(
ImmutableSet.of(new RoaringBitmapSerdeFactory(), new ConciseBitmapSerdeFactory()),
ImmutableSet.of(
CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED,
CompressedObjectStrategy.CompressionStrategy.LZ4,
CompressedObjectStrategy.CompressionStrategy.LZF
),
ImmutableSet.of(
CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED,
CompressedObjectStrategy.CompressionStrategy.LZ4,
CompressedObjectStrategy.CompressionStrategy.LZF
)
@ -151,6 +155,119 @@ public class IndexMergerTest
assertDimCompression(index, indexSpec.getDimensionCompressionStrategy());
}
@Test
public void testPersistWithDifferentDims() throws Exception
{
IncrementalIndex toPersist = IncrementalIndexTest.createIndex(null);
toPersist.add(
new MapBasedInputRow(
1,
Arrays.asList("dim1", "dim2"),
ImmutableMap.<String, Object>of("dim1", "1", "dim2", "2")
)
);
toPersist.add(
new MapBasedInputRow(
1,
Arrays.asList("dim1"),
ImmutableMap.<String, Object>of("dim1", "3")
)
);
final File tempDir = temporaryFolder.newFolder();
QueryableIndex index = closer.closeLater(
INDEX_IO.loadIndex(
INDEX_MERGER.persist(
toPersist,
tempDir,
null,
indexSpec
)
)
);
Assert.assertEquals(2, index.getColumn(Column.TIME_COLUMN_NAME).getLength());
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions()));
Assert.assertEquals(3, index.getColumnNames().size());
assertDimCompression(index, indexSpec.getDimensionCompressionStrategy());
final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(index);
Iterable<Rowboat> boats = adapter.getRows();
List<Rowboat> boatList = new ArrayList<>();
for (Rowboat boat : boats) {
boatList.add(boat);
}
Assert.assertEquals(2, boatList.size());
Assert.assertArrayEquals(new int[][]{{0}, {1}}, boatList.get(0).getDims());
Assert.assertArrayEquals(new int[][]{{1}, {0}}, boatList.get(1).getDims());
checkBitmapIndex(new ArrayList<Integer>(), adapter.getBitmapIndex("dim1", ""));
checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("dim1", "1"));
checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("dim1", "3"));
checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("dim2", ""));
checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("dim2", "2"));
}
@Test
public void testMergeRetainsValues() throws Exception
{
final long timestamp = System.currentTimeMillis();
IncrementalIndex toPersist1 = IncrementalIndexTest.createIndex(null);
IncrementalIndexTest.populateIndex(timestamp, toPersist1);
final File tempDir1 = temporaryFolder.newFolder();
final File mergedDir = temporaryFolder.newFolder();
final IndexableAdapter incrementalAdapter = new IncrementalIndexAdapter(
toPersist1.getInterval(),
toPersist1,
indexSpec.getBitmapSerdeFactory()
.getBitmapFactory()
);
QueryableIndex index1 = closer.closeLater(
INDEX_IO.loadIndex(
INDEX_MERGER.persist(
toPersist1,
tempDir1,
null,
indexSpec
)
)
);
final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1);
INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter);
Assert.assertEquals(2, index1.getColumn(Column.TIME_COLUMN_NAME).getLength());
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions()));
Assert.assertEquals(3, index1.getColumnNames().size());
QueryableIndex merged = closer.closeLater(
INDEX_IO.loadIndex(
INDEX_MERGER.mergeQueryableIndex(
ImmutableList.of(index1),
new AggregatorFactory[]{new CountAggregatorFactory("count")},
mergedDir,
indexSpec
)
)
);
Assert.assertEquals(2, merged.getColumn(Column.TIME_COLUMN_NAME).getLength());
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions()));
Assert.assertEquals(3, merged.getColumnNames().size());
INDEX_IO.validateTwoSegments(tempDir1, mergedDir);
assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy());
assertDimCompression(merged, indexSpec.getDimensionCompressionStrategy());
}
@Test
public void testPersistWithSegmentMetadata() throws Exception
{
@ -345,65 +462,6 @@ public class IndexMergerTest
assertDimCompression(merged, indexSpec.getDimensionCompressionStrategy());
}
@Test
public void testMergeRetainsValues() throws Exception
{
final long timestamp = System.currentTimeMillis();
IncrementalIndex toPersist1 = IncrementalIndexTest.createIndex(null);
IncrementalIndexTest.populateIndex(timestamp, toPersist1);
final File tempDir1 = temporaryFolder.newFolder();
final File mergedDir = temporaryFolder.newFolder();
final IndexableAdapter incrementalAdapter = new IncrementalIndexAdapter(
toPersist1.getInterval(),
toPersist1,
indexSpec.getBitmapSerdeFactory()
.getBitmapFactory()
);
QueryableIndex index1 = closer.closeLater(
INDEX_IO.loadIndex(
INDEX_MERGER.persist(
toPersist1,
tempDir1,
null,
indexSpec
)
)
);
final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1);
INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter);
Assert.assertEquals(2, index1.getColumn(Column.TIME_COLUMN_NAME).getLength());
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions()));
Assert.assertEquals(3, index1.getColumnNames().size());
QueryableIndex merged = closer.closeLater(
INDEX_IO.loadIndex(
INDEX_MERGER.mergeQueryableIndex(
ImmutableList.of(index1),
new AggregatorFactory[]{new CountAggregatorFactory("count")},
mergedDir,
indexSpec
)
)
);
Assert.assertEquals(2, merged.getColumn(Column.TIME_COLUMN_NAME).getLength());
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions()));
Assert.assertEquals(3, merged.getColumnNames().size());
INDEX_IO.validateTwoSegments(tempDir1, mergedDir);
assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy());
assertDimCompression(merged, indexSpec.getDimensionCompressionStrategy());
}
@Test
public void testAppendRetainsValues() throws Exception
{
@ -656,6 +714,9 @@ public class IndexMergerTest
throws Exception
{
// Java voodoo
if (expectedStrategy == null || expectedStrategy == CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED) {
return;
}
Object encodedColumn = index.getColumn("dim2").getDictionaryEncoding();
Field field = SimpleDictionaryEncodedColumn.class.getDeclaredField("column");
@ -666,6 +727,7 @@ public class IndexMergerTest
compressedSupplierField.setAccessible(true);
Object supplier = compressedSupplierField.get(obj);
Field compressionField = supplier.getClass().getDeclaredField("compression");
compressionField.setAccessible(true);
@ -811,6 +873,171 @@ public class IndexMergerTest
Assert.assertArrayEquals(new Object[]{1L}, boatList.get(3).getMetrics());
Assert.assertArrayEquals(new int[][]{{2}, {0}}, boatList.get(4).getDims());
Assert.assertArrayEquals(new Object[]{1L}, boatList.get(4).getMetrics());
checkBitmapIndex(Lists.newArrayList(0, 1, 2), adapter.getBitmapIndex("dimA", ""));
checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("dimA", "1"));
checkBitmapIndex(Lists.newArrayList(4), adapter.getBitmapIndex("dimA", "2"));
checkBitmapIndex(Lists.newArrayList(3, 4), adapter.getBitmapIndex("dimB", ""));
checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("dimB", "1"));
checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("dimB", "2"));
checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("dimB", "3"));
}
@Test
public void testJointDimMerge() throws Exception
{
// (d1, d2, d3) from only one index, and their dim values are ('empty', 'has null', 'no null')
// (d4, d5, d6, d7, d8, d9) are from both indexes
// d4: 'empty' join 'empty'
// d5: 'empty' join 'has null'
// d6: 'empty' join 'no null'
// d7: 'has null' join 'has null'
// d8: 'has null' join 'no null'
// d9: 'no null' join 'no null'
IncrementalIndex toPersistA = new OnheapIncrementalIndex(
0L,
QueryGranularity.NONE,
new AggregatorFactory[]{new CountAggregatorFactory("count")},
1000
);
toPersistA.add(
new MapBasedInputRow(
1,
Arrays.asList("d1", "d2", "d3", "d4", "d5", "d6", "d7", "d8", "d9"),
ImmutableMap.<String, Object>of(
"d1", "", "d2", "", "d3", "310", "d7", "", "d9", "910"
)
)
);
toPersistA.add(
new MapBasedInputRow(
2,
Arrays.asList("d1", "d2", "d3", "d4", "d5", "d6", "d7", "d8", "d9"),
ImmutableMap.<String, Object>of(
"d2", "210", "d3", "311", "d7", "710", "d8", "810", "d9", "911"
)
)
);
IncrementalIndex toPersistB = new OnheapIncrementalIndex(
0L,
QueryGranularity.NONE,
new AggregatorFactory[]{new CountAggregatorFactory("count")},
1000
);
toPersistB.add(
new MapBasedInputRow(
3,
Arrays.asList("d4", "d5", "d6", "d7", "d8", "d9"),
ImmutableMap.<String, Object>of(
"d5", "520", "d6", "620", "d7", "720", "d8", "820", "d9", "920"
)
)
);
toPersistB.add(
new MapBasedInputRow(
4,
Arrays.asList("d4", "d5", "d6", "d7", "d8", "d9"),
ImmutableMap.<String, Object>of(
"d5", "", "d6", "621", "d7", "", "d8", "821", "d9", "921"
)
)
);
final File tmpDirA = temporaryFolder.newFolder();
final File tmpDirB = temporaryFolder.newFolder();
final File tmpDirMerged = temporaryFolder.newFolder();
QueryableIndex indexA = closer.closeLater(
INDEX_IO.loadIndex(
INDEX_MERGER.persist(
toPersistA,
tmpDirA,
null,
indexSpec
)
)
);
QueryableIndex indexB = closer.closeLater(
INDEX_IO.loadIndex(
INDEX_MERGER.persist(
toPersistB,
tmpDirB,
null,
indexSpec
)
)
);
final QueryableIndex merged = closer.closeLater(
INDEX_IO.loadIndex(
INDEX_MERGER.mergeQueryableIndex(
Arrays.asList(indexA, indexB),
new AggregatorFactory[]{new CountAggregatorFactory("count")},
tmpDirMerged,
indexSpec
)
)
);
final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(merged);
Iterable<Rowboat> boats = adapter.getRows();
List<Rowboat> boatList = new ArrayList<>();
for (Rowboat boat : boats) {
boatList.add(boat);
}
Assert.assertEquals(
ImmutableList.of("d2", "d3", "d5", "d6", "d7", "d8", "d9"),
ImmutableList.copyOf(adapter.getDimensionNames())
);
Assert.assertEquals(4, boatList.size());
Assert.assertArrayEquals(new int[][]{{0}, {1}, {0}, {0}, {0}, {0}, {0}}, boatList.get(0).getDims());
Assert.assertArrayEquals(new int[][]{{1}, {2}, {0}, {0}, {1}, {1}, {1}}, boatList.get(1).getDims());
Assert.assertArrayEquals(new int[][]{{0}, {0}, {1}, {1}, {2}, {2}, {2}}, boatList.get(2).getDims());
Assert.assertArrayEquals(new int[][]{{0}, {0}, {0}, {2}, {0}, {3}, {3}}, boatList.get(3).getDims());
checkBitmapIndex(Lists.newArrayList(0, 2, 3), adapter.getBitmapIndex("d2", ""));
checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d2", "210"));
checkBitmapIndex(Lists.newArrayList(2, 3), adapter.getBitmapIndex("d3", ""));
checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("d3", "310"));
checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d3", "311"));
checkBitmapIndex(Lists.newArrayList(0, 1, 3), adapter.getBitmapIndex("d5", ""));
checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d5", "520"));
checkBitmapIndex(Lists.newArrayList(0, 1), adapter.getBitmapIndex("d6", ""));
checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d6", "620"));
checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("d6", "621"));
checkBitmapIndex(Lists.newArrayList(0, 3), adapter.getBitmapIndex("d7", ""));
checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d7", "710"));
checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d7", "720"));
checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("d8", ""));
checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d8", "810"));
checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d8", "820"));
checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("d8", "821"));
checkBitmapIndex(new ArrayList<Integer>(), adapter.getBitmapIndex("d9", ""));
checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("d9", "910"));
checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d9", "911"));
checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d9", "920"));
checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("d9", "921"));
}
private void checkBitmapIndex(ArrayList<Integer> expectIndex, IndexedInts index)
{
Assert.assertEquals(expectIndex.size(), index.size());
int i = 0;
Iterator it = index.iterator();
while (it.hasNext()) {
Assert.assertEquals(expectIndex.get(i), it.next());
i++;
}
}
private IncrementalIndex getIndexD3() throws Exception

View File

@ -0,0 +1,310 @@
/*
* 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.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.Files;
import io.druid.common.utils.JodaUtils;
import io.druid.data.input.InputRow;
import io.druid.data.input.MapBasedInputRow;
import io.druid.granularity.QueryGranularity;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.CountAggregatorFactory;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.ConciseBitmapSerdeFactory;
import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.incremental.OnheapIncrementalIndex;
import org.apache.commons.io.FileUtils;
import org.joda.time.DateTime;
import org.junit.After;
import org.junit.Assert;
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.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@RunWith(Parameterized.class)
public class IndexMergerV9CompatibilityTest
{
@Rule
public final CloserRule closer = new CloserRule(false);
private static final long TIMESTAMP = DateTime.parse("2014-01-01").getMillis();
private static final AggregatorFactory[] DEFAULT_AGG_FACTORIES = new AggregatorFactory[]{
new CountAggregatorFactory(
"count"
)
};
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(),
CompressedObjectStrategy.CompressionStrategy.LZ4,
CompressedObjectStrategy.CompressionStrategy.LZ4
);
private static final List<String> DIMS = ImmutableList.of("dim0", "dim1");
private static final Function<Collection<Map<String, Object>>, Object[]> OBJECT_MAKER = new Function<Collection<Map<String, Object>>, Object[]>()
{
@Nullable
@Override
public Object[] apply(Collection<Map<String, Object>> input)
{
final ArrayList<InputRow> list = new ArrayList<>();
int i = 0;
for (final Map<String, Object> map : input) {
list.add(new MapBasedInputRow(TIMESTAMP + i++, DIMS, map));
}
return new Object[]{list};
}
};
@SafeVarargs
public static Collection<Object[]> permute(Map<String, Object>... maps)
{
if (maps == null) {
return ImmutableList.<Object[]>of();
}
return Collections2.transform(
Collections2.permutations(
Arrays.asList(maps)
),
OBJECT_MAKER
);
}
@Parameterized.Parameters
public static Iterable<Object[]> paramFeeder()
{
final Map<String, Object> map1 = ImmutableMap.<String, Object>of(
DIMS.get(0), ImmutableList.<String>of("dim00", "dim01"),
DIMS.get(1), "dim10"
);
final List<String> nullList = Collections.singletonList(null);
final Map<String, Object> map2 = ImmutableMap.<String, Object>of(
DIMS.get(0), nullList,
DIMS.get(1), "dim10"
);
final Map<String, Object> map3 = ImmutableMap.<String, Object>of(
DIMS.get(0),
ImmutableList.<String>of("dim00", "dim01")
);
final Map<String, Object> map4 = ImmutableMap.<String, Object>of();
final Map<String, Object> map5 = ImmutableMap.<String, Object>of(DIMS.get(1), "dim10");
final Map<String, Object> map6 = new HashMap<>();
map6.put(DIMS.get(1), null); // ImmutableMap cannot take null
return Iterables.<Object[]>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<InputRow> events;
public IndexMergerV9CompatibilityTest(
final Collection<InputRow> events
)
{
this.events = events;
}
IncrementalIndex toPersist;
File tmpDir;
File persistTmpDir;
@Before
public void setUp() throws IOException
{
toPersist = new OnheapIncrementalIndex(
JodaUtils.MIN_INSTANT,
QueryGranularity.NONE,
DEFAULT_AGG_FACTORIES,
1000000
);
for (InputRow event : events) {
toPersist.add(event);
}
tmpDir = Files.createTempDir();
persistTmpDir = new File(tmpDir, "persistDir");
INDEX_MERGER.persist(toPersist, persistTmpDir, null, INDEX_SPEC);
}
@After
public void tearDown() throws IOException
{
FileUtils.deleteDirectory(tmpDir);
}
@Test
public void testPersistWithSegmentMetadata() throws IOException
{
File outDir = Files.createTempDir();
QueryableIndex index = null;
try {
outDir = Files.createTempDir();
Map<String, Object> segmentMetadata = ImmutableMap.<String, Object>of("key", "value");
index = INDEX_IO.loadIndex(INDEX_MERGER_V9.persist(toPersist, outDir, segmentMetadata, INDEX_SPEC));
Assert.assertEquals(segmentMetadata, index.getMetaData());
}
finally {
if (index != null) {
index.close();
;
}
if (outDir != null) {
FileUtils.deleteDirectory(outDir);
}
}
}
@Test
public void testSimpleReprocess() throws IOException
{
final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(
closer.closeLater(
INDEX_IO.loadIndex(
persistTmpDir
)
)
);
Assert.assertEquals(events.size(), adapter.getNumRows());
reprocessAndValidate(persistTmpDir, new File(tmpDir, "reprocessed"));
}
private File reprocessAndValidate(File inDir, File tmpDir) throws IOException
{
final File outDir = INDEX_MERGER_V9.convert(
inDir,
tmpDir,
INDEX_SPEC
);
INDEX_IO.validateTwoSegments(persistTmpDir, outDir);
return outDir;
}
private File appendAndValidate(File inDir, File tmpDir) throws IOException
{
final File outDir = INDEX_MERGER.append(
ImmutableList.<IndexableAdapter>of(new QueryableIndexIndexableAdapter(closer.closeLater(INDEX_IO.loadIndex(inDir)))),
tmpDir,
INDEX_SPEC
);
INDEX_IO.validateTwoSegments(persistTmpDir, outDir);
return outDir;
}
@Test
public void testIdempotentReprocess() 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");
reprocessAndValidate(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());
reprocessAndValidate(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());
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);
}
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,707 @@
/*
* 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.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.metamx.collections.spatial.search.RadiusBound;
import com.metamx.collections.spatial.search.RectangularBound;
import io.druid.data.input.MapBasedInputRow;
import io.druid.data.input.impl.DimensionsSpec;
import io.druid.data.input.impl.SpatialDimensionSchema;
import io.druid.granularity.QueryGranularity;
import io.druid.query.Druids;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.Result;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.CountAggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory;
import io.druid.query.filter.SpatialDimFilter;
import io.druid.query.timeseries.TimeseriesQuery;
import io.druid.query.timeseries.TimeseriesQueryEngine;
import io.druid.query.timeseries.TimeseriesQueryQueryToolChest;
import io.druid.query.timeseries.TimeseriesQueryRunnerFactory;
import io.druid.query.timeseries.TimeseriesResultValue;
import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.incremental.IncrementalIndexSchema;
import io.druid.segment.incremental.OnheapIncrementalIndex;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.File;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.Random;
/**
*/
@RunWith(Parameterized.class)
public class IndexMergerV9WithSpatialIndexTest
{
private static IndexMergerV9 INDEX_MERGER_V9 = TestHelper.getTestIndexMergerV9();
private static IndexIO INDEX_IO = TestHelper.getTestIndexIO();
public static final int NUM_POINTS = 5000;
private static Interval DATA_INTERVAL = new Interval("2013-01-01/2013-01-07");
private static AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{
new CountAggregatorFactory("rows"),
new LongSumAggregatorFactory("val", "val")
};
private static List<String> DIMS = Lists.newArrayList("dim", "lat", "long", "lat2", "long2");
@Parameterized.Parameters
public static Collection<?> constructorFeeder() throws IOException
{
final IndexSpec indexSpec = new IndexSpec();
final IncrementalIndex rtIndex = makeIncrementalIndex();
final QueryableIndex mMappedTestIndex = makeQueryableIndex(indexSpec);
final QueryableIndex mergedRealtimeIndex = makeMergedQueryableIndex(indexSpec);
return Arrays.asList(
new Object[][]{
{
new IncrementalIndexSegment(rtIndex, null)
},
{
new QueryableIndexSegment(null, mMappedTestIndex)
},
{
new QueryableIndexSegment(null, mergedRealtimeIndex)
}
}
);
}
private static IncrementalIndex makeIncrementalIndex() throws IOException
{
IncrementalIndex theIndex = new OnheapIncrementalIndex(
new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis())
.withQueryGranularity(QueryGranularity.DAY)
.withMetrics(METRIC_AGGS)
.withDimensionsSpec(
new DimensionsSpec(
null,
null,
Arrays.asList(
new SpatialDimensionSchema(
"dim.geo",
Arrays.asList("lat", "long")
),
new SpatialDimensionSchema(
"spatialIsRad",
Arrays.asList("lat2", "long2")
)
)
)
).build(),
false,
NUM_POINTS
);
theIndex.add(
new MapBasedInputRow(
new DateTime("2013-01-01").getMillis(),
DIMS,
ImmutableMap.<String, Object>of(
"timestamp", new DateTime("2013-01-01").toString(),
"dim", "foo",
"lat", 0.0f,
"long", 0.0f,
"val", 17L
)
)
);
theIndex.add(
new MapBasedInputRow(
new DateTime("2013-01-02").getMillis(),
DIMS,
ImmutableMap.<String, Object>of(
"timestamp", new DateTime("2013-01-02").toString(),
"dim", "foo",
"lat", 1.0f,
"long", 3.0f,
"val", 29L
)
)
);
theIndex.add(
new MapBasedInputRow(
new DateTime("2013-01-03").getMillis(),
DIMS,
ImmutableMap.<String, Object>of(
"timestamp", new DateTime("2013-01-03").toString(),
"dim", "foo",
"lat", 4.0f,
"long", 2.0f,
"val", 13L
)
)
);
theIndex.add(
new MapBasedInputRow(
new DateTime("2013-01-04").getMillis(),
DIMS,
ImmutableMap.<String, Object>of(
"timestamp", new DateTime("2013-01-04").toString(),
"dim", "foo",
"lat", 7.0f,
"long", 3.0f,
"val", 91L
)
)
);
theIndex.add(
new MapBasedInputRow(
new DateTime("2013-01-05").getMillis(),
DIMS,
ImmutableMap.<String, Object>of(
"timestamp", new DateTime("2013-01-05").toString(),
"dim", "foo",
"lat", 8.0f,
"long", 6.0f,
"val", 47L
)
)
);
theIndex.add(
new MapBasedInputRow(
new DateTime("2013-01-05").getMillis(),
DIMS,
ImmutableMap.<String, Object>of(
"timestamp", new DateTime("2013-01-05").toString(),
"dim", "foo",
"lat", "_mmx.unknown",
"long", "_mmx.unknown",
"val", 101L
)
)
);
theIndex.add(
new MapBasedInputRow(
new DateTime("2013-01-05").getMillis(),
DIMS,
ImmutableMap.<String, Object>of(
"timestamp", new DateTime("2013-01-05").toString(),
"dim", "foo",
"dim.geo", "_mmx.unknown",
"val", 501L
)
)
);
theIndex.add(
new MapBasedInputRow(
new DateTime("2013-01-05").getMillis(),
DIMS,
ImmutableMap.<String, Object>of(
"timestamp", new DateTime("2013-01-05").toString(),
"lat2", 0.0f,
"long2", 0.0f,
"val", 13L
)
)
);
// Add a bunch of random points
Random rand = new Random();
for (int i = 8; i < NUM_POINTS; i++) {
theIndex.add(
new MapBasedInputRow(
new DateTime("2013-01-01").getMillis(),
DIMS,
ImmutableMap.<String, Object>of(
"timestamp", new DateTime("2013-01-01").toString(),
"dim", "boo",
"lat", (float) (rand.nextFloat() * 10 + 10.0),
"long", (float) (rand.nextFloat() * 10 + 10.0),
"val", i
)
)
);
}
return theIndex;
}
private static QueryableIndex makeQueryableIndex(IndexSpec indexSpec) throws IOException
{
IncrementalIndex theIndex = makeIncrementalIndex();
File tmpFile = File.createTempFile("billy", "yay");
tmpFile.delete();
tmpFile.mkdirs();
tmpFile.deleteOnExit();
INDEX_MERGER_V9.persist(theIndex, tmpFile, null, indexSpec);
return INDEX_IO.loadIndex(tmpFile);
}
private static QueryableIndex makeMergedQueryableIndex(IndexSpec indexSpec)
{
try {
IncrementalIndex first = new OnheapIncrementalIndex(
new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis())
.withQueryGranularity(QueryGranularity.DAY)
.withMetrics(METRIC_AGGS)
.withDimensionsSpec(
new DimensionsSpec(
null,
null,
Arrays.asList(
new SpatialDimensionSchema(
"dim.geo",
Arrays.asList("lat", "long")
),
new SpatialDimensionSchema(
"spatialIsRad",
Arrays.asList("lat2", "long2")
)
)
)
).build(),
false,
1000
);
IncrementalIndex second = new OnheapIncrementalIndex(
new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis())
.withQueryGranularity(QueryGranularity.DAY)
.withMetrics(METRIC_AGGS)
.withDimensionsSpec(
new DimensionsSpec(
null,
null,
Arrays.asList(
new SpatialDimensionSchema(
"dim.geo",
Arrays.asList("lat", "long")
),
new SpatialDimensionSchema(
"spatialIsRad",
Arrays.asList("lat2", "long2")
)
)
)
).build(),
false,
1000
);
IncrementalIndex third = new OnheapIncrementalIndex(
new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis())
.withQueryGranularity(QueryGranularity.DAY)
.withMetrics(METRIC_AGGS)
.withDimensionsSpec(
new DimensionsSpec(
null,
null,
Arrays.asList(
new SpatialDimensionSchema(
"dim.geo",
Arrays.asList("lat", "long")
),
new SpatialDimensionSchema(
"spatialIsRad",
Arrays.asList("lat2", "long2")
)
)
)
).build(),
false,
NUM_POINTS
);
first.add(
new MapBasedInputRow(
new DateTime("2013-01-01").getMillis(),
DIMS,
ImmutableMap.<String, Object>of(
"timestamp", new DateTime("2013-01-01").toString(),
"dim", "foo",
"lat", 0.0f,
"long", 0.0f,
"val", 17L
)
)
);
first.add(
new MapBasedInputRow(
new DateTime("2013-01-02").getMillis(),
DIMS,
ImmutableMap.<String, Object>of(
"timestamp", new DateTime("2013-01-02").toString(),
"dim", "foo",
"lat", 1.0f,
"long", 3.0f,
"val", 29L
)
)
);
first.add(
new MapBasedInputRow(
new DateTime("2013-01-03").getMillis(),
DIMS,
ImmutableMap.<String, Object>of(
"timestamp", new DateTime("2013-01-03").toString(),
"dim", "foo",
"lat", 4.0f,
"long", 2.0f,
"val", 13L
)
)
);
first.add(
new MapBasedInputRow(
new DateTime("2013-01-05").getMillis(),
DIMS,
ImmutableMap.<String, Object>of(
"timestamp", new DateTime("2013-01-05").toString(),
"dim", "foo",
"lat", "_mmx.unknown",
"long", "_mmx.unknown",
"val", 101L
)
)
);
first.add(
new MapBasedInputRow(
new DateTime("2013-01-05").getMillis(),
DIMS,
ImmutableMap.<String, Object>of(
"timestamp", new DateTime("2013-01-05").toString(),
"dim", "foo",
"dim.geo", "_mmx.unknown",
"val", 501L
)
)
);
second.add(
new MapBasedInputRow(
new DateTime("2013-01-04").getMillis(),
DIMS,
ImmutableMap.<String, Object>of(
"timestamp", new DateTime("2013-01-04").toString(),
"dim", "foo",
"lat", 7.0f,
"long", 3.0f,
"val", 91L
)
)
);
second.add(
new MapBasedInputRow(
new DateTime("2013-01-05").getMillis(),
DIMS,
ImmutableMap.<String, Object>of(
"timestamp", new DateTime("2013-01-05").toString(),
"dim", "foo",
"lat", 8.0f,
"long", 6.0f,
"val", 47L
)
)
);
second.add(
new MapBasedInputRow(
new DateTime("2013-01-05").getMillis(),
DIMS,
ImmutableMap.<String, Object>of(
"timestamp", new DateTime("2013-01-05").toString(),
"lat2", 0.0f,
"long2", 0.0f,
"val", 13L
)
)
);
// Add a bunch of random points
Random rand = new Random();
for (int i = 8; i < NUM_POINTS; i++) {
third.add(
new MapBasedInputRow(
new DateTime("2013-01-01").getMillis(),
DIMS,
ImmutableMap.<String, Object>of(
"timestamp", new DateTime("2013-01-01").toString(),
"dim", "boo",
"lat", (float) (rand.nextFloat() * 10 + 10.0),
"long", (float) (rand.nextFloat() * 10 + 10.0),
"val", i
)
)
);
}
File tmpFile = File.createTempFile("yay", "who");
tmpFile.delete();
File firstFile = new File(tmpFile, "first");
File secondFile = new File(tmpFile, "second");
File thirdFile = new File(tmpFile, "third");
File mergedFile = new File(tmpFile, "merged");
firstFile.mkdirs();
firstFile.deleteOnExit();
secondFile.mkdirs();
secondFile.deleteOnExit();
thirdFile.mkdirs();
thirdFile.deleteOnExit();
mergedFile.mkdirs();
mergedFile.deleteOnExit();
INDEX_MERGER_V9.persist(first, DATA_INTERVAL, firstFile, null, indexSpec);
INDEX_MERGER_V9.persist(second, DATA_INTERVAL, secondFile, null, indexSpec);
INDEX_MERGER_V9.persist(third, DATA_INTERVAL, thirdFile, null, indexSpec);
QueryableIndex mergedRealtime = INDEX_IO.loadIndex(
INDEX_MERGER_V9.mergeQueryableIndex(
Arrays.asList(
INDEX_IO.loadIndex(firstFile),
INDEX_IO.loadIndex(secondFile),
INDEX_IO.loadIndex(thirdFile)
),
METRIC_AGGS,
mergedFile,
indexSpec
)
);
return mergedRealtime;
}
catch (IOException e) {
throw Throwables.propagate(e);
}
}
private final Segment segment;
public IndexMergerV9WithSpatialIndexTest(Segment segment)
{
this.segment = segment;
}
@Test
public void testSpatialQuery()
{
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource("test")
.granularity(QueryGranularity.ALL)
.intervals(Arrays.asList(new Interval("2013-01-01/2013-01-07")))
.filters(
new SpatialDimFilter(
"dim.geo",
new RadiusBound(new float[]{0.0f, 0.0f}, 5)
)
)
.aggregators(
Arrays.<AggregatorFactory>asList(
new CountAggregatorFactory("rows"),
new LongSumAggregatorFactory("val", "val")
)
)
.build();
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
new Result<>(
new DateTime("2013-01-01T00:00:00.000Z"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>builder()
.put("rows", 3L)
.put("val", 59L)
.build()
)
)
);
try {
TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
QueryRunner runner = new FinalizeResultsQueryRunner(
factory.createRunner(segment),
factory.getToolchest()
);
TestHelper.assertExpectedResults(expectedResults, runner.run(query, Maps.newHashMap()));
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
@Test
public void testSpatialQueryWithOtherSpatialDim()
{
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource("test")
.granularity(QueryGranularity.ALL)
.intervals(Arrays.asList(new Interval("2013-01-01/2013-01-07")))
.filters(
new SpatialDimFilter(
"spatialIsRad",
new RadiusBound(new float[]{0.0f, 0.0f}, 5)
)
)
.aggregators(
Arrays.<AggregatorFactory>asList(
new CountAggregatorFactory("rows"),
new LongSumAggregatorFactory("val", "val")
)
)
.build();
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
new Result<>(
new DateTime("2013-01-01T00:00:00.000Z"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>builder()
.put("rows", 1L)
.put("val", 13L)
.build()
)
)
);
try {
TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
QueryRunner runner = new FinalizeResultsQueryRunner(
factory.createRunner(segment),
factory.getToolchest()
);
TestHelper.assertExpectedResults(expectedResults, runner.run(query, Maps.newHashMap()));
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
@Test
public void testSpatialQueryMorePoints()
{
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource("test")
.granularity(QueryGranularity.DAY)
.intervals(Arrays.asList(new Interval("2013-01-01/2013-01-07")))
.filters(
new SpatialDimFilter(
"dim.geo",
new RectangularBound(new float[]{0.0f, 0.0f}, new float[]{9.0f, 9.0f})
)
)
.aggregators(
Arrays.<AggregatorFactory>asList(
new CountAggregatorFactory("rows"),
new LongSumAggregatorFactory("val", "val")
)
)
.build();
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
new Result<>(
new DateTime("2013-01-01T00:00:00.000Z"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>builder()
.put("rows", 1L)
.put("val", 17L)
.build()
)
),
new Result<>(
new DateTime("2013-01-02T00:00:00.000Z"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>builder()
.put("rows", 1L)
.put("val", 29L)
.build()
)
),
new Result<>(
new DateTime("2013-01-03T00:00:00.000Z"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>builder()
.put("rows", 1L)
.put("val", 13L)
.build()
)
),
new Result<>(
new DateTime("2013-01-04T00:00:00.000Z"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>builder()
.put("rows", 1L)
.put("val", 91L)
.build()
)
),
new Result<>(
new DateTime("2013-01-05T00:00:00.000Z"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>builder()
.put("rows", 1L)
.put("val", 47L)
.build()
)
)
);
try {
TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
QueryRunner runner = new FinalizeResultsQueryRunner(
factory.createRunner(segment),
factory.getToolchest()
);
TestHelper.assertExpectedResults(expectedResults, runner.run(query, Maps.newHashMap()));
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
}

View File

@ -36,6 +36,7 @@ import java.util.Iterator;
public class TestHelper
{
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 = new DefaultObjectMapper();
@ -52,6 +53,7 @@ public class TestHelper
}
);
INDEX_MERGER = new IndexMerger(JSON_MAPPER, INDEX_IO);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO);
}
@ -60,6 +62,11 @@ public class TestHelper
return INDEX_MERGER;
}
public static IndexMergerV9 getTestIndexMergerV9()
{
return INDEX_MERGER_V9;
}
public static IndexIO getTestIndexIO()
{
return INDEX_IO;

View File

@ -79,6 +79,8 @@ public class CompressedFloatsSupplierSerializerTest extends CompressionStrategyT
}
);
Assert.assertEquals(baos.size(), serializer.getSerializedSize());
IndexedFloats floats = CompressedFloatsIndexedSupplier
.fromByteBuffer(ByteBuffer.wrap(baos.toByteArray()), order)
.get();
@ -90,4 +92,45 @@ public class CompressedFloatsSupplierSerializerTest extends CompressionStrategyT
floats.close();
}
@Test
public void testEmpty() throws Exception
{
final ByteOrder order = ByteOrder.nativeOrder();
final int sizePer = 999;
CompressedFloatsSupplierSerializer serializer = new CompressedFloatsSupplierSerializer(
sizePer,
new GenericIndexedWriter<ResourceHolder<FloatBuffer>>(
new IOPeonForTesting(),
"test",
CompressedFloatBufferObjectStrategy.getBufferForOrder(
order,
compressionStrategy,
sizePer
)
),
compressionStrategy
);
serializer.open();
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
serializer.closeAndConsolidate(
new OutputSupplier<OutputStream>()
{
@Override
public OutputStream getOutput() throws IOException
{
return baos;
}
}
);
Assert.assertEquals(baos.size(), serializer.getSerializedSize());
IndexedFloats floats = CompressedFloatsIndexedSupplier
.fromByteBuffer(ByteBuffer.wrap(baos.toByteArray()), order)
.get();
Assert.assertEquals(0, floats.size());
floats.close();
}
}

View File

@ -0,0 +1,168 @@
/*
* 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.data;
import com.google.common.base.Function;
import com.google.common.collect.Iterables;
import com.google.common.collect.Sets;
import com.google.common.primitives.Ints;
import com.metamx.common.guava.CloseQuietly;
import org.apache.commons.io.IOUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.nio.channels.Channels;
import java.nio.channels.WritableByteChannel;
import java.util.List;
import java.util.Random;
import java.util.Set;
import static org.junit.Assert.assertEquals;
@RunWith(Parameterized.class)
public class CompressedIntsIndexedWriterTest
{
@Parameterized.Parameters(name = "{index}: compression={0}, byteOrder={1}")
public static Iterable<Object[]> compressionStrategiesAndByteOrders()
{
Set<List<Object>> combinations = Sets.cartesianProduct(
Sets.newHashSet(CompressedObjectStrategy.CompressionStrategy.values()),
Sets.newHashSet(ByteOrder.BIG_ENDIAN, ByteOrder.LITTLE_ENDIAN)
);
return Iterables.transform(
combinations, new Function<List, Object[]>()
{
@Override
public Object[] apply(List input)
{
return new Object[]{input.get(0), input.get(1)};
}
}
);
}
private static final int[] MAX_VALUES = new int[]{0xFF, 0xFFFF, 0xFFFFFF, 0x0FFFFFFF};
private static final int[] CHUNK_FACTORS = new int[]{1, 2, 100, CompressedIntsIndexedSupplier.MAX_INTS_IN_BUFFER};
private final IOPeon ioPeon = new TmpFileIOPeon();
private final CompressedObjectStrategy.CompressionStrategy compressionStrategy;
private final ByteOrder byteOrder;
private final Random rand = new Random(0);
private int[] vals;
public CompressedIntsIndexedWriterTest(
CompressedObjectStrategy.CompressionStrategy compressionStrategy,
ByteOrder byteOrder
)
{
this.compressionStrategy = compressionStrategy;
this.byteOrder = byteOrder;
}
@Before
public void setUp() throws Exception
{
vals = null;
}
@After
public void tearDown() throws Exception
{
ioPeon.cleanup();
}
private void generateVals(final int totalSize, final int maxValue) throws IOException
{
vals = new int[totalSize];
for (int i = 0; i < vals.length; ++i) {
vals[i] = rand.nextInt(maxValue);
}
}
private void checkSerializedSizeAndData(int chunkFactor) throws Exception
{
CompressedIntsIndexedWriter writer = new CompressedIntsIndexedWriter(
ioPeon, "test", chunkFactor, byteOrder, compressionStrategy
);
CompressedIntsIndexedSupplier supplierFromList = CompressedIntsIndexedSupplier.fromList(
Ints.asList(vals), chunkFactor, byteOrder, compressionStrategy
);
writer.open();
for (int val : vals) {
writer.add(val);
}
writer.close();
long writtenLength = writer.getSerializedSize();
final WritableByteChannel outputChannel = Channels.newChannel(ioPeon.makeOutputStream("output"));
writer.writeToChannel(outputChannel);
outputChannel.close();
assertEquals(writtenLength, supplierFromList.getSerializedSize());
// read from ByteBuffer and check values
CompressedIntsIndexedSupplier supplierFromByteBuffer = CompressedIntsIndexedSupplier.fromByteBuffer(
ByteBuffer.wrap(IOUtils.toByteArray(ioPeon.makeInputStream("output"))), byteOrder
);
IndexedInts indexedInts = supplierFromByteBuffer.get();
assertEquals(vals.length, indexedInts.size());
for (int i = 0; i < vals.length; ++i) {
assertEquals(vals[i], indexedInts.get(i));
}
CloseQuietly.close(indexedInts);
}
@Test
public void testSmallData() throws Exception
{
// less than one chunk
for (int maxValue : MAX_VALUES) {
for (int chunkFactor : CHUNK_FACTORS) {
generateVals(rand.nextInt(chunkFactor), maxValue);
checkSerializedSizeAndData(chunkFactor);
}
}
}
@Test
public void testLargeData() throws Exception
{
// more than one chunk
for (int maxValue : MAX_VALUES) {
for (int chunkFactor : CHUNK_FACTORS) {
generateVals((rand.nextInt(5) + 5) * chunkFactor + rand.nextInt(chunkFactor), maxValue);
checkSerializedSizeAndData(chunkFactor);
}
}
}
@Test
public void testWriteEmpty() throws Exception
{
vals = new int[0];
checkSerializedSizeAndData(2);
}
}

View File

@ -75,6 +75,8 @@ public class CompressedLongsSupplierSerializerTest extends CompressionStrategyTe
}
);
Assert.assertEquals(baos.size(), serializer.getSerializedSize());
IndexedLongs longs = CompressedLongsIndexedSupplier
.fromByteBuffer(ByteBuffer.wrap(baos.toByteArray()), order)
.get();
@ -85,4 +87,40 @@ public class CompressedLongsSupplierSerializerTest extends CompressionStrategyTe
}
longs.close();
}
@Test
public void testEmpty() throws Exception
{
final ByteOrder order = ByteOrder.nativeOrder();
final int sizePer = 999;
CompressedLongsSupplierSerializer serializer = new CompressedLongsSupplierSerializer(
sizePer,
new GenericIndexedWriter<ResourceHolder<LongBuffer>>(
new IOPeonForTesting(),
"test",
CompressedLongBufferObjectStrategy.getBufferForOrder(order, compressionStrategy, sizePer)
),
compressionStrategy
);
serializer.open();
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
serializer.closeAndConsolidate(
new OutputSupplier<OutputStream>()
{
@Override
public OutputStream getOutput() throws IOException
{
return baos;
}
}
);
Assert.assertEquals(baos.size(), serializer.getSerializedSize());
IndexedLongs longs = CompressedLongsIndexedSupplier
.fromByteBuffer(ByteBuffer.wrap(baos.toByteArray()), order)
.get();
Assert.assertEquals(0, longs.size());
longs.close();
}
}

View File

@ -40,12 +40,13 @@ import java.util.List;
*/
public class CompressedVSizeIndexedSupplierTest
{
private List<int[]> vals;
protected List<int[]> vals;
private CompressedVSizeIndexedSupplier indexedSupplier;
protected WritableSupplier<IndexedMultivalue<IndexedInts>> indexedSupplier;
@Before
public void setUpSimple(){
public void setUpSimple()
{
vals = Arrays.asList(
new int[1],
new int[]{1, 2, 3, 4, 5},
@ -70,7 +71,8 @@ public class CompressedVSizeIndexedSupplierTest
}
@After
public void teardown(){
public void teardown()
{
indexedSupplier = null;
vals = null;
}
@ -89,7 +91,7 @@ public class CompressedVSizeIndexedSupplierTest
final byte[] bytes = baos.toByteArray();
Assert.assertEquals(indexedSupplier.getSerializedSize(), bytes.length);
CompressedVSizeIndexedSupplier deserializedIndexed = CompressedVSizeIndexedSupplier.fromByteBuffer(
WritableSupplier<IndexedMultivalue<IndexedInts>> deserializedIndexed = fromByteBuffer(
ByteBuffer.wrap(bytes),
ByteOrder.nativeOrder()
);
@ -98,26 +100,28 @@ public class CompressedVSizeIndexedSupplierTest
}
@Test(expected = IllegalArgumentException.class)
public void testGetInvalidElementInRow(){
public void testGetInvalidElementInRow()
{
indexedSupplier.get().get(3).get(15);
}
@Test
public void testIterators(){
public void testIterators()
{
Iterator<IndexedInts> iterator = indexedSupplier.get().iterator();
int row = 0;
while(iterator.hasNext()){
while (iterator.hasNext()) {
final int[] ints = vals.get(row);
final IndexedInts vSizeIndexedInts = iterator.next();
Assert.assertEquals(ints.length, vSizeIndexedInts.size());
Iterator<Integer> valsIterator = vSizeIndexedInts.iterator();
int j=0;
while(valsIterator.hasNext()){
Assert.assertEquals((Integer)ints[j], valsIterator.next());
int j = 0;
while (valsIterator.hasNext()) {
Assert.assertEquals((Integer) ints[j], valsIterator.next());
j++;
}
row ++;
row++;
}
}
@ -134,4 +138,11 @@ public class CompressedVSizeIndexedSupplierTest
}
}
}
protected WritableSupplier<IndexedMultivalue<IndexedInts>> fromByteBuffer(ByteBuffer buffer, ByteOrder order)
{
return CompressedVSizeIndexedSupplier.fromByteBuffer(
buffer, ByteOrder.nativeOrder()
);
}
}

View File

@ -0,0 +1,220 @@
/*
* 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.data;
import com.google.common.base.Function;
import com.google.common.collect.Iterables;
import com.google.common.collect.Ordering;
import com.google.common.collect.Sets;
import com.google.common.primitives.Ints;
import com.metamx.common.guava.CloseQuietly;
import io.druid.segment.CompressedVSizeIndexedV3Supplier;
import org.apache.commons.io.IOUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import javax.annotation.Nullable;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.nio.channels.Channels;
import java.nio.channels.WritableByteChannel;
import java.util.ArrayList;
import java.util.List;
import java.util.Random;
import java.util.Set;
import static org.junit.Assert.assertEquals;
@RunWith(Parameterized.class)
public class CompressedVSizeIndexedV3WriterTest
{
@Parameterized.Parameters(name = "{index}: compression={0}, byteOrder={1}")
public static Iterable<Object[]> compressionStrategiesAndByteOrders()
{
Set<List<Object>> combinations = Sets.cartesianProduct(
Sets.newHashSet(CompressedObjectStrategy.CompressionStrategy.values()),
Sets.newHashSet(ByteOrder.BIG_ENDIAN, ByteOrder.LITTLE_ENDIAN)
);
return Iterables.transform(
combinations, new Function<List, Object[]>()
{
@Override
public Object[] apply(List input)
{
return new Object[]{input.get(0), input.get(1)};
}
}
);
}
private static final int[] OFFSET_CHUNK_FACTORS = new int[]{
1,
2,
100,
CompressedIntsIndexedSupplier.MAX_INTS_IN_BUFFER
};
private static final int[] MAX_VALUES = new int[]{0xFF, 0xFFFF, 0xFFFFFF, 0x0FFFFFFF};
private final IOPeon ioPeon = new TmpFileIOPeon();
private final CompressedObjectStrategy.CompressionStrategy compressionStrategy;
private final ByteOrder byteOrder;
private final Random rand = new Random(0);
private List<int[]> vals;
public CompressedVSizeIndexedV3WriterTest(
CompressedObjectStrategy.CompressionStrategy compressionStrategy,
ByteOrder byteOrder
)
{
this.compressionStrategy = compressionStrategy;
this.byteOrder = byteOrder;
}
private void generateVals(final int totalSize, final int maxValue) throws IOException
{
vals = new ArrayList<>(totalSize);
for (int i = 0; i < totalSize; ++i) {
int len = rand.nextInt(2) + 1;
int[] subVals = new int[len];
for (int j = 0; j < len; ++j) {
subVals[j] = rand.nextInt(maxValue);
}
vals.add(subVals);
}
}
private void checkSerializedSizeAndData(int offsetChunkFactor, int valueChunkFactor) throws Exception
{
int maxValue = vals.size() > 0 ? getMaxValue(vals) : 0;
CompressedIntsIndexedWriter offsetWriter = new CompressedIntsIndexedWriter(
ioPeon, "offset", offsetChunkFactor, byteOrder, compressionStrategy
);
CompressedVSizeIntsIndexedWriter valueWriter = new CompressedVSizeIntsIndexedWriter(
ioPeon, "value", maxValue, valueChunkFactor, byteOrder, compressionStrategy
);
CompressedVSizeIndexedV3Writer writer = new CompressedVSizeIndexedV3Writer(offsetWriter, valueWriter);
CompressedVSizeIndexedV3Supplier supplierFromIterable = CompressedVSizeIndexedV3Supplier.fromIterable(
Iterables.transform(
vals, new Function<int[], IndexedInts>()
{
@Nullable
@Override
public IndexedInts apply(@Nullable final int[] input)
{
return new ArrayBasedIndexedInts(input);
}
}
), offsetChunkFactor, maxValue, byteOrder, compressionStrategy
);
writer.open();
for (int[] val : vals) {
writer.add(val);
}
writer.close();
long writtenLength = writer.getSerializedSize();
final WritableByteChannel outputChannel = Channels.newChannel(ioPeon.makeOutputStream("output"));
writer.writeToChannel(outputChannel);
outputChannel.close();
assertEquals(writtenLength, supplierFromIterable.getSerializedSize());
// read from ByteBuffer and check values
CompressedVSizeIndexedV3Supplier supplierFromByteBuffer = CompressedVSizeIndexedV3Supplier.fromByteBuffer(
ByteBuffer.wrap(IOUtils.toByteArray(ioPeon.makeInputStream("output"))), byteOrder
);
IndexedMultivalue<IndexedInts> indexedMultivalue = supplierFromByteBuffer.get();
assertEquals(indexedMultivalue.size(), vals.size());
for (int i = 0; i < vals.size(); ++i) {
IndexedInts subVals = indexedMultivalue.get(i);
assertEquals(subVals.size(), vals.get(i).length);
for (int j = 0; j < subVals.size(); ++j) {
assertEquals(subVals.get(j), vals.get(i)[j]);
}
}
CloseQuietly.close(indexedMultivalue);
}
int getMaxValue(final List<int[]> vals)
{
return Ordering.natural().max(
Iterables.transform(
vals, new Function<int[], Integer>()
{
@Nullable
@Override
public Integer apply(int[] input)
{
return input.length > 0 ? Ints.max(input) : 0;
}
}
)
);
}
@Before
public void setUp() throws Exception
{
vals = null;
}
@After
public void tearDown() throws Exception
{
ioPeon.cleanup();
}
@Test
public void testSmallData() throws Exception
{
// less than one chunk
for (int offsetChunk : OFFSET_CHUNK_FACTORS) {
for (int maxValue : MAX_VALUES) {
final int valueChunk = CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(maxValue);
generateVals(rand.nextInt(valueChunk), maxValue);
checkSerializedSizeAndData(offsetChunk, valueChunk);
}
}
}
@Test
public void testLargeData() throws Exception
{
// more than one chunk
for (int offsetChunk : OFFSET_CHUNK_FACTORS) {
for (int maxValue : MAX_VALUES) {
final int valueChunk = CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(maxValue);
generateVals((rand.nextInt(2) + 1) * valueChunk + rand.nextInt(valueChunk), maxValue);
checkSerializedSizeAndData(offsetChunk, valueChunk);
}
}
}
@Test
public void testEmpty() throws Exception
{
vals = new ArrayList<>();
checkSerializedSizeAndData(1, 2);
}
}

View File

@ -0,0 +1,164 @@
/*
* 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.data;
import com.google.common.base.Function;
import com.google.common.collect.Iterables;
import com.google.common.collect.Sets;
import com.google.common.primitives.Ints;
import com.metamx.common.guava.CloseQuietly;
import org.apache.commons.io.IOUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.nio.channels.Channels;
import java.nio.channels.WritableByteChannel;
import java.util.List;
import java.util.Random;
import java.util.Set;
import static org.junit.Assert.assertEquals;
@RunWith(Parameterized.class)
public class CompressedVSizeIntsIndexedWriterTest
{
@Parameterized.Parameters(name = "{index}: compression={0}, byteOrder={1}")
public static Iterable<Object[]> compressionStrategiesAndByteOrders()
{
Set<List<Object>> combinations = Sets.cartesianProduct(
Sets.newHashSet(CompressedObjectStrategy.CompressionStrategy.values()),
Sets.newHashSet(ByteOrder.BIG_ENDIAN, ByteOrder.LITTLE_ENDIAN)
);
return Iterables.transform(
combinations, new Function<List, Object[]>()
{
@Override
public Object[] apply(List input)
{
return new Object[]{input.get(0), input.get(1)};
}
}
);
}
private static final int[] MAX_VALUES = new int[]{0xFF, 0xFFFF, 0xFFFFFF, 0x0FFFFFFF};
private final IOPeon ioPeon = new TmpFileIOPeon();
private final CompressedObjectStrategy.CompressionStrategy compressionStrategy;
private final ByteOrder byteOrder;
private final Random rand = new Random(0);
private int[] vals;
public CompressedVSizeIntsIndexedWriterTest(
CompressedObjectStrategy.CompressionStrategy compressionStrategy,
ByteOrder byteOrder
)
{
this.compressionStrategy = compressionStrategy;
this.byteOrder = byteOrder;
}
@Before
public void setUp() throws Exception
{
vals = null;
}
@After
public void tearDown() throws Exception
{
ioPeon.cleanup();
}
private void generateVals(final int totalSize, final int maxValue) throws IOException
{
vals = new int[totalSize];
for (int i = 0; i < vals.length; ++i) {
vals[i] = rand.nextInt(maxValue);
}
}
private void checkSerializedSizeAndData(int chunkSize) throws Exception
{
CompressedVSizeIntsIndexedWriter writer = new CompressedVSizeIntsIndexedWriter(
ioPeon, "test", vals.length > 0 ? Ints.max(vals) : 0, chunkSize, byteOrder, compressionStrategy
);
CompressedVSizeIntsIndexedSupplier supplierFromList = CompressedVSizeIntsIndexedSupplier.fromList(
Ints.asList(vals), vals.length > 0 ? Ints.max(vals) : 0, chunkSize, byteOrder, compressionStrategy
);
writer.open();
for (int val : vals) {
writer.add(val);
}
writer.close();
long writtenLength = writer.getSerializedSize();
final WritableByteChannel outputChannel = Channels.newChannel(ioPeon.makeOutputStream("output"));
writer.writeToChannel(outputChannel);
outputChannel.close();
assertEquals(writtenLength, supplierFromList.getSerializedSize());
// read from ByteBuffer and check values
CompressedVSizeIntsIndexedSupplier supplierFromByteBuffer = CompressedVSizeIntsIndexedSupplier.fromByteBuffer(
ByteBuffer.wrap(IOUtils.toByteArray(ioPeon.makeInputStream("output"))), byteOrder
);
IndexedInts indexedInts = supplierFromByteBuffer.get();
for (int i = 0; i < vals.length; ++i) {
assertEquals(vals[i], indexedInts.get(i));
}
CloseQuietly.close(indexedInts);
}
@Test
public void testSmallData() throws Exception
{
// less than one chunk
for (int maxValue : MAX_VALUES) {
final int maxChunkSize = CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(maxValue);
generateVals(rand.nextInt(maxChunkSize), maxValue);
checkSerializedSizeAndData(maxChunkSize);
}
}
@Test
public void testLargeData() throws Exception
{
// more than one chunk
for (int maxValue : MAX_VALUES) {
final int maxChunkSize = CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(maxValue);
generateVals((rand.nextInt(5) + 5) * maxChunkSize + rand.nextInt(maxChunkSize), maxValue);
checkSerializedSizeAndData(maxChunkSize);
}
}
@Test
public void testEmpty() throws Exception
{
vals = new int[0];
checkSerializedSizeAndData(2);
}
}

View File

@ -0,0 +1,111 @@
/*
* 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.data;
import com.google.common.primitives.Ints;
import org.apache.commons.io.IOUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.Channels;
import java.nio.channels.WritableByteChannel;
import java.util.Random;
import static org.junit.Assert.assertEquals;
public class VSizeIndexedIntsWriterTest
{
private static final int[] MAX_VALUES = new int[]{0xFF, 0xFFFF, 0xFFFFFF, 0x0FFFFFFF};
private final IOPeon ioPeon = new TmpFileIOPeon();
private final Random rand = new Random(0);
private int[] vals;
@Before
public void setUp() throws Exception
{
vals = null;
}
@After
public void tearDown() throws Exception
{
ioPeon.cleanup();
}
private void generateVals(final int totalSize, final int maxValue) throws IOException
{
vals = new int[totalSize];
for (int i = 0; i < vals.length; ++i) {
vals[i] = rand.nextInt(maxValue);
}
}
private void checkSerializedSizeAndData() throws Exception
{
int maxValue = vals.length == 0 ? 0 : Ints.max(vals);
VSizeIndexedIntsWriter writer = new VSizeIndexedIntsWriter(
ioPeon, "test", maxValue
);
VSizeIndexedInts intsFromList = VSizeIndexedInts.fromList(
Ints.asList(vals), maxValue
);
writer.open();
for (int val : vals) {
writer.add(val);
}
writer.close();
long writtenLength = writer.getSerializedSize();
final WritableByteChannel outputChannel = Channels.newChannel(ioPeon.makeOutputStream("output"));
writer.writeToChannel(outputChannel);
outputChannel.close();
assertEquals(writtenLength, intsFromList.getSerializedSize());
// read from ByteBuffer and check values
VSizeIndexedInts intsFromByteBuffer = VSizeIndexedInts.readFromByteBuffer(
ByteBuffer.wrap(IOUtils.toByteArray(ioPeon.makeInputStream("output")))
);
assertEquals(vals.length, intsFromByteBuffer.size());
for (int i = 0; i < vals.length; ++i) {
assertEquals(vals[i], intsFromByteBuffer.get(i));
}
}
@Test
public void testAdd() throws Exception
{
for (int maxValue : MAX_VALUES) {
generateVals(rand.nextInt(100) + 10, maxValue);
checkSerializedSizeAndData();
}
}
@Test
public void testWriteEmpty() throws Exception
{
vals = new int[0];
checkSerializedSizeAndData();
}
}

View File

@ -46,6 +46,7 @@ public class RealtimeTuningConfig implements TuningConfig
private static final int defaultMaxPendingPersists = 0;
private static final ShardSpec defaultShardSpec = new NoneShardSpec();
private static final IndexSpec defaultIndexSpec = new IndexSpec();
private static final Boolean defaultBuildV9Directly = Boolean.FALSE;
// Might make sense for this to be a builder
public static RealtimeTuningConfig makeDefaultTuningConfig()
@ -59,7 +60,8 @@ public class RealtimeTuningConfig implements TuningConfig
defaultRejectionPolicyFactory,
defaultMaxPendingPersists,
defaultShardSpec,
defaultIndexSpec
defaultIndexSpec,
defaultBuildV9Directly
);
}
@ -72,6 +74,7 @@ public class RealtimeTuningConfig implements TuningConfig
private final int maxPendingPersists;
private final ShardSpec shardSpec;
private final IndexSpec indexSpec;
private final Boolean buildV9Directly;
@JsonCreator
public RealtimeTuningConfig(
@ -83,7 +86,8 @@ public class RealtimeTuningConfig implements TuningConfig
@JsonProperty("rejectionPolicy") RejectionPolicyFactory rejectionPolicyFactory,
@JsonProperty("maxPendingPersists") Integer maxPendingPersists,
@JsonProperty("shardSpec") ShardSpec shardSpec,
@JsonProperty("indexSpec") IndexSpec indexSpec
@JsonProperty("indexSpec") IndexSpec indexSpec,
@JsonProperty("buildV9Directly") Boolean buildV9Directly
)
{
this.maxRowsInMemory = maxRowsInMemory == null ? defaultMaxRowsInMemory : maxRowsInMemory;
@ -99,6 +103,7 @@ public class RealtimeTuningConfig implements TuningConfig
this.maxPendingPersists = maxPendingPersists == null ? defaultMaxPendingPersists : maxPendingPersists;
this.shardSpec = shardSpec == null ? defaultShardSpec : shardSpec;
this.indexSpec = indexSpec == null ? defaultIndexSpec : indexSpec;
this.buildV9Directly = buildV9Directly == null ? defaultBuildV9Directly : buildV9Directly;
}
@JsonProperty
@ -155,6 +160,11 @@ public class RealtimeTuningConfig implements TuningConfig
return indexSpec;
}
@JsonProperty
public Boolean getBuildV9Directly() {
return buildV9Directly;
}
public RealtimeTuningConfig withVersioningPolicy(VersioningPolicy policy)
{
return new RealtimeTuningConfig(
@ -166,7 +176,8 @@ public class RealtimeTuningConfig implements TuningConfig
rejectionPolicyFactory,
maxPendingPersists,
shardSpec,
indexSpec
indexSpec,
buildV9Directly
);
}
@ -181,7 +192,8 @@ public class RealtimeTuningConfig implements TuningConfig
rejectionPolicyFactory,
maxPendingPersists,
shardSpec,
indexSpec
indexSpec,
buildV9Directly
);
}
}

View File

@ -116,7 +116,7 @@ public class FireDepartmentTest
null
),
new RealtimeTuningConfig(
null, null, null, null, null, null, null, null, null
null, null, null, null, null, null, null, null, null, null
)
);

View File

@ -150,6 +150,7 @@ public class RealtimeManagerTest
null,
null,
null,
null,
null
);
plumber = new TestPlumber(new Sink(new Interval("0/P5000Y"), schema, tuningConfig, new DateTime().toString()));

View File

@ -185,6 +185,7 @@ public class RealtimePlumberSchoolTest
rejectionPolicy,
null,
null,
null,
null
);

View File

@ -66,6 +66,7 @@ public class SinkTest
null,
null,
null,
null,
null
);
final Sink sink = new Sink(interval, schema, tuningConfig, version);