mirror of https://github.com/apache/druid.git
Merge pull request #2138 from KurtYoung/feature-build-v9
build v9 directly
This commit is contained in:
commit
f6a1a4ae20
|
@ -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
|
||||
|
||||
|
|
|
@ -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
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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())
|
||||
|
|
|
@ -381,6 +381,7 @@ public class BatchDeltaIngestionTest
|
|||
null,
|
||||
false,
|
||||
false,
|
||||
null,
|
||||
null
|
||||
)
|
||||
)
|
||||
|
|
|
@ -160,6 +160,7 @@ public class DetermineHashedPartitionsJobTest
|
|||
null,
|
||||
false,
|
||||
false,
|
||||
null,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
|
|
@ -264,6 +264,7 @@ public class DeterminePartitionsJobTest
|
|||
null,
|
||||
false,
|
||||
false,
|
||||
null,
|
||||
null
|
||||
)
|
||||
)
|
||||
|
|
|
@ -207,6 +207,7 @@ public class HadoopDruidIndexerConfigTest
|
|||
null,
|
||||
false,
|
||||
false,
|
||||
null,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
|
|
@ -53,6 +53,7 @@ public class HadoopTuningConfigTest
|
|||
null,
|
||||
true,
|
||||
true,
|
||||
null,
|
||||
null
|
||||
);
|
||||
|
||||
|
|
|
@ -395,6 +395,7 @@ public class IndexGeneratorJobTest
|
|||
ImmutableMap.of(JobContext.NUM_REDUCES, "0"), //verifies that set num reducers is ignored
|
||||
false,
|
||||
useCombiner,
|
||||
null,
|
||||
null
|
||||
)
|
||||
)
|
||||
|
|
|
@ -115,6 +115,7 @@ public class JobHelperTest
|
|||
),
|
||||
false,
|
||||
false,
|
||||
null,
|
||||
null
|
||||
)
|
||||
)
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -201,6 +201,7 @@ public class HadoopConverterJobTest
|
|||
null,
|
||||
false,
|
||||
false,
|
||||
null,
|
||||
null
|
||||
)
|
||||
)
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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(),
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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.
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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()
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
)
|
||||
);
|
||||
|
||||
|
|
|
@ -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()));
|
||||
|
|
|
@ -185,6 +185,7 @@ public class RealtimePlumberSchoolTest
|
|||
rejectionPolicy,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
|
||||
|
|
|
@ -66,6 +66,7 @@ public class SinkTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
final Sink sink = new Sink(interval, schema, tuningConfig, version);
|
||||
|
|
Loading…
Reference in New Issue