mirror of https://github.com/apache/druid.git
backport projections (#17257)
* abstract `IncrementalIndex` cursor stuff to prepare for using different "views" of the data based on the cursor build spec (#17064) * abstract `IncrementalIndex` cursor stuff to prepare to allow for possibility of using different "views" of the data based on the cursor build spec changes: * introduce `IncrementalIndexRowSelector` interface to capture how `IncrementalIndexCursor` and `IncrementalIndexColumnSelectorFactory` read data * `IncrementalIndex` implements `IncrementalIndexRowSelector` * move `FactsHolder` interface to separate file * other minor refactorings * add DataSchema.Builder to tidy stuff up a bit (#17065) * add DataSchema.Builder to tidy stuff up a bit * fixes * fixes * more style fixes * review stuff * Projections prototype (#17214)
This commit is contained in:
parent
1435b9f4bd
commit
7b3fc4e768
|
@ -34,7 +34,6 @@ import org.apache.druid.indexing.common.task.Task;
|
||||||
import org.apache.druid.indexing.common.task.Tasks;
|
import org.apache.druid.indexing.common.task.Tasks;
|
||||||
import org.apache.druid.java.util.common.Intervals;
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||||
import org.apache.druid.segment.IndexSpec;
|
import org.apache.druid.segment.IndexSpec;
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
|
@ -66,18 +65,19 @@ public class K8sTestUtils
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
new IndexTask.IndexIngestionSpec(
|
new IndexTask.IndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"foo",
|
.withDataSource("foo")
|
||||||
new TimestampSpec(null, null, null),
|
.withTimestamp(new TimestampSpec(null, null, null))
|
||||||
DimensionsSpec.EMPTY,
|
.withDimensions(DimensionsSpec.EMPTY)
|
||||||
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
|
.withAggregators(new DoubleSumAggregatorFactory("met", "met"))
|
||||||
new UniformGranularitySpec(
|
.withGranularity(
|
||||||
Granularities.DAY,
|
new UniformGranularitySpec(
|
||||||
null,
|
Granularities.DAY,
|
||||||
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
|
null,
|
||||||
),
|
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
|
||||||
null
|
)
|
||||||
),
|
)
|
||||||
|
.build(),
|
||||||
new IndexTask.IndexIOConfig(
|
new IndexTask.IndexIOConfig(
|
||||||
new LocalInputSource(new File("lol"), "rofl"),
|
new LocalInputSource(new File("lol"), "rofl"),
|
||||||
new NoopInputFormat(),
|
new NoopInputFormat(),
|
||||||
|
|
|
@ -48,7 +48,6 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
|
import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
|
||||||
import org.apache.druid.segment.realtime.ChatHandlerProvider;
|
import org.apache.druid.segment.realtime.ChatHandlerProvider;
|
||||||
import org.apache.druid.segment.transform.TransformSpec;
|
|
||||||
import org.apache.druid.server.security.AuthorizerMapper;
|
import org.apache.druid.server.security.AuthorizerMapper;
|
||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
@ -211,14 +210,13 @@ public class MaterializedViewSupervisorSpec implements SupervisorSpec
|
||||||
);
|
);
|
||||||
|
|
||||||
// generate DataSchema
|
// generate DataSchema
|
||||||
DataSchema dataSchema = new DataSchema(
|
DataSchema dataSchema = DataSchema.builder()
|
||||||
dataSourceName,
|
.withDataSource(dataSourceName)
|
||||||
parser,
|
.withParserMap(parser)
|
||||||
aggregators,
|
.withAggregators(aggregators)
|
||||||
granularitySpec,
|
.withGranularity(granularitySpec)
|
||||||
TransformSpec.NONE,
|
.withObjectMapper(objectMapper)
|
||||||
objectMapper
|
.build();
|
||||||
);
|
|
||||||
|
|
||||||
// generate DatasourceIngestionSpec
|
// generate DatasourceIngestionSpec
|
||||||
DatasourceIngestionSpec datasourceIngestionSpec = new DatasourceIngestionSpec(
|
DatasourceIngestionSpec datasourceIngestionSpec = new DatasourceIngestionSpec(
|
||||||
|
|
|
@ -53,7 +53,6 @@ import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig;
|
import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig;
|
||||||
import org.apache.druid.segment.metadata.SegmentSchemaManager;
|
import org.apache.druid.segment.metadata.SegmentSchemaManager;
|
||||||
import org.apache.druid.segment.realtime.ChatHandlerProvider;
|
import org.apache.druid.segment.realtime.ChatHandlerProvider;
|
||||||
import org.apache.druid.segment.transform.TransformSpec;
|
|
||||||
import org.apache.druid.server.security.AuthorizerMapper;
|
import org.apache.druid.server.security.AuthorizerMapper;
|
||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
|
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
|
||||||
|
@ -237,14 +236,10 @@ public class MaterializedViewSupervisorTest
|
||||||
Map<Interval, HadoopIndexTask> runningTasks = runningTasksPair.lhs;
|
Map<Interval, HadoopIndexTask> runningTasks = runningTasksPair.lhs;
|
||||||
Map<Interval, String> runningVersion = runningTasksPair.rhs;
|
Map<Interval, String> runningVersion = runningTasksPair.rhs;
|
||||||
|
|
||||||
DataSchema dataSchema = new DataSchema(
|
DataSchema dataSchema = DataSchema.builder()
|
||||||
"test_datasource",
|
.withDataSource("test_datasource")
|
||||||
null,
|
.withObjectMapper(objectMapper)
|
||||||
null,
|
.build();
|
||||||
null,
|
|
||||||
TransformSpec.NONE,
|
|
||||||
objectMapper
|
|
||||||
);
|
|
||||||
HadoopIOConfig hadoopIOConfig = new HadoopIOConfig(new HashMap<>(), null, null);
|
HadoopIOConfig hadoopIOConfig = new HadoopIOConfig(new HashMap<>(), null, null);
|
||||||
HadoopIngestionSpec spec = new HadoopIngestionSpec(dataSchema, hadoopIOConfig, null);
|
HadoopIngestionSpec spec = new HadoopIngestionSpec(dataSchema, hadoopIOConfig, null);
|
||||||
HadoopIndexTask task1 = new HadoopIndexTask(
|
HadoopIndexTask task1 = new HadoopIndexTask(
|
||||||
|
|
|
@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import org.apache.druid.data.input.InputFormat;
|
import org.apache.druid.data.input.InputFormat;
|
||||||
import org.apache.druid.data.input.impl.DimensionSchema;
|
import org.apache.druid.data.input.impl.DimensionSchema;
|
||||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
|
||||||
import org.apache.druid.data.input.impl.JsonInputFormat;
|
import org.apache.druid.data.input.impl.JsonInputFormat;
|
||||||
import org.apache.druid.data.input.impl.StringDimensionSchema;
|
import org.apache.druid.data.input.impl.StringDimensionSchema;
|
||||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||||
|
@ -44,7 +43,6 @@ import org.apache.druid.java.util.common.parsers.JSONPathSpec;
|
||||||
import org.apache.druid.java.util.emitter.EmittingLogger;
|
import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||||
import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig;
|
import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig;
|
||||||
import org.apache.druid.java.util.metrics.StubServiceEmitter;
|
import org.apache.druid.java.util.metrics.StubServiceEmitter;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import org.apache.druid.segment.TestHelper;
|
import org.apache.druid.segment.TestHelper;
|
||||||
import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
|
import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
|
||||||
|
@ -102,16 +100,19 @@ public class RabbitStreamSupervisorTest extends EasyMockSupport
|
||||||
dimensions.add(StringDimensionSchema.create("dim1"));
|
dimensions.add(StringDimensionSchema.create("dim1"));
|
||||||
dimensions.add(StringDimensionSchema.create("dim2"));
|
dimensions.add(StringDimensionSchema.create("dim2"));
|
||||||
|
|
||||||
return new DataSchema(
|
return DataSchema.builder()
|
||||||
dataSource,
|
.withDataSource(dataSource)
|
||||||
new TimestampSpec("timestamp", "iso", null),
|
.withTimestamp(new TimestampSpec("timestamp", "iso", null))
|
||||||
new DimensionsSpec(dimensions),
|
.withDimensions(dimensions)
|
||||||
new AggregatorFactory[] {new CountAggregatorFactory("rows")},
|
.withAggregators(new CountAggregatorFactory("rows"))
|
||||||
new UniformGranularitySpec(
|
.withGranularity(
|
||||||
Granularities.HOUR,
|
new UniformGranularitySpec(
|
||||||
Granularities.NONE,
|
Granularities.HOUR,
|
||||||
ImmutableList.of()),
|
Granularities.NONE,
|
||||||
null);
|
ImmutableList.of()
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
@BeforeClass
|
@BeforeClass
|
||||||
|
|
|
@ -278,6 +278,26 @@ public abstract class HllSketchAggregatorFactory extends AggregatorFactory
|
||||||
&& stringEncoding == that.stringEncoding;
|
&& stringEncoding == that.stringEncoding;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public AggregatorFactory substituteCombiningFactory(AggregatorFactory preAggregated)
|
||||||
|
{
|
||||||
|
if (this == preAggregated) {
|
||||||
|
return getCombiningFactory();
|
||||||
|
}
|
||||||
|
if (getClass() != preAggregated.getClass()) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
HllSketchAggregatorFactory that = (HllSketchAggregatorFactory) preAggregated;
|
||||||
|
if (lgK == that.lgK && tgtHllType == that.tgtHllType && stringEncoding == that.stringEncoding && Objects.equals(
|
||||||
|
fieldName,
|
||||||
|
that.fieldName
|
||||||
|
)) {
|
||||||
|
return getCombiningFactory();
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int hashCode()
|
public int hashCode()
|
||||||
{
|
{
|
||||||
|
|
|
@ -231,6 +231,42 @@ public class HllSketchAggregatorFactoryTest
|
||||||
Assert.assertArrayEquals(target.getCacheKey(), other.getCacheKey());
|
Assert.assertArrayEquals(target.getCacheKey(), other.getCacheKey());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCanSubstitute()
|
||||||
|
{
|
||||||
|
HllSketchBuildAggregatorFactory factory = new HllSketchBuildAggregatorFactory(
|
||||||
|
NAME,
|
||||||
|
FIELD_NAME,
|
||||||
|
LG_K,
|
||||||
|
TGT_HLL_TYPE,
|
||||||
|
STRING_ENCODING,
|
||||||
|
true,
|
||||||
|
true
|
||||||
|
);
|
||||||
|
HllSketchBuildAggregatorFactory other = new HllSketchBuildAggregatorFactory(
|
||||||
|
"other name",
|
||||||
|
FIELD_NAME,
|
||||||
|
LG_K,
|
||||||
|
TGT_HLL_TYPE,
|
||||||
|
STRING_ENCODING,
|
||||||
|
false,
|
||||||
|
false
|
||||||
|
);
|
||||||
|
|
||||||
|
HllSketchBuildAggregatorFactory incompatible = new HllSketchBuildAggregatorFactory(
|
||||||
|
NAME,
|
||||||
|
"different field",
|
||||||
|
LG_K,
|
||||||
|
TGT_HLL_TYPE,
|
||||||
|
STRING_ENCODING,
|
||||||
|
false,
|
||||||
|
false
|
||||||
|
);
|
||||||
|
Assert.assertNotNull(other.substituteCombiningFactory(factory));
|
||||||
|
Assert.assertNotNull(factory.substituteCombiningFactory(other));
|
||||||
|
Assert.assertNull(factory.substituteCombiningFactory(incompatible));
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testToString()
|
public void testToString()
|
||||||
{
|
{
|
||||||
|
|
|
@ -92,7 +92,6 @@ import org.apache.druid.query.QueryPlus;
|
||||||
import org.apache.druid.query.QueryRunnerFactory;
|
import org.apache.druid.query.QueryRunnerFactory;
|
||||||
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
|
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
|
||||||
import org.apache.druid.query.SegmentDescriptor;
|
import org.apache.druid.query.SegmentDescriptor;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||||
import org.apache.druid.query.filter.SelectorDimFilter;
|
import org.apache.druid.query.filter.SelectorDimFilter;
|
||||||
|
@ -1262,28 +1261,27 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
||||||
|
|
||||||
final KafkaIndexTask task = createTask(
|
final KafkaIndexTask task = createTask(
|
||||||
null,
|
null,
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"test_ds",
|
.withDataSource("test_ds")
|
||||||
new TimestampSpec("timestamp", "iso", null),
|
.withTimestamp(new TimestampSpec("timestamp", "iso", null))
|
||||||
new DimensionsSpec(
|
.withDimensions(
|
||||||
Arrays.asList(
|
new StringDimensionSchema("dim1"),
|
||||||
new StringDimensionSchema("dim1"),
|
new StringDimensionSchema("dim1t"),
|
||||||
new StringDimensionSchema("dim1t"),
|
new StringDimensionSchema("dim2"),
|
||||||
new StringDimensionSchema("dim2"),
|
new LongDimensionSchema("dimLong"),
|
||||||
new LongDimensionSchema("dimLong"),
|
new FloatDimensionSchema("dimFloat"),
|
||||||
new FloatDimensionSchema("dimFloat"),
|
new StringDimensionSchema("kafka.topic"),
|
||||||
new StringDimensionSchema("kafka.topic"),
|
new LongDimensionSchema("kafka.offset"),
|
||||||
new LongDimensionSchema("kafka.offset"),
|
new StringDimensionSchema("kafka.header.encoding")
|
||||||
new StringDimensionSchema("kafka.header.encoding")
|
)
|
||||||
)
|
.withAggregators(
|
||||||
),
|
new DoubleSumAggregatorFactory("met1sum", "met1"),
|
||||||
new AggregatorFactory[]{
|
new CountAggregatorFactory("rows")
|
||||||
new DoubleSumAggregatorFactory("met1sum", "met1"),
|
)
|
||||||
new CountAggregatorFactory("rows")
|
.withGranularity(
|
||||||
},
|
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)
|
||||||
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
|
)
|
||||||
null
|
.build(),
|
||||||
),
|
|
||||||
new KafkaIndexTaskIOConfig(
|
new KafkaIndexTaskIOConfig(
|
||||||
0,
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
|
@ -1337,26 +1335,25 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
||||||
|
|
||||||
final KafkaIndexTask task = createTask(
|
final KafkaIndexTask task = createTask(
|
||||||
null,
|
null,
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"test_ds",
|
.withDataSource("test_ds")
|
||||||
new TimestampSpec("timestamp", "iso", null),
|
.withTimestamp(new TimestampSpec("timestamp", "iso", null))
|
||||||
new DimensionsSpec(
|
.withDimensions(
|
||||||
Arrays.asList(
|
new StringDimensionSchema("dim1"),
|
||||||
new StringDimensionSchema("dim1"),
|
new StringDimensionSchema("dim1t"),
|
||||||
new StringDimensionSchema("dim1t"),
|
new StringDimensionSchema("dim2"),
|
||||||
new StringDimensionSchema("dim2"),
|
new LongDimensionSchema("dimLong"),
|
||||||
new LongDimensionSchema("dimLong"),
|
new FloatDimensionSchema("dimFloat"),
|
||||||
new FloatDimensionSchema("dimFloat"),
|
new StringDimensionSchema("kafka.testheader.encoding")
|
||||||
new StringDimensionSchema("kafka.testheader.encoding")
|
)
|
||||||
)
|
.withAggregators(
|
||||||
),
|
new DoubleSumAggregatorFactory("met1sum", "met1"),
|
||||||
new AggregatorFactory[]{
|
new CountAggregatorFactory("rows")
|
||||||
new DoubleSumAggregatorFactory("met1sum", "met1"),
|
)
|
||||||
new CountAggregatorFactory("rows")
|
.withGranularity(
|
||||||
},
|
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)
|
||||||
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
|
)
|
||||||
null
|
.build(),
|
||||||
),
|
|
||||||
new KafkaIndexTaskIOConfig(
|
new KafkaIndexTaskIOConfig(
|
||||||
0,
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
|
@ -2887,16 +2884,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
||||||
|
|
||||||
private static DataSchema cloneDataSchema(final DataSchema dataSchema)
|
private static DataSchema cloneDataSchema(final DataSchema dataSchema)
|
||||||
{
|
{
|
||||||
return new DataSchema(
|
return DataSchema.builder(dataSchema).withObjectMapper(OBJECT_MAPPER).build();
|
||||||
dataSchema.getDataSource(),
|
|
||||||
dataSchema.getTimestampSpec(),
|
|
||||||
dataSchema.getDimensionsSpec(),
|
|
||||||
dataSchema.getAggregators(),
|
|
||||||
dataSchema.getGranularitySpec(),
|
|
||||||
dataSchema.getTransformSpec(),
|
|
||||||
dataSchema.getParserMap(),
|
|
||||||
OBJECT_MAPPER
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -46,7 +46,6 @@ import org.apache.druid.jackson.DefaultObjectMapper;
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.java.util.common.parsers.JSONPathSpec;
|
import org.apache.druid.java.util.common.parsers.JSONPathSpec;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||||
import org.apache.druid.segment.TestHelper;
|
import org.apache.druid.segment.TestHelper;
|
||||||
|
@ -81,45 +80,30 @@ public class KafkaSamplerSpecTest extends InitializedNullHandlingTest
|
||||||
|
|
||||||
private static final ObjectMapper OBJECT_MAPPER = TestHelper.makeJsonMapper();
|
private static final ObjectMapper OBJECT_MAPPER = TestHelper.makeJsonMapper();
|
||||||
private static final String TOPIC = "sampling";
|
private static final String TOPIC = "sampling";
|
||||||
private static final DataSchema DATA_SCHEMA = new DataSchema(
|
private static final DataSchema DATA_SCHEMA =
|
||||||
"test_ds",
|
DataSchema.builder()
|
||||||
new TimestampSpec("timestamp", "iso", null),
|
.withDataSource("test_ds")
|
||||||
new DimensionsSpec(
|
.withTimestamp(new TimestampSpec("timestamp", "iso", null))
|
||||||
Arrays.asList(
|
.withDimensions(
|
||||||
new StringDimensionSchema("dim1"),
|
new StringDimensionSchema("dim1"),
|
||||||
new StringDimensionSchema("dim1t"),
|
new StringDimensionSchema("dim1t"),
|
||||||
new StringDimensionSchema("dim2"),
|
new StringDimensionSchema("dim2"),
|
||||||
new LongDimensionSchema("dimLong"),
|
new LongDimensionSchema("dimLong"),
|
||||||
new FloatDimensionSchema("dimFloat")
|
new FloatDimensionSchema("dimFloat")
|
||||||
)
|
)
|
||||||
),
|
.withAggregators(
|
||||||
new AggregatorFactory[]{
|
new DoubleSumAggregatorFactory("met1sum", "met1"),
|
||||||
new DoubleSumAggregatorFactory("met1sum", "met1"),
|
new CountAggregatorFactory("rows")
|
||||||
new CountAggregatorFactory("rows")
|
)
|
||||||
},
|
.withGranularity(
|
||||||
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
|
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)
|
||||||
null
|
)
|
||||||
);
|
.build();
|
||||||
|
|
||||||
private static final DataSchema DATA_SCHEMA_KAFKA_TIMESTAMP = new DataSchema(
|
private static final DataSchema DATA_SCHEMA_KAFKA_TIMESTAMP =
|
||||||
"test_ds",
|
DataSchema.builder(DATA_SCHEMA)
|
||||||
new TimestampSpec("kafka.timestamp", "iso", null),
|
.withTimestamp(new TimestampSpec("kafka.timestamp", "iso", null))
|
||||||
new DimensionsSpec(
|
.build();
|
||||||
Arrays.asList(
|
|
||||||
new StringDimensionSchema("dim1"),
|
|
||||||
new StringDimensionSchema("dim1t"),
|
|
||||||
new StringDimensionSchema("dim2"),
|
|
||||||
new LongDimensionSchema("dimLong"),
|
|
||||||
new FloatDimensionSchema("dimFloat")
|
|
||||||
)
|
|
||||||
),
|
|
||||||
new AggregatorFactory[]{
|
|
||||||
new DoubleSumAggregatorFactory("met1sum", "met1"),
|
|
||||||
new CountAggregatorFactory("rows")
|
|
||||||
},
|
|
||||||
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
|
|
||||||
null
|
|
||||||
);
|
|
||||||
|
|
||||||
private static TestingCluster zkServer;
|
private static TestingCluster zkServer;
|
||||||
private static TestBroker kafkaServer;
|
private static TestBroker kafkaServer;
|
||||||
|
@ -364,17 +348,18 @@ public class KafkaSamplerSpecTest extends InitializedNullHandlingTest
|
||||||
);
|
);
|
||||||
InputRowParser parser = new StringInputRowParser(new JSONParseSpec(timestampSpec, dimensionsSpec, JSONPathSpec.DEFAULT, null, null), "UTF8");
|
InputRowParser parser = new StringInputRowParser(new JSONParseSpec(timestampSpec, dimensionsSpec, JSONPathSpec.DEFAULT, null, null), "UTF8");
|
||||||
|
|
||||||
DataSchema dataSchema = new DataSchema(
|
DataSchema dataSchema = DataSchema.builder()
|
||||||
"test_ds",
|
.withDataSource("test_ds")
|
||||||
objectMapper.readValue(objectMapper.writeValueAsBytes(parser), Map.class),
|
.withParserMap(
|
||||||
new AggregatorFactory[]{
|
objectMapper.readValue(objectMapper.writeValueAsBytes(parser), Map.class)
|
||||||
new DoubleSumAggregatorFactory("met1sum", "met1"),
|
)
|
||||||
new CountAggregatorFactory("rows")
|
.withAggregators(
|
||||||
},
|
new DoubleSumAggregatorFactory("met1sum", "met1"),
|
||||||
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
|
new CountAggregatorFactory("rows")
|
||||||
null,
|
)
|
||||||
objectMapper
|
.withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null))
|
||||||
);
|
.withObjectMapper(objectMapper)
|
||||||
|
.build();
|
||||||
|
|
||||||
KafkaSupervisorSpec supervisorSpec = new KafkaSupervisorSpec(
|
KafkaSupervisorSpec supervisorSpec = new KafkaSupervisorSpec(
|
||||||
null,
|
null,
|
||||||
|
|
|
@ -30,7 +30,6 @@ import com.google.common.util.concurrent.ListenableFuture;
|
||||||
import org.apache.curator.test.TestingCluster;
|
import org.apache.curator.test.TestingCluster;
|
||||||
import org.apache.druid.data.input.InputFormat;
|
import org.apache.druid.data.input.InputFormat;
|
||||||
import org.apache.druid.data.input.impl.DimensionSchema;
|
import org.apache.druid.data.input.impl.DimensionSchema;
|
||||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
|
||||||
import org.apache.druid.data.input.impl.JsonInputFormat;
|
import org.apache.druid.data.input.impl.JsonInputFormat;
|
||||||
import org.apache.druid.data.input.impl.StringDimensionSchema;
|
import org.apache.druid.data.input.impl.StringDimensionSchema;
|
||||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||||
|
@ -83,7 +82,6 @@ import org.apache.druid.java.util.emitter.service.AlertBuilder;
|
||||||
import org.apache.druid.java.util.emitter.service.AlertEvent;
|
import org.apache.druid.java.util.emitter.service.AlertEvent;
|
||||||
import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig;
|
import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig;
|
||||||
import org.apache.druid.java.util.metrics.StubServiceEmitter;
|
import org.apache.druid.java.util.metrics.StubServiceEmitter;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import org.apache.druid.segment.TestHelper;
|
import org.apache.druid.segment.TestHelper;
|
||||||
import org.apache.druid.segment.incremental.ParseExceptionReport;
|
import org.apache.druid.segment.incremental.ParseExceptionReport;
|
||||||
|
@ -5128,18 +5126,19 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
dimensions.add(StringDimensionSchema.create("dim1"));
|
dimensions.add(StringDimensionSchema.create("dim1"));
|
||||||
dimensions.add(StringDimensionSchema.create("dim2"));
|
dimensions.add(StringDimensionSchema.create("dim2"));
|
||||||
|
|
||||||
return new DataSchema(
|
return DataSchema.builder()
|
||||||
dataSource,
|
.withDataSource(dataSource)
|
||||||
new TimestampSpec("timestamp", "iso", null),
|
.withTimestamp(new TimestampSpec("timestamp", "iso", null))
|
||||||
new DimensionsSpec(dimensions),
|
.withDimensions(dimensions)
|
||||||
new AggregatorFactory[]{new CountAggregatorFactory("rows")},
|
.withAggregators(new CountAggregatorFactory("rows"))
|
||||||
new UniformGranularitySpec(
|
.withGranularity(
|
||||||
Granularities.HOUR,
|
new UniformGranularitySpec(
|
||||||
Granularities.NONE,
|
Granularities.HOUR,
|
||||||
ImmutableList.of()
|
Granularities.NONE,
|
||||||
),
|
ImmutableList.of()
|
||||||
null
|
)
|
||||||
);
|
)
|
||||||
|
.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
private KafkaIndexTask createKafkaIndexTask(
|
private KafkaIndexTask createKafkaIndexTask(
|
||||||
|
|
|
@ -50,7 +50,8 @@ import java.util.Collections;
|
||||||
|
|
||||||
public class KinesisIndexTaskSerdeTest
|
public class KinesisIndexTaskSerdeTest
|
||||||
{
|
{
|
||||||
private static final DataSchema DATA_SCHEMA = new DataSchema("dataSource", null, null, null, null, null, null, null);
|
private static final DataSchema DATA_SCHEMA =
|
||||||
|
DataSchema.builder().withDataSource("dataSource").build();
|
||||||
private static final KinesisIndexTaskTuningConfig TUNING_CONFIG = new KinesisIndexTaskTuningConfig(
|
private static final KinesisIndexTaskTuningConfig TUNING_CONFIG = new KinesisIndexTaskTuningConfig(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
|
|
|
@ -2411,16 +2411,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
||||||
|
|
||||||
private static DataSchema cloneDataSchema(final DataSchema dataSchema)
|
private static DataSchema cloneDataSchema(final DataSchema dataSchema)
|
||||||
{
|
{
|
||||||
return new DataSchema(
|
return DataSchema.builder(dataSchema).withObjectMapper(OBJECT_MAPPER).build();
|
||||||
dataSchema.getDataSource(),
|
|
||||||
dataSchema.getTimestampSpec(),
|
|
||||||
dataSchema.getDimensionsSpec(),
|
|
||||||
dataSchema.getAggregators(),
|
|
||||||
dataSchema.getGranularitySpec(),
|
|
||||||
dataSchema.getTransformSpec(),
|
|
||||||
dataSchema.getParserMap(),
|
|
||||||
OBJECT_MAPPER
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -49,7 +49,6 @@ import org.apache.druid.jackson.DefaultObjectMapper;
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.java.util.common.parsers.JSONPathSpec;
|
import org.apache.druid.java.util.common.parsers.JSONPathSpec;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
|
@ -75,25 +74,25 @@ public class KinesisSamplerSpecTest extends EasyMockSupport
|
||||||
{
|
{
|
||||||
private static final String STREAM = "sampling";
|
private static final String STREAM = "sampling";
|
||||||
private static final String SHARD_ID = "1";
|
private static final String SHARD_ID = "1";
|
||||||
private static final DataSchema DATA_SCHEMA = new DataSchema(
|
private static final DataSchema DATA_SCHEMA =
|
||||||
"test_ds",
|
DataSchema.builder()
|
||||||
new TimestampSpec("timestamp", "iso", null),
|
.withDataSource("test_ds")
|
||||||
new DimensionsSpec(
|
.withTimestamp(new TimestampSpec("timestamp", "iso", null))
|
||||||
Arrays.asList(
|
.withDimensions(
|
||||||
new StringDimensionSchema("dim1"),
|
new StringDimensionSchema("dim1"),
|
||||||
new StringDimensionSchema("dim1t"),
|
new StringDimensionSchema("dim1t"),
|
||||||
new StringDimensionSchema("dim2"),
|
new StringDimensionSchema("dim2"),
|
||||||
new LongDimensionSchema("dimLong"),
|
new LongDimensionSchema("dimLong"),
|
||||||
new FloatDimensionSchema("dimFloat")
|
new FloatDimensionSchema("dimFloat")
|
||||||
)
|
)
|
||||||
),
|
.withAggregators(
|
||||||
new AggregatorFactory[]{
|
new DoubleSumAggregatorFactory("met1sum", "met1"),
|
||||||
new DoubleSumAggregatorFactory("met1sum", "met1"),
|
new CountAggregatorFactory("rows")
|
||||||
new CountAggregatorFactory("rows")
|
)
|
||||||
},
|
.withGranularity(
|
||||||
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
|
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)
|
||||||
null
|
)
|
||||||
);
|
.build();
|
||||||
|
|
||||||
static {
|
static {
|
||||||
NullHandling.initializeForTests();
|
NullHandling.initializeForTests();
|
||||||
|
@ -192,17 +191,18 @@ public class KinesisSamplerSpecTest extends EasyMockSupport
|
||||||
);
|
);
|
||||||
InputRowParser parser = new StringInputRowParser(new JSONParseSpec(timestampSpec, dimensionsSpec, JSONPathSpec.DEFAULT, null, null), "UTF8");
|
InputRowParser parser = new StringInputRowParser(new JSONParseSpec(timestampSpec, dimensionsSpec, JSONPathSpec.DEFAULT, null, null), "UTF8");
|
||||||
|
|
||||||
DataSchema dataSchema = new DataSchema(
|
DataSchema dataSchema = DataSchema.builder()
|
||||||
"test_ds",
|
.withDataSource("test_ds")
|
||||||
objectMapper.readValue(objectMapper.writeValueAsBytes(parser), Map.class),
|
.withParserMap(
|
||||||
new AggregatorFactory[]{
|
objectMapper.readValue(objectMapper.writeValueAsBytes(parser), Map.class)
|
||||||
new DoubleSumAggregatorFactory("met1sum", "met1"),
|
)
|
||||||
new CountAggregatorFactory("rows")
|
.withAggregators(
|
||||||
},
|
new DoubleSumAggregatorFactory("met1sum", "met1"),
|
||||||
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
|
new CountAggregatorFactory("rows")
|
||||||
null,
|
)
|
||||||
objectMapper
|
.withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null))
|
||||||
);
|
.withObjectMapper(objectMapper)
|
||||||
|
.build();
|
||||||
|
|
||||||
KinesisSupervisorSpec supervisorSpec = new KinesisSupervisorSpec(
|
KinesisSupervisorSpec supervisorSpec = new KinesisSupervisorSpec(
|
||||||
null,
|
null,
|
||||||
|
|
|
@ -28,7 +28,6 @@ import com.google.common.util.concurrent.Futures;
|
||||||
import com.google.common.util.concurrent.ListenableFuture;
|
import com.google.common.util.concurrent.ListenableFuture;
|
||||||
import org.apache.druid.data.input.InputFormat;
|
import org.apache.druid.data.input.InputFormat;
|
||||||
import org.apache.druid.data.input.impl.DimensionSchema;
|
import org.apache.druid.data.input.impl.DimensionSchema;
|
||||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
|
||||||
import org.apache.druid.data.input.impl.JsonInputFormat;
|
import org.apache.druid.data.input.impl.JsonInputFormat;
|
||||||
import org.apache.druid.data.input.impl.StringDimensionSchema;
|
import org.apache.druid.data.input.impl.StringDimensionSchema;
|
||||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||||
|
@ -79,7 +78,6 @@ import org.apache.druid.java.util.emitter.service.AlertBuilder;
|
||||||
import org.apache.druid.java.util.emitter.service.AlertEvent;
|
import org.apache.druid.java.util.emitter.service.AlertEvent;
|
||||||
import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig;
|
import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig;
|
||||||
import org.apache.druid.java.util.metrics.StubServiceEmitter;
|
import org.apache.druid.java.util.metrics.StubServiceEmitter;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import org.apache.druid.segment.TestHelper;
|
import org.apache.druid.segment.TestHelper;
|
||||||
import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
|
import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
|
||||||
|
@ -5479,18 +5477,19 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
||||||
dimensions.add(StringDimensionSchema.create("dim1"));
|
dimensions.add(StringDimensionSchema.create("dim1"));
|
||||||
dimensions.add(StringDimensionSchema.create("dim2"));
|
dimensions.add(StringDimensionSchema.create("dim2"));
|
||||||
|
|
||||||
return new DataSchema(
|
return DataSchema.builder()
|
||||||
dataSource,
|
.withDataSource(dataSource)
|
||||||
new TimestampSpec("timestamp", "iso", null),
|
.withTimestamp(new TimestampSpec("timestamp", "iso", null))
|
||||||
new DimensionsSpec(dimensions),
|
.withDimensions(dimensions)
|
||||||
new AggregatorFactory[]{new CountAggregatorFactory("rows")},
|
.withAggregators(new CountAggregatorFactory("rows"))
|
||||||
new UniformGranularitySpec(
|
.withGranularity(
|
||||||
Granularities.HOUR,
|
new UniformGranularitySpec(
|
||||||
Granularities.NONE,
|
Granularities.HOUR,
|
||||||
ImmutableList.of()
|
Granularities.NONE,
|
||||||
),
|
ImmutableList.of()
|
||||||
null
|
)
|
||||||
);
|
)
|
||||||
|
.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -53,14 +53,12 @@ import org.apache.druid.segment.column.ValueType;
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
|
import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
|
||||||
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
|
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
|
||||||
import org.apache.druid.segment.transform.TransformSpec;
|
|
||||||
import org.apache.druid.sql.calcite.planner.ColumnMappings;
|
import org.apache.druid.sql.calcite.planner.ColumnMappings;
|
||||||
import org.apache.druid.sql.calcite.rel.DruidQuery;
|
import org.apache.druid.sql.calcite.rel.DruidQuery;
|
||||||
import org.apache.druid.utils.CollectionUtils;
|
import org.apache.druid.utils.CollectionUtils;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.LinkedHashSet;
|
import java.util.LinkedHashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -96,14 +94,13 @@ public final class SegmentGenerationUtils
|
||||||
destination.getDimensionSchemas()
|
destination.getDimensionSchemas()
|
||||||
);
|
);
|
||||||
|
|
||||||
return new DataSchema(
|
return DataSchema.builder()
|
||||||
destination.getDataSource(),
|
.withDataSource(destination.getDataSource())
|
||||||
new TimestampSpec(ColumnHolder.TIME_COLUMN_NAME, "millis", null),
|
.withTimestamp(new TimestampSpec(ColumnHolder.TIME_COLUMN_NAME, "millis", null))
|
||||||
dimensionsAndAggregators.lhs,
|
.withDimensions(dimensionsAndAggregators.lhs)
|
||||||
dimensionsAndAggregators.rhs.toArray(new AggregatorFactory[0]),
|
.withAggregators(dimensionsAndAggregators.rhs.toArray(new AggregatorFactory[0]))
|
||||||
makeGranularitySpecForIngestion(querySpec.getQuery(), querySpec.getColumnMappings(), isRollupQuery, jsonMapper),
|
.withGranularity(makeGranularitySpecForIngestion(querySpec.getQuery(), querySpec.getColumnMappings(), isRollupQuery, jsonMapper))
|
||||||
new TransformSpec(null, Collections.emptyList())
|
.build();
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private static GranularitySpec makeGranularitySpecForIngestion(
|
private static GranularitySpec makeGranularitySpecForIngestion(
|
||||||
|
|
|
@ -259,19 +259,21 @@ public class MSQCompactionRunnerTest
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
|
|
||||||
DataSchema dataSchema = new DataSchema(
|
DataSchema dataSchema =
|
||||||
DATA_SOURCE,
|
DataSchema.builder()
|
||||||
new TimestampSpec(TIMESTAMP_COLUMN, null, null),
|
.withDataSource(DATA_SOURCE)
|
||||||
new DimensionsSpec(DIMENSIONS),
|
.withTimestamp(new TimestampSpec(TIMESTAMP_COLUMN, null, null))
|
||||||
new AggregatorFactory[]{},
|
.withDimensions(DIMENSIONS)
|
||||||
new UniformGranularitySpec(
|
.withGranularity(
|
||||||
SEGMENT_GRANULARITY.getDefaultGranularity(),
|
new UniformGranularitySpec(
|
||||||
null,
|
SEGMENT_GRANULARITY.getDefaultGranularity(),
|
||||||
false,
|
null,
|
||||||
Collections.singletonList(COMPACTION_INTERVAL)
|
false,
|
||||||
),
|
Collections.singletonList(COMPACTION_INTERVAL)
|
||||||
new TransformSpec(dimFilter, Collections.emptyList())
|
)
|
||||||
);
|
)
|
||||||
|
.withTransform(new TransformSpec(dimFilter, Collections.emptyList()))
|
||||||
|
.build();
|
||||||
|
|
||||||
|
|
||||||
List<MSQControllerTask> msqControllerTasks = MSQ_COMPACTION_RUNNER.createMsqControllerTasks(
|
List<MSQControllerTask> msqControllerTasks = MSQ_COMPACTION_RUNNER.createMsqControllerTasks(
|
||||||
|
|
|
@ -296,7 +296,7 @@ public class ChannelStageOutputReaderTest extends InitializedNullHandlingTest
|
||||||
frameReader = FrameReader.create(adapter.getRowSignature());
|
frameReader = FrameReader.create(adapter.getRowSignature());
|
||||||
frameList = FrameSequenceBuilder.fromCursorFactory(adapter)
|
frameList = FrameSequenceBuilder.fromCursorFactory(adapter)
|
||||||
.frameType(FrameType.ROW_BASED)
|
.frameType(FrameType.ROW_BASED)
|
||||||
.maxRowsPerFrame(IntMath.divide(index.size(), MAX_FRAMES, RoundingMode.CEILING))
|
.maxRowsPerFrame(IntMath.divide(index.numRows(), MAX_FRAMES, RoundingMode.CEILING))
|
||||||
.frames()
|
.frames()
|
||||||
.toList();
|
.toList();
|
||||||
}
|
}
|
||||||
|
|
|
@ -433,30 +433,33 @@ public class BatchDeltaIngestionTest
|
||||||
{
|
{
|
||||||
HadoopDruidIndexerConfig config = new HadoopDruidIndexerConfig(
|
HadoopDruidIndexerConfig config = new HadoopDruidIndexerConfig(
|
||||||
new HadoopIngestionSpec(
|
new HadoopIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"website",
|
.withDataSource("website")
|
||||||
MAPPER.convertValue(
|
.withParserMap(MAPPER.convertValue(
|
||||||
new StringInputRowParser(
|
new StringInputRowParser(
|
||||||
new CSVParseSpec(
|
new CSVParseSpec(
|
||||||
new TimestampSpec("timestamp", "yyyyMMddHH", null),
|
new TimestampSpec("timestamp", "yyyyMMddHH", null),
|
||||||
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host"))),
|
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host"))),
|
||||||
null,
|
null,
|
||||||
ImmutableList.of("timestamp", "host", "host2", "visited_num"),
|
ImmutableList.of("timestamp", "host", "host2", "visited_num"),
|
||||||
false,
|
false,
|
||||||
0
|
0
|
||||||
),
|
),
|
||||||
null
|
null
|
||||||
),
|
),
|
||||||
Map.class
|
Map.class
|
||||||
),
|
))
|
||||||
aggregators != null ? aggregators : new AggregatorFactory[]{
|
.withAggregators(aggregators != null ? aggregators : new AggregatorFactory[]{
|
||||||
new LongSumAggregatorFactory("visited_sum", "visited_num"),
|
new LongSumAggregatorFactory("visited_sum", "visited_num"),
|
||||||
new HyperUniquesAggregatorFactory("unique_hosts", "host2")
|
new HyperUniquesAggregatorFactory("unique_hosts", "host2")
|
||||||
},
|
})
|
||||||
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, ImmutableList.of(INTERVAL_FULL)),
|
.withGranularity(new UniformGranularitySpec(
|
||||||
null,
|
Granularities.DAY,
|
||||||
MAPPER
|
Granularities.NONE,
|
||||||
),
|
ImmutableList.of(INTERVAL_FULL)
|
||||||
|
))
|
||||||
|
.withObjectMapper(MAPPER)
|
||||||
|
.build(),
|
||||||
new HadoopIOConfig(
|
new HadoopIOConfig(
|
||||||
inputSpec,
|
inputSpec,
|
||||||
null,
|
null,
|
||||||
|
|
|
@ -32,7 +32,6 @@ import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularity;
|
import org.apache.druid.java.util.common.granularity.Granularity;
|
||||||
import org.apache.druid.java.util.common.granularity.PeriodGranularity;
|
import org.apache.druid.java.util.common.granularity.PeriodGranularity;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||||
|
@ -158,46 +157,45 @@ public class DetermineHashedPartitionsJobTest
|
||||||
}
|
}
|
||||||
|
|
||||||
HadoopIngestionSpec ingestionSpec = new HadoopIngestionSpec(
|
HadoopIngestionSpec ingestionSpec = new HadoopIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"test_schema",
|
.withDataSource("test_schema")
|
||||||
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
|
.withParserMap(HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
|
||||||
new StringInputRowParser(
|
new StringInputRowParser(
|
||||||
new DelimitedParseSpec(
|
new DelimitedParseSpec(
|
||||||
new TimestampSpec("ts", null, null),
|
new TimestampSpec("ts", null, null),
|
||||||
new DimensionsSpec(
|
new DimensionsSpec(
|
||||||
DimensionsSpec.getDefaultSchemas(ImmutableList.of(
|
DimensionsSpec.getDefaultSchemas(ImmutableList.of(
|
||||||
"market",
|
"market",
|
||||||
"quality",
|
"quality",
|
||||||
"placement",
|
"placement",
|
||||||
"placementish"
|
"placementish"
|
||||||
))
|
))
|
||||||
),
|
),
|
||||||
"\t",
|
"\t",
|
||||||
null,
|
null,
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
"ts",
|
"ts",
|
||||||
"market",
|
"market",
|
||||||
"quality",
|
"quality",
|
||||||
"placement",
|
"placement",
|
||||||
"placementish",
|
"placementish",
|
||||||
"index"
|
"index"
|
||||||
),
|
),
|
||||||
false,
|
false,
|
||||||
0
|
0
|
||||||
),
|
),
|
||||||
null
|
null
|
||||||
),
|
),
|
||||||
Map.class
|
Map.class
|
||||||
),
|
))
|
||||||
new AggregatorFactory[]{new DoubleSumAggregatorFactory("index", "index")},
|
.withAggregators(new DoubleSumAggregatorFactory("index", "index"))
|
||||||
new UniformGranularitySpec(
|
.withGranularity(new UniformGranularitySpec(
|
||||||
segmentGranularity,
|
segmentGranularity,
|
||||||
Granularities.NONE,
|
Granularities.NONE,
|
||||||
intervals
|
intervals
|
||||||
),
|
))
|
||||||
null,
|
.withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER)
|
||||||
HadoopDruidIndexerConfig.JSON_MAPPER
|
.build(),
|
||||||
),
|
|
||||||
new HadoopIOConfig(
|
new HadoopIOConfig(
|
||||||
ImmutableMap.of(
|
ImmutableMap.of(
|
||||||
"paths",
|
"paths",
|
||||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
|
||||||
import org.apache.druid.java.util.common.FileUtils;
|
import org.apache.druid.java.util.common.FileUtils;
|
||||||
import org.apache.druid.java.util.common.Intervals;
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||||
|
@ -280,33 +279,36 @@ public class DeterminePartitionsJobTest
|
||||||
|
|
||||||
config = new HadoopDruidIndexerConfig(
|
config = new HadoopDruidIndexerConfig(
|
||||||
new HadoopIngestionSpec(
|
new HadoopIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"website",
|
.withDataSource("website")
|
||||||
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
|
.withParserMap(
|
||||||
new StringInputRowParser(
|
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
|
||||||
new CSVParseSpec(
|
new StringInputRowParser(
|
||||||
new TimestampSpec("timestamp", "yyyyMMddHH", null),
|
new CSVParseSpec(
|
||||||
new DimensionsSpec(
|
new TimestampSpec("timestamp", "yyyyMMddHH", null),
|
||||||
DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "country"))
|
new DimensionsSpec(
|
||||||
),
|
DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "country"))
|
||||||
null,
|
),
|
||||||
ImmutableList.of("timestamp", "host", "country", "visited_num"),
|
null,
|
||||||
false,
|
ImmutableList.of("timestamp", "host", "country", "visited_num"),
|
||||||
0
|
false,
|
||||||
),
|
0
|
||||||
null
|
),
|
||||||
),
|
null
|
||||||
Map.class
|
),
|
||||||
),
|
Map.class
|
||||||
new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")},
|
)
|
||||||
new UniformGranularitySpec(
|
)
|
||||||
Granularities.DAY,
|
.withAggregators(new LongSumAggregatorFactory("visited_num", "visited_num"))
|
||||||
Granularities.NONE,
|
.withGranularity(
|
||||||
ImmutableList.of(Intervals.of(interval))
|
new UniformGranularitySpec(
|
||||||
),
|
Granularities.DAY,
|
||||||
null,
|
Granularities.NONE,
|
||||||
HadoopDruidIndexerConfig.JSON_MAPPER
|
ImmutableList.of(Intervals.of(interval))
|
||||||
),
|
)
|
||||||
|
)
|
||||||
|
.withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER)
|
||||||
|
.build(),
|
||||||
new HadoopIOConfig(
|
new HadoopIOConfig(
|
||||||
ImmutableMap.of(
|
ImmutableMap.of(
|
||||||
"paths",
|
"paths",
|
||||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec;
|
||||||
import org.apache.druid.java.util.common.FileUtils;
|
import org.apache.druid.java.util.common.FileUtils;
|
||||||
import org.apache.druid.java.util.common.Intervals;
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||||
|
@ -328,35 +327,36 @@ public class DetermineRangePartitionsJobTest
|
||||||
|
|
||||||
config = new HadoopDruidIndexerConfig(
|
config = new HadoopDruidIndexerConfig(
|
||||||
new HadoopIngestionSpec(
|
new HadoopIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"website",
|
.withDataSource("website")
|
||||||
null,
|
.withAggregators(new LongSumAggregatorFactory("visited_num", "visited_num"))
|
||||||
null,
|
.withGranularity(
|
||||||
new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")},
|
new UniformGranularitySpec(
|
||||||
new UniformGranularitySpec(
|
Granularities.DAY,
|
||||||
Granularities.DAY,
|
Granularities.NONE,
|
||||||
Granularities.NONE,
|
ImmutableList.of(Intervals.of(interval))
|
||||||
ImmutableList.of(Intervals.of(interval))
|
)
|
||||||
),
|
)
|
||||||
null,
|
.withParserMap(
|
||||||
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
|
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
|
||||||
new StringInputRowParser(
|
new StringInputRowParser(
|
||||||
new CSVParseSpec(
|
new CSVParseSpec(
|
||||||
new TimestampSpec("timestamp", "yyyyMMddHH", null),
|
new TimestampSpec("timestamp", "yyyyMMddHH", null),
|
||||||
new DimensionsSpec(
|
new DimensionsSpec(
|
||||||
DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "country"))
|
DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "country"))
|
||||||
),
|
),
|
||||||
null,
|
null,
|
||||||
ImmutableList.of("timestamp", "host", "country", "visited_num"),
|
ImmutableList.of("timestamp", "host", "country", "visited_num"),
|
||||||
false,
|
false,
|
||||||
0
|
0
|
||||||
),
|
),
|
||||||
null
|
null
|
||||||
),
|
),
|
||||||
Map.class
|
Map.class
|
||||||
),
|
)
|
||||||
HadoopDruidIndexerConfig.JSON_MAPPER
|
)
|
||||||
),
|
.withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER)
|
||||||
|
.build(),
|
||||||
new HadoopIOConfig(
|
new HadoopIOConfig(
|
||||||
ImmutableMap.of(
|
ImmutableMap.of(
|
||||||
"paths",
|
"paths",
|
||||||
|
|
|
@ -37,7 +37,6 @@ import org.apache.druid.jackson.DefaultObjectMapper;
|
||||||
import org.apache.druid.java.util.common.DateTimes;
|
import org.apache.druid.java.util.common.DateTimes;
|
||||||
import org.apache.druid.java.util.common.Intervals;
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||||
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
|
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
|
||||||
|
@ -217,18 +216,18 @@ public class HadoopDruidIndexerConfigTest
|
||||||
|
|
||||||
private static class HadoopIngestionSpecBuilder
|
private static class HadoopIngestionSpecBuilder
|
||||||
{
|
{
|
||||||
private static final DataSchema DATA_SCHEMA = new DataSchema(
|
private static final DataSchema DATA_SCHEMA =
|
||||||
"foo",
|
DataSchema.builder()
|
||||||
null,
|
.withDataSource("foo")
|
||||||
new AggregatorFactory[0],
|
.withGranularity(
|
||||||
new UniformGranularitySpec(
|
new UniformGranularitySpec(
|
||||||
Granularities.MINUTE,
|
Granularities.MINUTE,
|
||||||
Granularities.MINUTE,
|
Granularities.MINUTE,
|
||||||
ImmutableList.of(Intervals.of("2010-01-01/P1D"))
|
ImmutableList.of(Intervals.of("2010-01-01/P1D"))
|
||||||
),
|
)
|
||||||
null,
|
)
|
||||||
HadoopDruidIndexerConfigTest.JSON_MAPPER
|
.withObjectMapper(HadoopDruidIndexerConfigTest.JSON_MAPPER)
|
||||||
);
|
.build();
|
||||||
|
|
||||||
private static final HadoopIOConfig HADOOP_IO_CONFIG = new HadoopIOConfig(
|
private static final HadoopIOConfig HADOOP_IO_CONFIG = new HadoopIOConfig(
|
||||||
ImmutableMap.of("paths", "bar", "type", "static"),
|
ImmutableMap.of("paths", "bar", "type", "static"),
|
||||||
|
|
|
@ -31,7 +31,6 @@ import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.java.util.common.jackson.JacksonUtils;
|
import org.apache.druid.java.util.common.jackson.JacksonUtils;
|
||||||
import org.apache.druid.java.util.common.parsers.JSONPathSpec;
|
import org.apache.druid.java.util.common.parsers.JSONPathSpec;
|
||||||
import org.apache.druid.math.expr.ExprMacroTable;
|
import org.apache.druid.math.expr.ExprMacroTable;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import org.apache.druid.query.filter.SelectorDimFilter;
|
import org.apache.druid.query.filter.SelectorDimFilter;
|
||||||
import org.apache.druid.segment.TestHelper;
|
import org.apache.druid.segment.TestHelper;
|
||||||
|
@ -58,27 +57,29 @@ import java.util.stream.Collectors;
|
||||||
public class HadoopDruidIndexerMapperTest
|
public class HadoopDruidIndexerMapperTest
|
||||||
{
|
{
|
||||||
private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper();
|
private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper();
|
||||||
private static final DataSchema DATA_SCHEMA = new DataSchema(
|
private static final DataSchema DATA_SCHEMA =
|
||||||
"test_ds",
|
DataSchema.builder()
|
||||||
JSON_MAPPER.convertValue(
|
.withDataSource("test_ds")
|
||||||
new HadoopyStringInputRowParser(
|
.withParserMap(
|
||||||
new JSONParseSpec(
|
JSON_MAPPER.convertValue(
|
||||||
new TimestampSpec("t", "auto", null),
|
new HadoopyStringInputRowParser(
|
||||||
new DimensionsSpec(
|
new JSONParseSpec(
|
||||||
DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2"))
|
new TimestampSpec("t", "auto", null),
|
||||||
),
|
new DimensionsSpec(
|
||||||
new JSONPathSpec(true, ImmutableList.of()),
|
DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2"))
|
||||||
ImmutableMap.of(),
|
),
|
||||||
null
|
new JSONPathSpec(true, ImmutableList.of()),
|
||||||
)
|
ImmutableMap.of(),
|
||||||
),
|
null
|
||||||
JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
|
)
|
||||||
),
|
),
|
||||||
new AggregatorFactory[]{new CountAggregatorFactory("rows")},
|
JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
|
||||||
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
|
)
|
||||||
null,
|
)
|
||||||
JSON_MAPPER
|
.withAggregators(new CountAggregatorFactory("rows"))
|
||||||
);
|
.withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null))
|
||||||
|
.withObjectMapper(JSON_MAPPER)
|
||||||
|
.build();
|
||||||
|
|
||||||
private static final HadoopIOConfig IO_CONFIG = new HadoopIOConfig(
|
private static final HadoopIOConfig IO_CONFIG = new HadoopIOConfig(
|
||||||
JSON_MAPPER.convertValue(
|
JSON_MAPPER.convertValue(
|
||||||
|
|
|
@ -34,7 +34,6 @@ import org.apache.druid.indexing.overlord.Segments;
|
||||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||||
import org.apache.druid.java.util.common.Intervals;
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
|
@ -274,18 +273,17 @@ public class HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest
|
||||||
throws Exception
|
throws Exception
|
||||||
{
|
{
|
||||||
HadoopIngestionSpec spec = new HadoopIngestionSpec(
|
HadoopIngestionSpec spec = new HadoopIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"foo",
|
.withDataSource("foo")
|
||||||
null,
|
.withGranularity(
|
||||||
new AggregatorFactory[0],
|
new UniformGranularitySpec(
|
||||||
new UniformGranularitySpec(
|
Granularities.DAY,
|
||||||
Granularities.DAY,
|
null,
|
||||||
null,
|
ImmutableList.of(Intervals.of("2010-01-01/P1D"))
|
||||||
ImmutableList.of(Intervals.of("2010-01-01/P1D"))
|
)
|
||||||
),
|
)
|
||||||
null,
|
.withObjectMapper(jsonMapper)
|
||||||
jsonMapper
|
.build(),
|
||||||
),
|
|
||||||
new HadoopIOConfig(
|
new HadoopIOConfig(
|
||||||
jsonMapper.convertValue(datasourcePathSpec, Map.class),
|
jsonMapper.convertValue(datasourcePathSpec, Map.class),
|
||||||
null,
|
null,
|
||||||
|
|
|
@ -64,30 +64,33 @@ public class IndexGeneratorCombinerTest
|
||||||
{
|
{
|
||||||
HadoopDruidIndexerConfig config = new HadoopDruidIndexerConfig(
|
HadoopDruidIndexerConfig config = new HadoopDruidIndexerConfig(
|
||||||
new HadoopIngestionSpec(
|
new HadoopIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"website",
|
.withDataSource("website")
|
||||||
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
|
.withParserMap(
|
||||||
new StringInputRowParser(
|
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
|
||||||
new TimeAndDimsParseSpec(
|
new StringInputRowParser(
|
||||||
new TimestampSpec("timestamp", "yyyyMMddHH", null),
|
new TimeAndDimsParseSpec(
|
||||||
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "keywords")))
|
new TimestampSpec("timestamp", "yyyyMMddHH", null),
|
||||||
),
|
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "keywords")))
|
||||||
null
|
),
|
||||||
),
|
null
|
||||||
Map.class
|
),
|
||||||
),
|
Map.class
|
||||||
new AggregatorFactory[]{
|
)
|
||||||
new LongSumAggregatorFactory("visited_sum", "visited"),
|
)
|
||||||
new HyperUniquesAggregatorFactory("unique_hosts", "host")
|
.withAggregators(
|
||||||
},
|
new LongSumAggregatorFactory("visited_sum", "visited"),
|
||||||
new UniformGranularitySpec(
|
new HyperUniquesAggregatorFactory("unique_hosts", "host")
|
||||||
Granularities.DAY,
|
)
|
||||||
Granularities.NONE,
|
.withGranularity(
|
||||||
ImmutableList.of(Intervals.of("2010/2011"))
|
new UniformGranularitySpec(
|
||||||
),
|
Granularities.DAY,
|
||||||
null,
|
Granularities.NONE,
|
||||||
HadoopDruidIndexerConfig.JSON_MAPPER
|
ImmutableList.of(Intervals.of("2010/2011"))
|
||||||
),
|
)
|
||||||
|
)
|
||||||
|
.withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER)
|
||||||
|
.build(),
|
||||||
new HadoopIOConfig(
|
new HadoopIOConfig(
|
||||||
ImmutableMap.of(
|
ImmutableMap.of(
|
||||||
"paths",
|
"paths",
|
||||||
|
|
|
@ -506,17 +506,19 @@ public class IndexGeneratorJobTest
|
||||||
|
|
||||||
config = new HadoopDruidIndexerConfig(
|
config = new HadoopDruidIndexerConfig(
|
||||||
new HadoopIngestionSpec(
|
new HadoopIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
datasourceName,
|
.withDataSource(datasourceName)
|
||||||
mapper.convertValue(
|
.withParserMap(mapper.convertValue(inputRowParser, Map.class))
|
||||||
inputRowParser,
|
.withAggregators(aggs)
|
||||||
Map.class
|
.withGranularity(
|
||||||
),
|
new UniformGranularitySpec(
|
||||||
aggs,
|
Granularities.DAY,
|
||||||
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, ImmutableList.of(this.interval)),
|
Granularities.NONE,
|
||||||
null,
|
ImmutableList.of(interval)
|
||||||
mapper
|
)
|
||||||
),
|
)
|
||||||
|
.withObjectMapper(mapper)
|
||||||
|
.build(),
|
||||||
new HadoopIOConfig(
|
new HadoopIOConfig(
|
||||||
ImmutableMap.copyOf(inputSpec),
|
ImmutableMap.copyOf(inputSpec),
|
||||||
null,
|
null,
|
||||||
|
|
|
@ -34,7 +34,6 @@ import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.java.util.common.jackson.JacksonUtils;
|
import org.apache.druid.java.util.common.jackson.JacksonUtils;
|
||||||
import org.apache.druid.java.util.common.parsers.JSONPathSpec;
|
import org.apache.druid.java.util.common.parsers.JSONPathSpec;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||||
import org.apache.druid.segment.TestHelper;
|
import org.apache.druid.segment.TestHelper;
|
||||||
|
@ -68,27 +67,30 @@ import java.util.Map;
|
||||||
public class JobHelperTest
|
public class JobHelperTest
|
||||||
{
|
{
|
||||||
private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper();
|
private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper();
|
||||||
private static final DataSchema DATA_SCHEMA = new DataSchema(
|
|
||||||
"test_ds",
|
private static final DataSchema DATA_SCHEMA =
|
||||||
JSON_MAPPER.convertValue(
|
DataSchema.builder()
|
||||||
new HadoopyStringInputRowParser(
|
.withDataSource("test_ds")
|
||||||
new JSONParseSpec(
|
.withParserMap(
|
||||||
new TimestampSpec("t", "auto", null),
|
JSON_MAPPER.convertValue(
|
||||||
new DimensionsSpec(
|
new HadoopyStringInputRowParser(
|
||||||
DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2"))
|
new JSONParseSpec(
|
||||||
),
|
new TimestampSpec("t", "auto", null),
|
||||||
new JSONPathSpec(true, ImmutableList.of()),
|
new DimensionsSpec(
|
||||||
ImmutableMap.of(),
|
DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2"))
|
||||||
null
|
),
|
||||||
)
|
new JSONPathSpec(true, ImmutableList.of()),
|
||||||
),
|
ImmutableMap.of(),
|
||||||
JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
|
null
|
||||||
),
|
)
|
||||||
new AggregatorFactory[]{new CountAggregatorFactory("rows")},
|
),
|
||||||
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
|
JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
|
||||||
null,
|
)
|
||||||
JSON_MAPPER
|
)
|
||||||
);
|
.withAggregators(new CountAggregatorFactory("rows"))
|
||||||
|
.withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null))
|
||||||
|
.withObjectMapper(JSON_MAPPER)
|
||||||
|
.build();
|
||||||
|
|
||||||
private static final HadoopIOConfig IO_CONFIG = new HadoopIOConfig(
|
private static final HadoopIOConfig IO_CONFIG = new HadoopIOConfig(
|
||||||
JSON_MAPPER.convertValue(
|
JSON_MAPPER.convertValue(
|
||||||
|
@ -123,27 +125,34 @@ public class JobHelperTest
|
||||||
dataFile = temporaryFolder.newFile();
|
dataFile = temporaryFolder.newFile();
|
||||||
config = new HadoopDruidIndexerConfig(
|
config = new HadoopDruidIndexerConfig(
|
||||||
new HadoopIngestionSpec(
|
new HadoopIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"website",
|
.withDataSource("website")
|
||||||
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
|
.withParserMap(
|
||||||
new StringInputRowParser(
|
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
|
||||||
new CSVParseSpec(
|
new StringInputRowParser(
|
||||||
new TimestampSpec("timestamp", "yyyyMMddHH", null),
|
new CSVParseSpec(
|
||||||
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host"))),
|
new TimestampSpec("timestamp", "yyyyMMddHH", null),
|
||||||
null,
|
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host"))),
|
||||||
ImmutableList.of("timestamp", "host", "visited_num"),
|
null,
|
||||||
false,
|
ImmutableList.of("timestamp", "host", "visited_num"),
|
||||||
0
|
false,
|
||||||
),
|
0
|
||||||
null
|
),
|
||||||
),
|
null
|
||||||
Map.class
|
),
|
||||||
),
|
Map.class
|
||||||
new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")},
|
)
|
||||||
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, ImmutableList.of(this.interval)),
|
)
|
||||||
null,
|
.withAggregators(new LongSumAggregatorFactory("visited_num", "visited_num"))
|
||||||
HadoopDruidIndexerConfig.JSON_MAPPER
|
.withGranularity(
|
||||||
),
|
new UniformGranularitySpec(
|
||||||
|
Granularities.DAY,
|
||||||
|
Granularities.NONE,
|
||||||
|
ImmutableList.of(this.interval)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER)
|
||||||
|
.build(),
|
||||||
new HadoopIOConfig(
|
new HadoopIOConfig(
|
||||||
ImmutableMap.of(
|
ImmutableMap.of(
|
||||||
"paths",
|
"paths",
|
||||||
|
|
|
@ -165,7 +165,7 @@ public class DatasourceRecordReaderSegmentReaderTest
|
||||||
Assert.assertEquals(18, count);
|
Assert.assertEquals(18, count);
|
||||||
|
|
||||||
// Check the index
|
// Check the index
|
||||||
Assert.assertEquals(9, index.size());
|
Assert.assertEquals(9, index.numRows());
|
||||||
final IncrementalIndexSegment queryable = new IncrementalIndexSegment(index, SegmentId.dummy("test"));
|
final IncrementalIndexSegment queryable = new IncrementalIndexSegment(index, SegmentId.dummy("test"));
|
||||||
final List<String> dimensions = index.getDimensionNames(false);
|
final List<String> dimensions = index.getDimensionNames(false);
|
||||||
Assert.assertEquals(2, dimensions.size());
|
Assert.assertEquals(2, dimensions.size());
|
||||||
|
|
|
@ -44,7 +44,6 @@ import org.apache.druid.initialization.Initialization;
|
||||||
import org.apache.druid.java.util.common.ISE;
|
import org.apache.druid.java.util.common.ISE;
|
||||||
import org.apache.druid.java.util.common.Intervals;
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||||
|
@ -308,33 +307,34 @@ public class DatasourcePathSpecTest
|
||||||
{
|
{
|
||||||
return new HadoopDruidIndexerConfig(
|
return new HadoopDruidIndexerConfig(
|
||||||
new HadoopIngestionSpec(
|
new HadoopIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
ingestionSpec1.getDataSource(),
|
.withDataSource(ingestionSpec1.getDataSource())
|
||||||
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
|
.withParserMap(
|
||||||
new StringInputRowParser(
|
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
|
||||||
new CSVParseSpec(
|
new StringInputRowParser(
|
||||||
new TimestampSpec("timestamp", "yyyyMMddHH", null),
|
new CSVParseSpec(
|
||||||
DimensionsSpec.EMPTY,
|
new TimestampSpec("timestamp", "yyyyMMddHH", null),
|
||||||
null,
|
DimensionsSpec.EMPTY,
|
||||||
ImmutableList.of("timestamp", "host", "visited"),
|
null,
|
||||||
false,
|
ImmutableList.of("timestamp", "host", "visited"),
|
||||||
0
|
false,
|
||||||
),
|
0
|
||||||
null
|
),
|
||||||
),
|
null
|
||||||
Map.class
|
),
|
||||||
),
|
Map.class
|
||||||
new AggregatorFactory[]{
|
)
|
||||||
new LongSumAggregatorFactory("visited_sum", "visited")
|
)
|
||||||
},
|
.withAggregators(new LongSumAggregatorFactory("visited_sum", "visited"))
|
||||||
new UniformGranularitySpec(
|
.withGranularity(
|
||||||
Granularities.DAY,
|
new UniformGranularitySpec(
|
||||||
Granularities.NONE,
|
Granularities.DAY,
|
||||||
ImmutableList.of(Intervals.of("2000/3000"))
|
Granularities.NONE,
|
||||||
),
|
ImmutableList.of(Intervals.of("2000/3000"))
|
||||||
null,
|
)
|
||||||
HadoopDruidIndexerConfig.JSON_MAPPER
|
)
|
||||||
),
|
.withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER)
|
||||||
|
.build(),
|
||||||
new HadoopIOConfig(
|
new HadoopIOConfig(
|
||||||
ImmutableMap.of(
|
ImmutableMap.of(
|
||||||
"paths",
|
"paths",
|
||||||
|
|
|
@ -34,7 +34,6 @@ import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularity;
|
import org.apache.druid.java.util.common.granularity.Granularity;
|
||||||
import org.apache.druid.java.util.common.granularity.PeriodGranularity;
|
import org.apache.druid.java.util.common.granularity.PeriodGranularity;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||||
import org.apache.hadoop.mapreduce.Job;
|
import org.apache.hadoop.mapreduce.Job;
|
||||||
|
@ -152,18 +151,17 @@ public class GranularityPathSpecTest
|
||||||
{
|
{
|
||||||
UserGroupInformation.setLoginUser(UserGroupInformation.createUserForTesting("test", new String[]{"testGroup"}));
|
UserGroupInformation.setLoginUser(UserGroupInformation.createUserForTesting("test", new String[]{"testGroup"}));
|
||||||
HadoopIngestionSpec spec = new HadoopIngestionSpec(
|
HadoopIngestionSpec spec = new HadoopIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"foo",
|
.withDataSource("foo")
|
||||||
null,
|
.withGranularity(
|
||||||
new AggregatorFactory[0],
|
new UniformGranularitySpec(
|
||||||
new UniformGranularitySpec(
|
Granularities.DAY,
|
||||||
Granularities.DAY,
|
Granularities.MINUTE,
|
||||||
Granularities.MINUTE,
|
ImmutableList.of(Intervals.of("2015-11-06T00:00Z/2015-11-07T00:00Z"))
|
||||||
ImmutableList.of(Intervals.of("2015-11-06T00:00Z/2015-11-07T00:00Z"))
|
)
|
||||||
),
|
)
|
||||||
null,
|
.withObjectMapper(jsonMapper)
|
||||||
jsonMapper
|
.build(),
|
||||||
),
|
|
||||||
new HadoopIOConfig(null, null, null),
|
new HadoopIOConfig(null, null, null),
|
||||||
DEFAULT_TUNING_CONFIG
|
DEFAULT_TUNING_CONFIG
|
||||||
);
|
);
|
||||||
|
@ -204,18 +202,17 @@ public class GranularityPathSpecTest
|
||||||
{
|
{
|
||||||
UserGroupInformation.setLoginUser(UserGroupInformation.createUserForTesting("test", new String[]{"testGroup"}));
|
UserGroupInformation.setLoginUser(UserGroupInformation.createUserForTesting("test", new String[]{"testGroup"}));
|
||||||
HadoopIngestionSpec spec = new HadoopIngestionSpec(
|
HadoopIngestionSpec spec = new HadoopIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"foo",
|
.withDataSource("foo")
|
||||||
null,
|
.withGranularity(
|
||||||
new AggregatorFactory[0],
|
new UniformGranularitySpec(
|
||||||
new UniformGranularitySpec(
|
Granularities.DAY,
|
||||||
Granularities.DAY,
|
Granularities.ALL,
|
||||||
Granularities.ALL,
|
ImmutableList.of(Intervals.of("2015-01-01T11Z/2015-01-02T05Z"))
|
||||||
ImmutableList.of(Intervals.of("2015-01-01T11Z/2015-01-02T05Z"))
|
)
|
||||||
),
|
)
|
||||||
null,
|
.withObjectMapper(jsonMapper)
|
||||||
jsonMapper
|
.build(),
|
||||||
),
|
|
||||||
new HadoopIOConfig(null, null, null),
|
new HadoopIOConfig(null, null, null),
|
||||||
DEFAULT_TUNING_CONFIG
|
DEFAULT_TUNING_CONFIG
|
||||||
);
|
);
|
||||||
|
|
|
@ -24,7 +24,6 @@ import org.apache.druid.indexer.HadoopDruidIndexerConfig;
|
||||||
import org.apache.druid.indexer.HadoopIOConfig;
|
import org.apache.druid.indexer.HadoopIOConfig;
|
||||||
import org.apache.druid.indexer.HadoopIngestionSpec;
|
import org.apache.druid.indexer.HadoopIngestionSpec;
|
||||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.hadoop.mapreduce.Job;
|
import org.apache.hadoop.mapreduce.Job;
|
||||||
import org.apache.hadoop.mapreduce.lib.input.MultipleInputs;
|
import org.apache.hadoop.mapreduce.lib.input.MultipleInputs;
|
||||||
|
@ -54,7 +53,7 @@ public class StaticPathSpecTest
|
||||||
Job job = new Job();
|
Job job = new Job();
|
||||||
StaticPathSpec pathSpec = new StaticPathSpec("/a/c,/a/b/{c,d}", null);
|
StaticPathSpec pathSpec = new StaticPathSpec("/a/c,/a/b/{c,d}", null);
|
||||||
|
|
||||||
DataSchema schema = new DataSchema("ds", null, new AggregatorFactory[0], null, null, jsonMapper);
|
DataSchema schema = DataSchema.builder().withDataSource("ds").withObjectMapper(jsonMapper).build();
|
||||||
HadoopIOConfig io = new HadoopIOConfig(null, null, null);
|
HadoopIOConfig io = new HadoopIOConfig(null, null, null);
|
||||||
pathSpec.addInputPaths(new HadoopDruidIndexerConfig(new HadoopIngestionSpec(schema, io, null)), job);
|
pathSpec.addInputPaths(new HadoopDruidIndexerConfig(new HadoopIngestionSpec(schema, io, null)), job);
|
||||||
|
|
||||||
|
|
|
@ -67,14 +67,13 @@ public class InputSourceSampler
|
||||||
{
|
{
|
||||||
private static final String SAMPLER_DATA_SOURCE = "sampler";
|
private static final String SAMPLER_DATA_SOURCE = "sampler";
|
||||||
|
|
||||||
private static final DataSchema DEFAULT_DATA_SCHEMA = new DataSchema(
|
|
||||||
SAMPLER_DATA_SOURCE,
|
private static final DataSchema DEFAULT_DATA_SCHEMA =
|
||||||
new TimestampSpec(null, null, null),
|
DataSchema.builder()
|
||||||
new DimensionsSpec(null),
|
.withDataSource(SAMPLER_DATA_SOURCE)
|
||||||
null,
|
.withTimestamp(new TimestampSpec(null, null, null))
|
||||||
null,
|
.withDimensions(DimensionsSpec.builder().build())
|
||||||
null
|
.build();
|
||||||
);
|
|
||||||
|
|
||||||
// We want to be able to sort the list of processed results back into the same order that we read them from the input
|
// We want to be able to sort the list of processed results back into the same order that we read them from the input
|
||||||
// source so that the rows in the data loader are not always changing. To do this, we add a temporary column to the
|
// source so that the rows in the data loader are not always changing. To do this, we add a temporary column to the
|
||||||
|
|
|
@ -33,7 +33,6 @@ import org.apache.druid.indexing.common.task.IndexTask;
|
||||||
import org.apache.druid.indexing.common.task.TaskResource;
|
import org.apache.druid.indexing.common.task.TaskResource;
|
||||||
import org.apache.druid.indexing.common.task.TuningConfigBuilder;
|
import org.apache.druid.indexing.common.task.TuningConfigBuilder;
|
||||||
import org.apache.druid.indexing.overlord.SegmentPublishResult;
|
import org.apache.druid.indexing.overlord.SegmentPublishResult;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.segment.IndexSpec;
|
import org.apache.druid.segment.IndexSpec;
|
||||||
import org.apache.druid.segment.SegmentSchemaMapping;
|
import org.apache.druid.segment.SegmentSchemaMapping;
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
|
@ -62,7 +61,7 @@ public class TestIndexTask extends IndexTask
|
||||||
id,
|
id,
|
||||||
taskResource,
|
taskResource,
|
||||||
new IndexIngestionSpec(
|
new IndexIngestionSpec(
|
||||||
new DataSchema(dataSource, null, new AggregatorFactory[]{}, null, null, mapper),
|
DataSchema.builder().withDataSource(dataSource).withObjectMapper(mapper).build(),
|
||||||
new IndexTask.IndexIOConfig(
|
new IndexTask.IndexIOConfig(
|
||||||
new LocalInputSource(new File("lol"), "rofl"),
|
new LocalInputSource(new File("lol"), "rofl"),
|
||||||
new JsonInputFormat(null, null, null, null, null),
|
new JsonInputFormat(null, null, null, null, null),
|
||||||
|
|
|
@ -939,18 +939,19 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
new ParallelIndexIngestionSpec(
|
new ParallelIndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
DATA_SOURCE,
|
.withDataSource(DATA_SOURCE)
|
||||||
new TimestampSpec("ts", "auto", null),
|
.withTimestamp(new TimestampSpec("ts", "auto", null))
|
||||||
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim"))),
|
.withDimensions(DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim")))
|
||||||
new AggregatorFactory[]{new LongSumAggregatorFactory("val", "val")},
|
.withAggregators(new LongSumAggregatorFactory("val", "val"))
|
||||||
new UniformGranularitySpec(
|
.withGranularity(
|
||||||
Granularities.HOUR,
|
new UniformGranularitySpec(
|
||||||
Granularities.MINUTE,
|
Granularities.HOUR,
|
||||||
ImmutableList.of(INTERVAL_TO_INDEX)
|
Granularities.MINUTE,
|
||||||
),
|
ImmutableList.of(INTERVAL_TO_INDEX)
|
||||||
null
|
)
|
||||||
),
|
)
|
||||||
|
.build(),
|
||||||
ioConfig,
|
ioConfig,
|
||||||
tuningConfig
|
tuningConfig
|
||||||
),
|
),
|
||||||
|
|
|
@ -1913,6 +1913,7 @@ public class CompactionTaskTest
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
|
null,
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
|
@ -27,7 +27,6 @@ import org.apache.druid.indexer.HadoopIngestionSpec;
|
||||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||||
import org.apache.druid.java.util.common.Intervals;
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||||
import org.apache.druid.server.security.Action;
|
import org.apache.druid.server.security.Action;
|
||||||
|
@ -50,15 +49,19 @@ public class HadoopIndexTaskTest
|
||||||
final HadoopIndexTask task = new HadoopIndexTask(
|
final HadoopIndexTask task = new HadoopIndexTask(
|
||||||
null,
|
null,
|
||||||
new HadoopIngestionSpec(
|
new HadoopIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"foo", null, new AggregatorFactory[0], new UniformGranularitySpec(
|
.withDataSource("foo")
|
||||||
Granularities.DAY,
|
.withGranularity(
|
||||||
null,
|
new UniformGranularitySpec(
|
||||||
ImmutableList.of(Intervals.of("2010-01-01/P1D"))
|
Granularities.DAY,
|
||||||
),
|
null,
|
||||||
null,
|
ImmutableList.of(Intervals.of("2010-01-01/P1D"))
|
||||||
jsonMapper
|
)
|
||||||
), new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null), null
|
)
|
||||||
|
.withObjectMapper(jsonMapper)
|
||||||
|
.build(),
|
||||||
|
new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null),
|
||||||
|
null
|
||||||
),
|
),
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
|
|
|
@ -25,7 +25,6 @@ import org.apache.druid.data.input.impl.NoopInputSource;
|
||||||
import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig;
|
import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig;
|
||||||
import org.apache.druid.indexing.common.task.IndexTask.IndexIngestionSpec;
|
import org.apache.druid.indexing.common.task.IndexTask.IndexIngestionSpec;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
|
import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
|
||||||
import org.junit.Rule;
|
import org.junit.Rule;
|
||||||
|
@ -45,14 +44,11 @@ public class IndexIngestionSpecTest
|
||||||
"Cannot use parser and inputSource together. Try using inputFormat instead of parser."
|
"Cannot use parser and inputSource together. Try using inputFormat instead of parser."
|
||||||
);
|
);
|
||||||
final IndexIngestionSpec spec = new IndexIngestionSpec(
|
final IndexIngestionSpec spec = new IndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"dataSource",
|
.withDataSource("dataSource")
|
||||||
ImmutableMap.of("fake", "parser map"),
|
.withParserMap(ImmutableMap.of("fake", "parser map"))
|
||||||
new AggregatorFactory[0],
|
.withGranularity(new ArbitraryGranularitySpec(Granularities.NONE, null))
|
||||||
new ArbitraryGranularitySpec(Granularities.NONE, null),
|
.build(),
|
||||||
null,
|
|
||||||
null
|
|
||||||
),
|
|
||||||
new IndexIOConfig(
|
new IndexIOConfig(
|
||||||
new NoopInputSource(),
|
new NoopInputSource(),
|
||||||
new NoopInputFormat(),
|
new NoopInputFormat(),
|
||||||
|
@ -69,14 +65,11 @@ public class IndexIngestionSpecTest
|
||||||
expectedException.expect(IllegalArgumentException.class);
|
expectedException.expect(IllegalArgumentException.class);
|
||||||
expectedException.expectMessage("Cannot use parser and inputSource together.");
|
expectedException.expectMessage("Cannot use parser and inputSource together.");
|
||||||
final IndexIngestionSpec spec = new IndexIngestionSpec(
|
final IndexIngestionSpec spec = new IndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"dataSource",
|
.withDataSource("dataSource")
|
||||||
ImmutableMap.of("fake", "parser map"),
|
.withParserMap(ImmutableMap.of("fake", "parser map"))
|
||||||
new AggregatorFactory[0],
|
.withGranularity(new ArbitraryGranularitySpec(Granularities.NONE, null))
|
||||||
new ArbitraryGranularitySpec(Granularities.NONE, null),
|
.build(),
|
||||||
null,
|
|
||||||
null
|
|
||||||
),
|
|
||||||
new IndexIOConfig(
|
new IndexIOConfig(
|
||||||
new NoopInputSource(),
|
new NoopInputSource(),
|
||||||
null,
|
null,
|
||||||
|
|
|
@ -166,6 +166,25 @@ public class IndexTaskTest extends IngestionTestBase
|
||||||
0
|
0
|
||||||
);
|
);
|
||||||
|
|
||||||
|
private static final DataSchema DATA_SCHEMA =
|
||||||
|
DataSchema.builder()
|
||||||
|
.withDataSource("test-json")
|
||||||
|
.withTimestamp(DEFAULT_TIMESTAMP_SPEC)
|
||||||
|
.withDimensions(
|
||||||
|
new StringDimensionSchema("ts"),
|
||||||
|
new StringDimensionSchema("dim"),
|
||||||
|
new LongDimensionSchema("valDim")
|
||||||
|
)
|
||||||
|
.withAggregators(new LongSumAggregatorFactory("valMet", "val"))
|
||||||
|
.withGranularity(
|
||||||
|
new UniformGranularitySpec(
|
||||||
|
Granularities.DAY,
|
||||||
|
Granularities.MINUTE,
|
||||||
|
Collections.singletonList(Intervals.of("2014/P1D"))
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.build();
|
||||||
|
|
||||||
@Parameterized.Parameters(name = "{0}, useInputFormatApi={1}")
|
@Parameterized.Parameters(name = "{0}, useInputFormatApi={1}")
|
||||||
public static Iterable<Object[]> constructorFeeder()
|
public static Iterable<Object[]> constructorFeeder()
|
||||||
{
|
{
|
||||||
|
@ -225,24 +244,7 @@ public class IndexTaskTest extends IngestionTestBase
|
||||||
{
|
{
|
||||||
IndexTask indexTask = createIndexTask(
|
IndexTask indexTask = createIndexTask(
|
||||||
new IndexIngestionSpec(
|
new IndexIngestionSpec(
|
||||||
new DataSchema(
|
DATA_SCHEMA,
|
||||||
"test-json",
|
|
||||||
DEFAULT_TIMESTAMP_SPEC,
|
|
||||||
new DimensionsSpec(
|
|
||||||
ImmutableList.of(
|
|
||||||
new StringDimensionSchema("ts"),
|
|
||||||
new StringDimensionSchema("dim"),
|
|
||||||
new LongDimensionSchema("valDim")
|
|
||||||
)
|
|
||||||
),
|
|
||||||
new AggregatorFactory[]{new LongSumAggregatorFactory("valMet", "val")},
|
|
||||||
new UniformGranularitySpec(
|
|
||||||
Granularities.DAY,
|
|
||||||
Granularities.MINUTE,
|
|
||||||
Collections.singletonList(Intervals.of("2014/P1D"))
|
|
||||||
),
|
|
||||||
null
|
|
||||||
),
|
|
||||||
new IndexIOConfig(
|
new IndexIOConfig(
|
||||||
new LocalInputSource(tmpDir, "druid*"),
|
new LocalInputSource(tmpDir, "druid*"),
|
||||||
DEFAULT_INPUT_FORMAT,
|
DEFAULT_INPUT_FORMAT,
|
||||||
|
@ -275,24 +277,7 @@ public class IndexTaskTest extends IngestionTestBase
|
||||||
|
|
||||||
IndexTask indexTask = createIndexTask(
|
IndexTask indexTask = createIndexTask(
|
||||||
new IndexIngestionSpec(
|
new IndexIngestionSpec(
|
||||||
new DataSchema(
|
DATA_SCHEMA,
|
||||||
"test-json",
|
|
||||||
DEFAULT_TIMESTAMP_SPEC,
|
|
||||||
new DimensionsSpec(
|
|
||||||
ImmutableList.of(
|
|
||||||
new StringDimensionSchema("ts"),
|
|
||||||
new StringDimensionSchema("dim"),
|
|
||||||
new LongDimensionSchema("valDim")
|
|
||||||
)
|
|
||||||
),
|
|
||||||
new AggregatorFactory[]{new LongSumAggregatorFactory("valMet", "val")},
|
|
||||||
new UniformGranularitySpec(
|
|
||||||
Granularities.DAY,
|
|
||||||
Granularities.MINUTE,
|
|
||||||
Collections.singletonList(Intervals.of("2014/P1D"))
|
|
||||||
),
|
|
||||||
null
|
|
||||||
),
|
|
||||||
new IndexIOConfig(
|
new IndexIOConfig(
|
||||||
new LocalInputSource(tmpDir, "druid*"),
|
new LocalInputSource(tmpDir, "druid*"),
|
||||||
DEFAULT_INPUT_FORMAT,
|
DEFAULT_INPUT_FORMAT,
|
||||||
|
@ -337,24 +322,7 @@ public class IndexTaskTest extends IngestionTestBase
|
||||||
|
|
||||||
IndexTask indexTask = createIndexTask(
|
IndexTask indexTask = createIndexTask(
|
||||||
new IndexIngestionSpec(
|
new IndexIngestionSpec(
|
||||||
new DataSchema(
|
DATA_SCHEMA,
|
||||||
"test-json",
|
|
||||||
DEFAULT_TIMESTAMP_SPEC,
|
|
||||||
new DimensionsSpec(
|
|
||||||
ImmutableList.of(
|
|
||||||
new StringDimensionSchema("ts"),
|
|
||||||
new StringDimensionSchema("dim"),
|
|
||||||
new LongDimensionSchema("valDim")
|
|
||||||
)
|
|
||||||
),
|
|
||||||
new AggregatorFactory[]{new LongSumAggregatorFactory("valMet", "val")},
|
|
||||||
new UniformGranularitySpec(
|
|
||||||
Granularities.DAY,
|
|
||||||
Granularities.MINUTE,
|
|
||||||
Collections.singletonList(Intervals.of("2014/P1D"))
|
|
||||||
),
|
|
||||||
null
|
|
||||||
),
|
|
||||||
new IndexIOConfig(
|
new IndexIOConfig(
|
||||||
new LocalInputSource(tmpDir, "druid*"),
|
new LocalInputSource(tmpDir, "druid*"),
|
||||||
DEFAULT_INPUT_FORMAT,
|
DEFAULT_INPUT_FORMAT,
|
||||||
|
@ -2698,20 +2666,20 @@ public class IndexTaskTest extends IngestionTestBase
|
||||||
if (inputFormat != null) {
|
if (inputFormat != null) {
|
||||||
Preconditions.checkArgument(parseSpec == null, "Can't use parseSpec");
|
Preconditions.checkArgument(parseSpec == null, "Can't use parseSpec");
|
||||||
return new IndexIngestionSpec(
|
return new IndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
DATASOURCE,
|
.withDataSource(DATASOURCE)
|
||||||
Preconditions.checkNotNull(timestampSpec, "timestampSpec"),
|
.withTimestamp(Preconditions.checkNotNull(timestampSpec, "timestampSpec"))
|
||||||
Preconditions.checkNotNull(dimensionsSpec, "dimensionsSpec"),
|
.withDimensions(Preconditions.checkNotNull(dimensionsSpec, "dimensionsSpec"))
|
||||||
new AggregatorFactory[]{
|
.withAggregators(new LongSumAggregatorFactory("val", "val"))
|
||||||
new LongSumAggregatorFactory("val", "val")
|
.withGranularity(
|
||||||
},
|
granularitySpec != null ? granularitySpec : new UniformGranularitySpec(
|
||||||
granularitySpec != null ? granularitySpec : new UniformGranularitySpec(
|
Granularities.DAY,
|
||||||
Granularities.DAY,
|
Granularities.MINUTE,
|
||||||
Granularities.MINUTE,
|
Collections.singletonList(Intervals.of("2014/2015"))
|
||||||
Collections.singletonList(Intervals.of("2014/2015"))
|
)
|
||||||
),
|
)
|
||||||
transformSpec
|
.withTransform(transformSpec)
|
||||||
),
|
.build(),
|
||||||
new IndexIOConfig(
|
new IndexIOConfig(
|
||||||
new LocalInputSource(baseDir, "druid*"),
|
new LocalInputSource(baseDir, "druid*"),
|
||||||
inputFormat,
|
inputFormat,
|
||||||
|
@ -2723,22 +2691,21 @@ public class IndexTaskTest extends IngestionTestBase
|
||||||
} else {
|
} else {
|
||||||
parseSpec = parseSpec != null ? parseSpec : DEFAULT_PARSE_SPEC;
|
parseSpec = parseSpec != null ? parseSpec : DEFAULT_PARSE_SPEC;
|
||||||
return new IndexIngestionSpec(
|
return new IndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
DATASOURCE,
|
.withDataSource(DATASOURCE)
|
||||||
parseSpec.getTimestampSpec(),
|
.withTimestamp(parseSpec.getTimestampSpec())
|
||||||
parseSpec.getDimensionsSpec(),
|
.withDimensions(parseSpec.getDimensionsSpec())
|
||||||
new AggregatorFactory[]{
|
.withAggregators(new LongSumAggregatorFactory("val", "val"))
|
||||||
new LongSumAggregatorFactory("val", "val")
|
.withGranularity(
|
||||||
},
|
granularitySpec != null ? granularitySpec : new UniformGranularitySpec(
|
||||||
granularitySpec != null ? granularitySpec : new UniformGranularitySpec(
|
Granularities.DAY,
|
||||||
Granularities.DAY,
|
Granularities.MINUTE,
|
||||||
Granularities.MINUTE,
|
Collections.singletonList(Intervals.of("2014/2015"))
|
||||||
Collections.singletonList(Intervals.of("2014/2015"))
|
)
|
||||||
),
|
)
|
||||||
transformSpec,
|
.withTransform(transformSpec)
|
||||||
null,
|
.withObjectMapper(objectMapper)
|
||||||
objectMapper
|
.build(),
|
||||||
),
|
|
||||||
new IndexIOConfig(
|
new IndexIOConfig(
|
||||||
new LocalInputSource(baseDir, "druid*"),
|
new LocalInputSource(baseDir, "druid*"),
|
||||||
createInputFormatFromParseSpec(parseSpec),
|
createInputFormatFromParseSpec(parseSpec),
|
||||||
|
|
|
@ -38,7 +38,6 @@ import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig;
|
||||||
import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig;
|
import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig;
|
||||||
import org.apache.druid.java.util.common.Intervals;
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||||
import org.apache.druid.segment.IndexSpec;
|
import org.apache.druid.segment.IndexSpec;
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
|
@ -220,18 +219,19 @@ public class TaskSerdeTest
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
new IndexIngestionSpec(
|
new IndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"foo",
|
.withDataSource("foo")
|
||||||
new TimestampSpec(null, null, null),
|
.withTimestamp(new TimestampSpec(null, null, null))
|
||||||
DimensionsSpec.EMPTY,
|
.withDimensions(DimensionsSpec.EMPTY)
|
||||||
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
|
.withAggregators(new DoubleSumAggregatorFactory("met", "met"))
|
||||||
new UniformGranularitySpec(
|
.withGranularity(
|
||||||
Granularities.DAY,
|
new UniformGranularitySpec(
|
||||||
null,
|
Granularities.DAY,
|
||||||
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
|
null,
|
||||||
),
|
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
|
||||||
null
|
)
|
||||||
),
|
)
|
||||||
|
.build(),
|
||||||
new IndexIOConfig(new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false),
|
new IndexIOConfig(new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false),
|
||||||
TuningConfigBuilder.forIndexTask()
|
TuningConfigBuilder.forIndexTask()
|
||||||
.withMaxRowsInMemory(10)
|
.withMaxRowsInMemory(10)
|
||||||
|
@ -288,18 +288,19 @@ public class TaskSerdeTest
|
||||||
null,
|
null,
|
||||||
new TaskResource("rofl", 2),
|
new TaskResource("rofl", 2),
|
||||||
new IndexIngestionSpec(
|
new IndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"foo",
|
.withDataSource("foo")
|
||||||
new TimestampSpec(null, null, null),
|
.withTimestamp(new TimestampSpec(null, null, null))
|
||||||
DimensionsSpec.EMPTY,
|
.withDimensions(DimensionsSpec.EMPTY)
|
||||||
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
|
.withAggregators(new DoubleSumAggregatorFactory("met", "met"))
|
||||||
new UniformGranularitySpec(
|
.withGranularity(
|
||||||
Granularities.DAY,
|
new UniformGranularitySpec(
|
||||||
null,
|
Granularities.DAY,
|
||||||
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
|
null,
|
||||||
),
|
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
|
||||||
null
|
)
|
||||||
),
|
)
|
||||||
|
.build(),
|
||||||
new IndexIOConfig(new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false),
|
new IndexIOConfig(new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false),
|
||||||
TuningConfigBuilder.forIndexTask()
|
TuningConfigBuilder.forIndexTask()
|
||||||
.withMaxRowsInMemory(10)
|
.withMaxRowsInMemory(10)
|
||||||
|
@ -412,15 +413,19 @@ public class TaskSerdeTest
|
||||||
final HadoopIndexTask task = new HadoopIndexTask(
|
final HadoopIndexTask task = new HadoopIndexTask(
|
||||||
null,
|
null,
|
||||||
new HadoopIngestionSpec(
|
new HadoopIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"foo", null, new AggregatorFactory[0], new UniformGranularitySpec(
|
.withDataSource("foo")
|
||||||
Granularities.DAY,
|
.withGranularity(
|
||||||
null,
|
new UniformGranularitySpec(
|
||||||
ImmutableList.of(Intervals.of("2010-01-01/P1D"))
|
Granularities.DAY,
|
||||||
),
|
null,
|
||||||
null,
|
ImmutableList.of(Intervals.of("2010-01-01/P1D"))
|
||||||
jsonMapper
|
)
|
||||||
), new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null), null
|
)
|
||||||
|
.withObjectMapper(jsonMapper)
|
||||||
|
.build(),
|
||||||
|
new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null),
|
||||||
|
null
|
||||||
),
|
),
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
|
@ -454,19 +459,18 @@ public class TaskSerdeTest
|
||||||
final HadoopIndexTask task = new HadoopIndexTask(
|
final HadoopIndexTask task = new HadoopIndexTask(
|
||||||
null,
|
null,
|
||||||
new HadoopIngestionSpec(
|
new HadoopIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"foo",
|
.withDataSource("foo")
|
||||||
null,
|
.withGranularity(
|
||||||
null,
|
new UniformGranularitySpec(
|
||||||
new AggregatorFactory[0],
|
Granularities.DAY,
|
||||||
new UniformGranularitySpec(
|
null, ImmutableList.of(Intervals.of("2010-01-01/P1D"))
|
||||||
Granularities.DAY,
|
)
|
||||||
null, ImmutableList.of(Intervals.of("2010-01-01/P1D"))
|
)
|
||||||
),
|
.withObjectMapper(jsonMapper)
|
||||||
null,
|
.build(),
|
||||||
null,
|
new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null),
|
||||||
jsonMapper
|
null
|
||||||
), new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null), null
|
|
||||||
),
|
),
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
|
|
|
@ -221,14 +221,13 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
|
||||||
dropExisting
|
dropExisting
|
||||||
);
|
);
|
||||||
ingestionSpec = new ParallelIndexIngestionSpec(
|
ingestionSpec = new ParallelIndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
DATASOURCE,
|
.withDataSource(DATASOURCE)
|
||||||
timestampSpec,
|
.withTimestamp(timestampSpec)
|
||||||
dimensionsSpec,
|
.withDimensions(dimensionsSpec)
|
||||||
DEFAULT_METRICS_SPEC,
|
.withAggregators(DEFAULT_METRICS_SPEC)
|
||||||
granularitySpec,
|
.withGranularity(granularitySpec)
|
||||||
null
|
.build(),
|
||||||
),
|
|
||||||
ioConfig,
|
ioConfig,
|
||||||
tuningConfig
|
tuningConfig
|
||||||
);
|
);
|
||||||
|
@ -241,14 +240,13 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
|
||||||
dropExisting
|
dropExisting
|
||||||
);
|
);
|
||||||
ingestionSpec = new ParallelIndexIngestionSpec(
|
ingestionSpec = new ParallelIndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
DATASOURCE,
|
.withDataSource(DATASOURCE)
|
||||||
parseSpec.getTimestampSpec(),
|
.withTimestamp(parseSpec.getTimestampSpec())
|
||||||
parseSpec.getDimensionsSpec(),
|
.withDimensions(parseSpec.getDimensionsSpec())
|
||||||
DEFAULT_METRICS_SPEC,
|
.withAggregators(DEFAULT_METRICS_SPEC)
|
||||||
granularitySpec,
|
.withGranularity(granularitySpec)
|
||||||
null
|
.build(),
|
||||||
),
|
|
||||||
ioConfig,
|
ioConfig,
|
||||||
tuningConfig
|
tuningConfig
|
||||||
);
|
);
|
||||||
|
|
|
@ -41,7 +41,6 @@ import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.java.util.common.guava.Comparators;
|
import org.apache.druid.java.util.common.guava.Comparators;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
|
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
|
||||||
|
@ -250,14 +249,13 @@ public class HashPartitionTaskKillTest extends AbstractMultiPhaseParallelIndexin
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
ingestionSpec = new ParallelIndexIngestionSpec(
|
ingestionSpec = new ParallelIndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
DATASOURCE,
|
.withDataSource(DATASOURCE)
|
||||||
timestampSpec,
|
.withTimestamp(timestampSpec)
|
||||||
dimensionsSpec,
|
.withDimensions(dimensionsSpec)
|
||||||
new AggregatorFactory[]{new LongSumAggregatorFactory("val", "val")},
|
.withAggregators(new LongSumAggregatorFactory("val", "val"))
|
||||||
granularitySpec,
|
.withGranularity(granularitySpec)
|
||||||
null
|
.build(),
|
||||||
),
|
|
||||||
ioConfig,
|
ioConfig,
|
||||||
tuningConfig
|
tuningConfig
|
||||||
);
|
);
|
||||||
|
@ -271,16 +269,13 @@ public class HashPartitionTaskKillTest extends AbstractMultiPhaseParallelIndexin
|
||||||
);
|
);
|
||||||
//noinspection unchecked
|
//noinspection unchecked
|
||||||
ingestionSpec = new ParallelIndexIngestionSpec(
|
ingestionSpec = new ParallelIndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"dataSource",
|
.withDataSource("dataSource")
|
||||||
parseSpec.getTimestampSpec(),
|
.withTimestamp(parseSpec.getTimestampSpec())
|
||||||
parseSpec.getDimensionsSpec(),
|
.withDimensions(parseSpec.getDimensionsSpec())
|
||||||
new AggregatorFactory[]{
|
.withAggregators(new LongSumAggregatorFactory("val", "val"))
|
||||||
new LongSumAggregatorFactory("val", "val")
|
.withGranularity(granularitySpec)
|
||||||
},
|
.build(),
|
||||||
granularitySpec,
|
|
||||||
null
|
|
||||||
),
|
|
||||||
ioConfig,
|
ioConfig,
|
||||||
tuningConfig
|
tuningConfig
|
||||||
);
|
);
|
||||||
|
|
|
@ -127,18 +127,19 @@ public class MultiPhaseParallelIndexingWithNullColumnTest extends AbstractMultiP
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
new ParallelIndexIngestionSpec(
|
new ParallelIndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
DATASOURCE,
|
.withDataSource(DATASOURCE)
|
||||||
TIMESTAMP_SPEC,
|
.withTimestamp(DEFAULT_TIMESTAMP_SPEC)
|
||||||
DIMENSIONS_SPEC.withDimensions(dimensionSchemas),
|
.withDimensions(DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas))
|
||||||
DEFAULT_METRICS_SPEC,
|
.withAggregators(DEFAULT_METRICS_SPEC)
|
||||||
new UniformGranularitySpec(
|
.withGranularity(
|
||||||
Granularities.DAY,
|
new UniformGranularitySpec(
|
||||||
Granularities.MINUTE,
|
Granularities.DAY,
|
||||||
INTERVAL_TO_INDEX
|
Granularities.MINUTE,
|
||||||
),
|
INTERVAL_TO_INDEX
|
||||||
null
|
)
|
||||||
),
|
)
|
||||||
|
.build(),
|
||||||
new ParallelIndexIOConfig(
|
new ParallelIndexIOConfig(
|
||||||
getInputSource(),
|
getInputSource(),
|
||||||
JSON_FORMAT,
|
JSON_FORMAT,
|
||||||
|
@ -177,18 +178,21 @@ public class MultiPhaseParallelIndexingWithNullColumnTest extends AbstractMultiP
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
new ParallelIndexIngestionSpec(
|
new ParallelIndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
DATASOURCE,
|
.withDataSource(DATASOURCE)
|
||||||
TIMESTAMP_SPEC,
|
.withTimestamp(TIMESTAMP_SPEC)
|
||||||
new DimensionsSpec.Builder().setDimensions(dimensionSchemas).setIncludeAllDimensions(true).build(),
|
.withDimensions(
|
||||||
DEFAULT_METRICS_SPEC,
|
DimensionsSpec.builder().setDimensions(dimensionSchemas).setIncludeAllDimensions(true).build()
|
||||||
new UniformGranularitySpec(
|
)
|
||||||
Granularities.DAY,
|
.withAggregators(DEFAULT_METRICS_SPEC)
|
||||||
Granularities.MINUTE,
|
.withGranularity(
|
||||||
INTERVAL_TO_INDEX
|
new UniformGranularitySpec(
|
||||||
),
|
Granularities.DAY,
|
||||||
null
|
Granularities.MINUTE,
|
||||||
),
|
INTERVAL_TO_INDEX
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.build(),
|
||||||
new ParallelIndexIOConfig(
|
new ParallelIndexIOConfig(
|
||||||
getInputSource(),
|
getInputSource(),
|
||||||
new JsonInputFormat(
|
new JsonInputFormat(
|
||||||
|
@ -237,18 +241,21 @@ public class MultiPhaseParallelIndexingWithNullColumnTest extends AbstractMultiP
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
new ParallelIndexIngestionSpec(
|
new ParallelIndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
DATASOURCE,
|
.withDataSource(DATASOURCE)
|
||||||
TIMESTAMP_SPEC,
|
.withTimestamp(TIMESTAMP_SPEC)
|
||||||
new DimensionsSpec.Builder().setIncludeAllDimensions(true).build(),
|
.withDimensions(
|
||||||
DEFAULT_METRICS_SPEC,
|
DimensionsSpec.builder().setIncludeAllDimensions(true).build()
|
||||||
new UniformGranularitySpec(
|
)
|
||||||
Granularities.DAY,
|
.withAggregators(DEFAULT_METRICS_SPEC)
|
||||||
Granularities.MINUTE,
|
.withGranularity(
|
||||||
null
|
new UniformGranularitySpec(
|
||||||
),
|
Granularities.DAY,
|
||||||
null
|
Granularities.MINUTE,
|
||||||
),
|
null
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.build(),
|
||||||
new ParallelIndexIOConfig(
|
new ParallelIndexIOConfig(
|
||||||
getInputSource(),
|
getInputSource(),
|
||||||
new JsonInputFormat(
|
new JsonInputFormat(
|
||||||
|
@ -303,20 +310,23 @@ public class MultiPhaseParallelIndexingWithNullColumnTest extends AbstractMultiP
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
new ParallelIndexIngestionSpec(
|
new ParallelIndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
DATASOURCE,
|
.withDataSource(DATASOURCE)
|
||||||
TIMESTAMP_SPEC,
|
.withTimestamp(TIMESTAMP_SPEC)
|
||||||
DIMENSIONS_SPEC.withDimensions(
|
.withDimensions(
|
||||||
DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "unknownDim"))
|
DIMENSIONS_SPEC.withDimensions(
|
||||||
),
|
DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "unknownDim"))
|
||||||
DEFAULT_METRICS_SPEC,
|
)
|
||||||
new UniformGranularitySpec(
|
)
|
||||||
Granularities.DAY,
|
.withAggregators(DEFAULT_METRICS_SPEC)
|
||||||
Granularities.MINUTE,
|
.withGranularity(
|
||||||
INTERVAL_TO_INDEX
|
new UniformGranularitySpec(
|
||||||
),
|
Granularities.DAY,
|
||||||
null
|
Granularities.MINUTE,
|
||||||
),
|
INTERVAL_TO_INDEX
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.build(),
|
||||||
new ParallelIndexIOConfig(
|
new ParallelIndexIOConfig(
|
||||||
getInputSource(),
|
getInputSource(),
|
||||||
JSON_FORMAT,
|
JSON_FORMAT,
|
||||||
|
|
|
@ -36,7 +36,6 @@ import org.apache.druid.indexing.common.task.TuningConfigBuilder;
|
||||||
import org.apache.druid.java.util.common.Intervals;
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.java.util.common.Pair;
|
import org.apache.druid.java.util.common.Pair;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||||
|
@ -147,20 +146,19 @@ public class ParallelIndexSupervisorTaskKillTest extends AbstractParallelIndexSu
|
||||||
final int numTotalSubTasks = inputSource.estimateNumSplits(new NoopInputFormat(), null);
|
final int numTotalSubTasks = inputSource.estimateNumSplits(new NoopInputFormat(), null);
|
||||||
// set up ingestion spec
|
// set up ingestion spec
|
||||||
final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec(
|
final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"dataSource",
|
.withDataSource("dataSource")
|
||||||
DEFAULT_TIMESTAMP_SPEC,
|
.withTimestamp(DEFAULT_TIMESTAMP_SPEC)
|
||||||
DEFAULT_DIMENSIONS_SPEC,
|
.withDimensions(DEFAULT_DIMENSIONS_SPEC)
|
||||||
new AggregatorFactory[]{
|
.withAggregators(new LongSumAggregatorFactory("val", "val"))
|
||||||
new LongSumAggregatorFactory("val", "val")
|
.withGranularity(
|
||||||
},
|
new UniformGranularitySpec(
|
||||||
new UniformGranularitySpec(
|
Granularities.DAY,
|
||||||
Granularities.DAY,
|
Granularities.MINUTE,
|
||||||
Granularities.MINUTE,
|
interval == null ? null : Collections.singletonList(interval)
|
||||||
interval == null ? null : Collections.singletonList(interval)
|
)
|
||||||
),
|
)
|
||||||
null
|
.build(),
|
||||||
),
|
|
||||||
ioConfig,
|
ioConfig,
|
||||||
TuningConfigBuilder.forParallelIndexTask().withMaxNumConcurrentSubTasks(numTotalSubTasks).build()
|
TuningConfigBuilder.forParallelIndexTask().withMaxNumConcurrentSubTasks(numTotalSubTasks).build()
|
||||||
);
|
);
|
||||||
|
|
|
@ -45,7 +45,6 @@ import org.apache.druid.java.util.common.DateTimes;
|
||||||
import org.apache.druid.java.util.common.ISE;
|
import org.apache.druid.java.util.common.ISE;
|
||||||
import org.apache.druid.java.util.common.Intervals;
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||||
|
@ -400,20 +399,19 @@ public class ParallelIndexSupervisorTaskResourceTest extends AbstractParallelInd
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec(
|
final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"dataSource",
|
.withDataSource("dataSource")
|
||||||
DEFAULT_TIMESTAMP_SPEC,
|
.withTimestamp(DEFAULT_TIMESTAMP_SPEC)
|
||||||
DEFAULT_DIMENSIONS_SPEC,
|
.withDimensions(DEFAULT_DIMENSIONS_SPEC)
|
||||||
new AggregatorFactory[]{
|
.withAggregators(new LongSumAggregatorFactory("val", "val"))
|
||||||
new LongSumAggregatorFactory("val", "val")
|
.withGranularity(
|
||||||
},
|
new UniformGranularitySpec(
|
||||||
new UniformGranularitySpec(
|
Granularities.DAY,
|
||||||
Granularities.DAY,
|
Granularities.MINUTE,
|
||||||
Granularities.MINUTE,
|
interval == null ? null : Collections.singletonList(interval)
|
||||||
interval == null ? null : Collections.singletonList(interval)
|
)
|
||||||
),
|
)
|
||||||
null
|
.build(),
|
||||||
),
|
|
||||||
ioConfig,
|
ioConfig,
|
||||||
TuningConfigBuilder.forParallelIndexTask().withMaxNumConcurrentSubTasks(NUM_SUB_TASKS).build()
|
TuningConfigBuilder.forParallelIndexTask().withMaxNumConcurrentSubTasks(NUM_SUB_TASKS).build()
|
||||||
);
|
);
|
||||||
|
|
|
@ -34,7 +34,6 @@ import org.apache.druid.indexing.common.task.TaskResource;
|
||||||
import org.apache.druid.indexing.common.task.TuningConfigBuilder;
|
import org.apache.druid.indexing.common.task.TuningConfigBuilder;
|
||||||
import org.apache.druid.java.util.common.Intervals;
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||||
|
@ -220,16 +219,19 @@ public class ParallelIndexSupervisorTaskSerdeTest
|
||||||
|
|
||||||
ParallelIndexIngestionSpec build()
|
ParallelIndexIngestionSpec build()
|
||||||
{
|
{
|
||||||
DataSchema dataSchema = new DataSchema(
|
DataSchema dataSchema = DataSchema.builder()
|
||||||
"dataSource",
|
.withDataSource("datasource")
|
||||||
TIMESTAMP_SPEC,
|
.withTimestamp(TIMESTAMP_SPEC)
|
||||||
DIMENSIONS_SPEC,
|
.withDimensions(DIMENSIONS_SPEC)
|
||||||
new AggregatorFactory[]{
|
.withAggregators(new LongSumAggregatorFactory("val", "val"))
|
||||||
new LongSumAggregatorFactory("val", "val")
|
.withGranularity(
|
||||||
},
|
new UniformGranularitySpec(
|
||||||
new UniformGranularitySpec(Granularities.DAY, Granularities.MINUTE, inputIntervals),
|
Granularities.DAY,
|
||||||
null
|
Granularities.MINUTE,
|
||||||
);
|
inputIntervals
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.build();
|
||||||
|
|
||||||
ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder
|
ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder
|
||||||
.forParallelIndexTask()
|
.forParallelIndexTask()
|
||||||
|
|
|
@ -263,14 +263,11 @@ public class ParallelIndexSupervisorTaskTest
|
||||||
.withLogParseExceptions(false)
|
.withLogParseExceptions(false)
|
||||||
.build();
|
.build();
|
||||||
final ParallelIndexIngestionSpec indexIngestionSpec = new ParallelIndexIngestionSpec(
|
final ParallelIndexIngestionSpec indexIngestionSpec = new ParallelIndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"datasource",
|
.withDataSource("datasource")
|
||||||
new TimestampSpec(null, null, null),
|
.withTimestamp(new TimestampSpec(null, null, null))
|
||||||
DimensionsSpec.EMPTY,
|
.withDimensions(DimensionsSpec.EMPTY)
|
||||||
null,
|
.build(),
|
||||||
null,
|
|
||||||
null
|
|
||||||
),
|
|
||||||
ioConfig,
|
ioConfig,
|
||||||
tuningConfig
|
tuningConfig
|
||||||
);
|
);
|
||||||
|
@ -325,25 +322,24 @@ public class ParallelIndexSupervisorTaskTest
|
||||||
expectedException.expect(IAE.class);
|
expectedException.expect(IAE.class);
|
||||||
expectedException.expectMessage("Cannot use parser and inputSource together. Try using inputFormat instead of parser.");
|
expectedException.expectMessage("Cannot use parser and inputSource together. Try using inputFormat instead of parser.");
|
||||||
new ParallelIndexIngestionSpec(
|
new ParallelIndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"datasource",
|
.withDataSource("datasource")
|
||||||
mapper.convertValue(
|
.withParserMap(
|
||||||
new StringInputRowParser(
|
mapper.convertValue(
|
||||||
new JSONParseSpec(
|
new StringInputRowParser(
|
||||||
new TimestampSpec(null, null, null),
|
new JSONParseSpec(
|
||||||
DimensionsSpec.EMPTY,
|
new TimestampSpec(null, null, null),
|
||||||
null,
|
DimensionsSpec.EMPTY,
|
||||||
null,
|
null,
|
||||||
null
|
null,
|
||||||
)
|
null
|
||||||
),
|
)
|
||||||
Map.class
|
),
|
||||||
),
|
Map.class
|
||||||
null,
|
)
|
||||||
null,
|
)
|
||||||
null,
|
.withObjectMapper(mapper)
|
||||||
mapper
|
.build(),
|
||||||
),
|
|
||||||
ioConfig,
|
ioConfig,
|
||||||
tuningConfig
|
tuningConfig
|
||||||
);
|
);
|
||||||
|
@ -559,14 +555,11 @@ public class ParallelIndexSupervisorTaskTest
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
final ParallelIndexIngestionSpec indexIngestionSpec = new ParallelIndexIngestionSpec(
|
final ParallelIndexIngestionSpec indexIngestionSpec = new ParallelIndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"datasource",
|
.withDataSource("datasource")
|
||||||
new TimestampSpec(null, null, null),
|
.withTimestamp(new TimestampSpec(null, null, null))
|
||||||
DimensionsSpec.EMPTY,
|
.withDimensions(DimensionsSpec.EMPTY)
|
||||||
null,
|
.build(),
|
||||||
null,
|
|
||||||
null
|
|
||||||
),
|
|
||||||
ioConfig,
|
ioConfig,
|
||||||
tuningConfig
|
tuningConfig
|
||||||
);
|
);
|
||||||
|
|
|
@ -31,11 +31,9 @@ import org.apache.druid.indexing.common.TestUtils;
|
||||||
import org.apache.druid.indexing.common.task.TaskResource;
|
import org.apache.druid.indexing.common.task.TaskResource;
|
||||||
import org.apache.druid.java.util.common.Intervals;
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
|
import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
|
||||||
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
|
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
|
||||||
import org.apache.druid.segment.transform.TransformSpec;
|
|
||||||
import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec;
|
import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec;
|
||||||
import org.apache.druid.timeline.partition.HashPartitionFunction;
|
import org.apache.druid.timeline.partition.HashPartitionFunction;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
@ -97,16 +95,13 @@ class ParallelIndexTestingFactory
|
||||||
DimensionsSpec.getDefaultSchemas(ImmutableList.of(SCHEMA_DIMENSION))
|
DimensionsSpec.getDefaultSchemas(ImmutableList.of(SCHEMA_DIMENSION))
|
||||||
);
|
);
|
||||||
|
|
||||||
return new DataSchema(
|
return DataSchema.builder()
|
||||||
DATASOURCE,
|
.withDataSource(DATASOURCE)
|
||||||
timestampSpec,
|
.withTimestamp(timestampSpec)
|
||||||
dimensionsSpec,
|
.withDimensions(dimensionsSpec)
|
||||||
new AggregatorFactory[]{},
|
.withGranularity(granularitySpec)
|
||||||
granularitySpec,
|
.withObjectMapper(NESTED_OBJECT_MAPPER)
|
||||||
TransformSpec.NONE,
|
.build();
|
||||||
null,
|
|
||||||
NESTED_OBJECT_MAPPER
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
static ParallelIndexIngestionSpec createIngestionSpec(
|
static ParallelIndexIngestionSpec createIngestionSpec(
|
||||||
|
|
|
@ -39,7 +39,6 @@ import org.apache.druid.indexing.common.task.TaskResource;
|
||||||
import org.apache.druid.java.util.common.ISE;
|
import org.apache.druid.java.util.common.ISE;
|
||||||
import org.apache.druid.java.util.common.Intervals;
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
|
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
|
||||||
|
@ -329,14 +328,13 @@ public class RangePartitionTaskKillTest extends AbstractMultiPhaseParallelIndexi
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
ingestionSpec = new ParallelIndexIngestionSpec(
|
ingestionSpec = new ParallelIndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
DATASOURCE,
|
.withDataSource(DATASOURCE)
|
||||||
timestampSpec,
|
.withTimestamp(timestampSpec)
|
||||||
dimensionsSpec,
|
.withDimensions(dimensionsSpec)
|
||||||
new AggregatorFactory[]{new LongSumAggregatorFactory("val", "val")},
|
.withAggregators(new LongSumAggregatorFactory("val", "val"))
|
||||||
granularitySpec,
|
.withGranularity(granularitySpec)
|
||||||
null
|
.build(),
|
||||||
),
|
|
||||||
ioConfig,
|
ioConfig,
|
||||||
tuningConfig
|
tuningConfig
|
||||||
);
|
);
|
||||||
|
|
|
@ -43,7 +43,6 @@ import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularity;
|
import org.apache.druid.java.util.common.granularity.Granularity;
|
||||||
import org.apache.druid.java.util.common.parsers.JSONPathSpec;
|
import org.apache.druid.java.util.common.parsers.JSONPathSpec;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||||
import org.apache.druid.segment.DataSegmentsWithSchemas;
|
import org.apache.druid.segment.DataSegmentsWithSchemas;
|
||||||
|
@ -391,20 +390,19 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
new ParallelIndexIngestionSpec(
|
new ParallelIndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"dataSource",
|
.withDataSource("dataSource")
|
||||||
DEFAULT_TIMESTAMP_SPEC,
|
.withTimestamp(DEFAULT_TIMESTAMP_SPEC)
|
||||||
DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas),
|
.withDimensions(DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas))
|
||||||
new AggregatorFactory[]{
|
.withAggregators(new LongSumAggregatorFactory("val", "val"))
|
||||||
new LongSumAggregatorFactory("val", "val")
|
.withGranularity(
|
||||||
},
|
new UniformGranularitySpec(
|
||||||
new UniformGranularitySpec(
|
Granularities.DAY,
|
||||||
Granularities.DAY,
|
Granularities.MINUTE,
|
||||||
Granularities.MINUTE,
|
Collections.singletonList(Intervals.of("2017-12/P1M"))
|
||||||
Collections.singletonList(Intervals.of("2017-12/P1M"))
|
)
|
||||||
),
|
)
|
||||||
null
|
.build(),
|
||||||
),
|
|
||||||
new ParallelIndexIOConfig(
|
new ParallelIndexIOConfig(
|
||||||
new SettableSplittableLocalInputSource(inputDir, VALID_INPUT_SOURCE_FILTER, true),
|
new SettableSplittableLocalInputSource(inputDir, VALID_INPUT_SOURCE_FILTER, true),
|
||||||
DEFAULT_INPUT_FORMAT,
|
DEFAULT_INPUT_FORMAT,
|
||||||
|
@ -444,20 +442,19 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
new ParallelIndexIngestionSpec(
|
new ParallelIndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"dataSource",
|
.withDataSource("dataSource")
|
||||||
DEFAULT_TIMESTAMP_SPEC,
|
.withTimestamp(DEFAULT_TIMESTAMP_SPEC)
|
||||||
DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas),
|
.withDimensions(DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas))
|
||||||
new AggregatorFactory[]{
|
.withAggregators(new LongSumAggregatorFactory("val", "val"))
|
||||||
new LongSumAggregatorFactory("val", "val")
|
.withGranularity(
|
||||||
},
|
new UniformGranularitySpec(
|
||||||
new UniformGranularitySpec(
|
Granularities.DAY,
|
||||||
Granularities.DAY,
|
Granularities.MINUTE,
|
||||||
Granularities.MINUTE,
|
Collections.singletonList(Intervals.of("2017-12/P1M"))
|
||||||
Collections.singletonList(Intervals.of("2017-12/P1M"))
|
)
|
||||||
),
|
)
|
||||||
null
|
.build(),
|
||||||
),
|
|
||||||
new ParallelIndexIOConfig(
|
new ParallelIndexIOConfig(
|
||||||
new SettableSplittableLocalInputSource(inputDir, VALID_INPUT_SOURCE_FILTER, true),
|
new SettableSplittableLocalInputSource(inputDir, VALID_INPUT_SOURCE_FILTER, true),
|
||||||
DEFAULT_INPUT_FORMAT,
|
DEFAULT_INPUT_FORMAT,
|
||||||
|
@ -785,21 +782,24 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
new ParallelIndexIngestionSpec(
|
new ParallelIndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"dataSource",
|
.withDataSource("dataSource")
|
||||||
DEFAULT_TIMESTAMP_SPEC,
|
.withTimestamp(DEFAULT_TIMESTAMP_SPEC)
|
||||||
DimensionsSpec.builder()
|
.withDimensions(
|
||||||
.setDefaultSchemaDimensions(ImmutableList.of("ts", "explicitDim"))
|
DimensionsSpec.builder()
|
||||||
.setIncludeAllDimensions(true)
|
.setDefaultSchemaDimensions(ImmutableList.of("ts", "explicitDim"))
|
||||||
.build(),
|
.setIncludeAllDimensions(true)
|
||||||
new AggregatorFactory[]{new CountAggregatorFactory("cnt")},
|
.build()
|
||||||
new UniformGranularitySpec(
|
)
|
||||||
Granularities.DAY,
|
.withAggregators(new CountAggregatorFactory("cnt"))
|
||||||
Granularities.MINUTE,
|
.withGranularity(
|
||||||
Collections.singletonList(interval)
|
new UniformGranularitySpec(
|
||||||
),
|
Granularities.DAY,
|
||||||
null
|
Granularities.MINUTE,
|
||||||
),
|
Collections.singletonList(interval)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.build(),
|
||||||
new ParallelIndexIOConfig(
|
new ParallelIndexIOConfig(
|
||||||
new SettableSplittableLocalInputSource(inputDir, "*.json", true),
|
new SettableSplittableLocalInputSource(inputDir, "*.json", true),
|
||||||
new JsonInputFormat(
|
new JsonInputFormat(
|
||||||
|
@ -868,21 +868,24 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
new ParallelIndexIngestionSpec(
|
new ParallelIndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"dataSource",
|
.withDataSource("dataSource")
|
||||||
DEFAULT_TIMESTAMP_SPEC,
|
.withTimestamp(DEFAULT_TIMESTAMP_SPEC)
|
||||||
DimensionsSpec.builder()
|
.withDimensions(
|
||||||
.setDefaultSchemaDimensions(ImmutableList.of("ts", "explicitDim"))
|
DimensionsSpec.builder()
|
||||||
.useSchemaDiscovery(true)
|
.setDefaultSchemaDimensions(ImmutableList.of("ts", "explicitDim"))
|
||||||
.build(),
|
.useSchemaDiscovery(true)
|
||||||
new AggregatorFactory[]{new CountAggregatorFactory("cnt")},
|
.build()
|
||||||
new UniformGranularitySpec(
|
)
|
||||||
Granularities.DAY,
|
.withAggregators(new CountAggregatorFactory("cnt"))
|
||||||
Granularities.MINUTE,
|
.withGranularity(
|
||||||
Collections.singletonList(interval)
|
new UniformGranularitySpec(
|
||||||
),
|
Granularities.DAY,
|
||||||
null
|
Granularities.MINUTE,
|
||||||
),
|
Collections.singletonList(interval)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.build(),
|
||||||
new ParallelIndexIOConfig(
|
new ParallelIndexIOConfig(
|
||||||
new SettableSplittableLocalInputSource(inputDir, "*.json", true),
|
new SettableSplittableLocalInputSource(inputDir, "*.json", true),
|
||||||
new JsonInputFormat(
|
new JsonInputFormat(
|
||||||
|
@ -948,20 +951,19 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
|
||||||
final ParallelIndexIngestionSpec ingestionSpec;
|
final ParallelIndexIngestionSpec ingestionSpec;
|
||||||
if (useInputFormatApi) {
|
if (useInputFormatApi) {
|
||||||
ingestionSpec = new ParallelIndexIngestionSpec(
|
ingestionSpec = new ParallelIndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"dataSource",
|
.withDataSource("dataSource")
|
||||||
DEFAULT_TIMESTAMP_SPEC,
|
.withTimestamp(DEFAULT_TIMESTAMP_SPEC)
|
||||||
DEFAULT_DIMENSIONS_SPEC,
|
.withDimensions(DEFAULT_DIMENSIONS_SPEC)
|
||||||
new AggregatorFactory[]{
|
.withAggregators(new LongSumAggregatorFactory("val", "val"))
|
||||||
new LongSumAggregatorFactory("val", "val")
|
.withGranularity(
|
||||||
},
|
new UniformGranularitySpec(
|
||||||
new UniformGranularitySpec(
|
segmentGranularity,
|
||||||
segmentGranularity,
|
Granularities.MINUTE,
|
||||||
Granularities.MINUTE,
|
interval == null ? null : Collections.singletonList(interval)
|
||||||
interval == null ? null : Collections.singletonList(interval)
|
)
|
||||||
),
|
)
|
||||||
null
|
.build(),
|
||||||
),
|
|
||||||
new ParallelIndexIOConfig(
|
new ParallelIndexIOConfig(
|
||||||
new SettableSplittableLocalInputSource(inputDir, inputSourceFilter, splittableInputSource),
|
new SettableSplittableLocalInputSource(inputDir, inputSourceFilter, splittableInputSource),
|
||||||
DEFAULT_INPUT_FORMAT,
|
DEFAULT_INPUT_FORMAT,
|
||||||
|
@ -972,18 +974,19 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
ingestionSpec = new ParallelIndexIngestionSpec(
|
ingestionSpec = new ParallelIndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"dataSource",
|
.withDataSource("dataSource")
|
||||||
DEFAULT_TIMESTAMP_SPEC,
|
.withTimestamp(DEFAULT_TIMESTAMP_SPEC)
|
||||||
DEFAULT_DIMENSIONS_SPEC,
|
.withDimensions(DEFAULT_DIMENSIONS_SPEC)
|
||||||
DEFAULT_METRICS_SPEC,
|
.withAggregators(DEFAULT_METRICS_SPEC)
|
||||||
new UniformGranularitySpec(
|
.withGranularity(
|
||||||
segmentGranularity,
|
new UniformGranularitySpec(
|
||||||
Granularities.MINUTE,
|
segmentGranularity,
|
||||||
interval == null ? null : Collections.singletonList(interval)
|
Granularities.MINUTE,
|
||||||
),
|
interval == null ? null : Collections.singletonList(interval)
|
||||||
null
|
)
|
||||||
),
|
)
|
||||||
|
.build(),
|
||||||
new ParallelIndexIOConfig(
|
new ParallelIndexIOConfig(
|
||||||
new LocalInputSource(inputDir, inputSourceFilter),
|
new LocalInputSource(inputDir, inputSourceFilter),
|
||||||
createInputFormatFromParseSpec(DEFAULT_PARSE_SPEC),
|
createInputFormatFromParseSpec(DEFAULT_PARSE_SPEC),
|
||||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.druid.data.input.impl.JsonInputFormat;
|
||||||
import org.apache.druid.data.input.impl.LocalInputSource;
|
import org.apache.druid.data.input.impl.LocalInputSource;
|
||||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||||
import org.apache.druid.indexing.common.TestUtils;
|
import org.apache.druid.indexing.common.TestUtils;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.server.security.Action;
|
import org.apache.druid.server.security.Action;
|
||||||
import org.apache.druid.server.security.Resource;
|
import org.apache.druid.server.security.Resource;
|
||||||
|
@ -48,14 +47,11 @@ public class SinglePhaseSubTaskSpecTest
|
||||||
"groupId",
|
"groupId",
|
||||||
"supervisorTaskId",
|
"supervisorTaskId",
|
||||||
new ParallelIndexIngestionSpec(
|
new ParallelIndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"dataSource",
|
.withDataSource("dataSource")
|
||||||
new TimestampSpec(null, null, null),
|
.withTimestamp(new TimestampSpec(null, null, null))
|
||||||
new DimensionsSpec(null),
|
.withDimensions(DimensionsSpec.builder().build())
|
||||||
new AggregatorFactory[0],
|
.build(),
|
||||||
null,
|
|
||||||
null
|
|
||||||
),
|
|
||||||
new ParallelIndexIOConfig(
|
new ParallelIndexIOConfig(
|
||||||
new LocalInputSource(new File("baseDir"), "filter"),
|
new LocalInputSource(new File("baseDir"), "filter"),
|
||||||
new JsonInputFormat(null, null, null, null, null),
|
new JsonInputFormat(null, null, null, null, null),
|
||||||
|
|
|
@ -67,8 +67,7 @@ public class TombstoneHelperTest
|
||||||
GranularitySpec granularitySpec = new UniformGranularitySpec(Granularities.DAY, null, false,
|
GranularitySpec granularitySpec = new UniformGranularitySpec(Granularities.DAY, null, false,
|
||||||
Collections.singletonList(interval)
|
Collections.singletonList(interval)
|
||||||
);
|
);
|
||||||
DataSchema dataSchema =
|
DataSchema dataSchema = DataSchema.builder().withDataSource("test").withGranularity(granularitySpec).build();
|
||||||
new DataSchema("test", null, null, null, granularitySpec, null);
|
|
||||||
// no segments will be pushed when all rows are thrown away, assume that:
|
// no segments will be pushed when all rows are thrown away, assume that:
|
||||||
List<DataSegment> pushedSegments = Collections.emptyList();
|
List<DataSegment> pushedSegments = Collections.emptyList();
|
||||||
|
|
||||||
|
@ -93,8 +92,7 @@ public class TombstoneHelperTest
|
||||||
GranularitySpec granularitySpec = new UniformGranularitySpec(Granularities.DAY, null, false,
|
GranularitySpec granularitySpec = new UniformGranularitySpec(Granularities.DAY, null, false,
|
||||||
Collections.singletonList(interval)
|
Collections.singletonList(interval)
|
||||||
);
|
);
|
||||||
DataSchema dataSchema =
|
DataSchema dataSchema = DataSchema.builder().withDataSource("test").withGranularity(granularitySpec).build();
|
||||||
new DataSchema("test", null, null, null, granularitySpec, null);
|
|
||||||
// no segments will be pushed when all rows are thrown away, assume that:
|
// no segments will be pushed when all rows are thrown away, assume that:
|
||||||
List<DataSegment> pushedSegments = Collections.emptyList();
|
List<DataSegment> pushedSegments = Collections.emptyList();
|
||||||
|
|
||||||
|
|
|
@ -122,17 +122,19 @@ public class InputRowSchemasTest extends InitializedNullHandlingTest
|
||||||
new DoubleDimensionSchema("d5")
|
new DoubleDimensionSchema("d5")
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
DataSchema schema = new DataSchema(
|
DataSchema schema =
|
||||||
"dataSourceName",
|
DataSchema.builder()
|
||||||
new TimestampSpec(null, null, null),
|
.withDataSource("dataSourceName")
|
||||||
dimensionsSpec,
|
.withTimestamp(new TimestampSpec(null, null, null))
|
||||||
new AggregatorFactory[]{
|
.withDimensions(dimensionsSpec)
|
||||||
new CountAggregatorFactory("count"),
|
.withAggregators(
|
||||||
new LongSumAggregatorFactory("met", "met")
|
new CountAggregatorFactory("count"),
|
||||||
},
|
new LongSumAggregatorFactory("met", "met")
|
||||||
new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null),
|
)
|
||||||
null
|
.withGranularity(
|
||||||
);
|
new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null)
|
||||||
|
)
|
||||||
|
.build();
|
||||||
|
|
||||||
InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(schema);
|
InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(schema);
|
||||||
Assert.assertEquals(timestampSpec, inputRowSchema.getTimestampSpec());
|
Assert.assertEquals(timestampSpec, inputRowSchema.getTimestampSpec());
|
||||||
|
@ -154,14 +156,12 @@ public class InputRowSchemasTest extends InitializedNullHandlingTest
|
||||||
new DoubleDimensionSchema("d5")
|
new DoubleDimensionSchema("d5")
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
DataSchema schema = new DataSchema(
|
DataSchema schema = DataSchema.builder()
|
||||||
"dataSourceName",
|
.withDataSource("dataSourceName")
|
||||||
new TimestampSpec(null, null, null),
|
.withTimestamp(new TimestampSpec(null, null, null))
|
||||||
dimensionsSpec,
|
.withDimensions(dimensionsSpec)
|
||||||
new AggregatorFactory[]{},
|
.withGranularity(new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null))
|
||||||
new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null),
|
.build();
|
||||||
null
|
|
||||||
);
|
|
||||||
|
|
||||||
InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(schema);
|
InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(schema);
|
||||||
Assert.assertEquals(timestampSpec, inputRowSchema.getTimestampSpec());
|
Assert.assertEquals(timestampSpec, inputRowSchema.getTimestampSpec());
|
||||||
|
|
|
@ -109,7 +109,6 @@ import org.apache.druid.query.DirectQueryProcessingPool;
|
||||||
import org.apache.druid.query.ForwardingQueryProcessingPool;
|
import org.apache.druid.query.ForwardingQueryProcessingPool;
|
||||||
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
|
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
|
||||||
import org.apache.druid.query.SegmentDescriptor;
|
import org.apache.druid.query.SegmentDescriptor;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||||
import org.apache.druid.segment.IndexIO;
|
import org.apache.druid.segment.IndexIO;
|
||||||
import org.apache.druid.segment.IndexMergerV9Factory;
|
import org.apache.druid.segment.IndexMergerV9Factory;
|
||||||
|
@ -672,18 +671,19 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
new IndexIngestionSpec(
|
new IndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"foo",
|
.withDataSource("foo")
|
||||||
new TimestampSpec(null, null, null),
|
.withTimestamp(new TimestampSpec(null, null, null))
|
||||||
DimensionsSpec.EMPTY,
|
.withDimensions(DimensionsSpec.EMPTY)
|
||||||
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
|
.withAggregators(new DoubleSumAggregatorFactory("met", "met"))
|
||||||
new UniformGranularitySpec(
|
.withGranularity(
|
||||||
Granularities.DAY,
|
new UniformGranularitySpec(
|
||||||
null,
|
Granularities.DAY,
|
||||||
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
|
null,
|
||||||
),
|
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
|
||||||
null
|
)
|
||||||
),
|
)
|
||||||
|
.build(),
|
||||||
new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false),
|
new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false),
|
||||||
TuningConfigBuilder.forIndexTask()
|
TuningConfigBuilder.forIndexTask()
|
||||||
.withMaxRowsPerSegment(10000)
|
.withMaxRowsPerSegment(10000)
|
||||||
|
@ -735,18 +735,18 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
new IndexIngestionSpec(
|
new IndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"foo",
|
.withDataSource("foo")
|
||||||
null,
|
.withAggregators(new DoubleSumAggregatorFactory("met", "met"))
|
||||||
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
|
.withGranularity(
|
||||||
new UniformGranularitySpec(
|
new UniformGranularitySpec(
|
||||||
Granularities.DAY,
|
Granularities.DAY,
|
||||||
null,
|
null,
|
||||||
ImmutableList.of(Intervals.of("2010-01-01/P1D"))
|
ImmutableList.of(Intervals.of("2010-01-01/P1D"))
|
||||||
),
|
)
|
||||||
null,
|
)
|
||||||
mapper
|
.withObjectMapper(mapper)
|
||||||
),
|
.build(),
|
||||||
new IndexIOConfig(new MockExceptionInputSource(), new NoopInputFormat(), false, false),
|
new IndexIOConfig(new MockExceptionInputSource(), new NoopInputFormat(), false, false),
|
||||||
TuningConfigBuilder.forIndexTask()
|
TuningConfigBuilder.forIndexTask()
|
||||||
.withMaxRowsPerSegment(10000)
|
.withMaxRowsPerSegment(10000)
|
||||||
|
@ -1165,18 +1165,19 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
new IndexIngestionSpec(
|
new IndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"foo",
|
.withDataSource("foo")
|
||||||
new TimestampSpec(null, null, null),
|
.withTimestamp(new TimestampSpec(null, null, null))
|
||||||
DimensionsSpec.EMPTY,
|
.withDimensions(DimensionsSpec.EMPTY)
|
||||||
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
|
.withAggregators(new DoubleSumAggregatorFactory("met", "met"))
|
||||||
new UniformGranularitySpec(
|
.withGranularity(
|
||||||
Granularities.DAY,
|
new UniformGranularitySpec(
|
||||||
null,
|
Granularities.DAY,
|
||||||
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
|
null,
|
||||||
),
|
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
|
||||||
null
|
)
|
||||||
),
|
)
|
||||||
|
.build(),
|
||||||
new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false),
|
new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false),
|
||||||
TuningConfigBuilder.forIndexTask()
|
TuningConfigBuilder.forIndexTask()
|
||||||
.withMaxRowsPerSegment(10000)
|
.withMaxRowsPerSegment(10000)
|
||||||
|
@ -1253,18 +1254,19 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
new IndexIngestionSpec(
|
new IndexIngestionSpec(
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"foo",
|
.withDataSource("foo")
|
||||||
new TimestampSpec(null, null, null),
|
.withTimestamp(new TimestampSpec(null, null, null))
|
||||||
DimensionsSpec.EMPTY,
|
.withDimensions(DimensionsSpec.EMPTY)
|
||||||
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
|
.withAggregators(new DoubleSumAggregatorFactory("met", "met"))
|
||||||
new UniformGranularitySpec(
|
.withGranularity(
|
||||||
Granularities.DAY,
|
new UniformGranularitySpec(
|
||||||
null,
|
Granularities.DAY,
|
||||||
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
|
null,
|
||||||
),
|
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
|
||||||
null
|
)
|
||||||
),
|
)
|
||||||
|
.build(),
|
||||||
new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false),
|
new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false),
|
||||||
TuningConfigBuilder.forIndexTask()
|
TuningConfigBuilder.forIndexTask()
|
||||||
.withMaxRowsPerSegment(10000)
|
.withMaxRowsPerSegment(10000)
|
||||||
|
|
|
@ -549,14 +549,15 @@ public class TaskQueueTest extends IngestionTestBase
|
||||||
new NoopTaskContextEnricher()
|
new NoopTaskContextEnricher()
|
||||||
);
|
);
|
||||||
|
|
||||||
final DataSchema dataSchema = new DataSchema(
|
final DataSchema dataSchema =
|
||||||
"DS",
|
DataSchema.builder()
|
||||||
new TimestampSpec(null, null, null),
|
.withDataSource("DS")
|
||||||
new DimensionsSpec(null),
|
.withTimestamp(new TimestampSpec(null, null, null))
|
||||||
null,
|
.withDimensions(DimensionsSpec.builder().build())
|
||||||
new UniformGranularitySpec(Granularities.YEAR, Granularities.DAY, null),
|
.withGranularity(
|
||||||
null
|
new UniformGranularitySpec(Granularities.YEAR, Granularities.DAY, null)
|
||||||
);
|
)
|
||||||
|
.build();
|
||||||
final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig(
|
final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig(
|
||||||
new HttpInputSource(Collections.singletonList(URI.create("http://host.org")),
|
new HttpInputSource(Collections.singletonList(URI.create("http://host.org")),
|
||||||
"user",
|
"user",
|
||||||
|
|
|
@ -47,14 +47,11 @@ public class CsvInputSourceSamplerTest extends InitializedNullHandlingTest
|
||||||
{
|
{
|
||||||
final TimestampSpec timestampSpec = new TimestampSpec(null, null, DateTimes.of("1970"));
|
final TimestampSpec timestampSpec = new TimestampSpec(null, null, DateTimes.of("1970"));
|
||||||
final DimensionsSpec dimensionsSpec = new DimensionsSpec(null);
|
final DimensionsSpec dimensionsSpec = new DimensionsSpec(null);
|
||||||
final DataSchema dataSchema = new DataSchema(
|
final DataSchema dataSchema = DataSchema.builder()
|
||||||
"sampler",
|
.withDataSource("sampler")
|
||||||
timestampSpec,
|
.withTimestamp(timestampSpec)
|
||||||
dimensionsSpec,
|
.withDimensions(dimensionsSpec)
|
||||||
null,
|
.build();
|
||||||
null,
|
|
||||||
null
|
|
||||||
);
|
|
||||||
|
|
||||||
final List<String> strCsvRows = ImmutableList.of(
|
final List<String> strCsvRows = ImmutableList.of(
|
||||||
"FirstName,LastName,Number,Gender",
|
"FirstName,LastName,Number,Gender",
|
||||||
|
|
|
@ -66,14 +66,11 @@ public class InputSourceSamplerDiscoveryTest extends InitializedNullHandlingTest
|
||||||
final SamplerResponse response = inputSourceSampler.sample(
|
final SamplerResponse response = inputSourceSampler.sample(
|
||||||
inputSource,
|
inputSource,
|
||||||
new JsonInputFormat(null, null, null, null, null),
|
new JsonInputFormat(null, null, null, null, null),
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"test",
|
.withDataSource("test")
|
||||||
new TimestampSpec("t", null, null),
|
.withTimestamp(new TimestampSpec("t", null, null))
|
||||||
DimensionsSpec.builder().useSchemaDiscovery(true).build(),
|
.withDimensions(DimensionsSpec.builder().useSchemaDiscovery(true).build())
|
||||||
null,
|
.build(),
|
||||||
null,
|
|
||||||
null
|
|
||||||
),
|
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -131,14 +128,11 @@ public class InputSourceSamplerDiscoveryTest extends InitializedNullHandlingTest
|
||||||
final SamplerResponse response = inputSourceSampler.sample(
|
final SamplerResponse response = inputSourceSampler.sample(
|
||||||
inputSource,
|
inputSource,
|
||||||
new JsonInputFormat(null, null, null, null, null),
|
new JsonInputFormat(null, null, null, null, null),
|
||||||
new DataSchema(
|
DataSchema.builder()
|
||||||
"test",
|
.withDataSource("test")
|
||||||
new TimestampSpec("t", null, null),
|
.withTimestamp(new TimestampSpec("t", null, null))
|
||||||
DimensionsSpec.builder().useSchemaDiscovery(true).build(),
|
.withDimensions(DimensionsSpec.builder().useSchemaDiscovery(true).build())
|
||||||
null,
|
.build(),
|
||||||
null,
|
|
||||||
null
|
|
||||||
),
|
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -189,14 +183,12 @@ public class InputSourceSamplerDiscoveryTest extends InitializedNullHandlingTest
|
||||||
public void testTypesClassicDiscovery()
|
public void testTypesClassicDiscovery()
|
||||||
{
|
{
|
||||||
final InputSource inputSource = new InlineInputSource(Strings.join(STR_JSON_ROWS, '\n'));
|
final InputSource inputSource = new InlineInputSource(Strings.join(STR_JSON_ROWS, '\n'));
|
||||||
final DataSchema dataSchema = new DataSchema(
|
final DataSchema dataSchema =
|
||||||
"test",
|
DataSchema.builder()
|
||||||
new TimestampSpec("t", null, null),
|
.withDataSource("test")
|
||||||
DimensionsSpec.builder().build(),
|
.withTimestamp(new TimestampSpec("t", null, null))
|
||||||
null,
|
.withDimensions(DimensionsSpec.builder().build())
|
||||||
null,
|
.build();
|
||||||
null
|
|
||||||
);
|
|
||||||
final SamplerResponse response = inputSourceSampler.sample(
|
final SamplerResponse response = inputSourceSampler.sample(
|
||||||
inputSource,
|
inputSource,
|
||||||
new JsonInputFormat(null, null, null, null, null),
|
new JsonInputFormat(null, null, null, null, null),
|
||||||
|
@ -248,23 +240,20 @@ public class InputSourceSamplerDiscoveryTest extends InitializedNullHandlingTest
|
||||||
public void testTypesNoDiscoveryExplicitSchema()
|
public void testTypesNoDiscoveryExplicitSchema()
|
||||||
{
|
{
|
||||||
final InputSource inputSource = new InlineInputSource(Strings.join(STR_JSON_ROWS, '\n'));
|
final InputSource inputSource = new InlineInputSource(Strings.join(STR_JSON_ROWS, '\n'));
|
||||||
final DataSchema dataSchema = new DataSchema(
|
final DataSchema dataSchema =
|
||||||
"test",
|
DataSchema.builder()
|
||||||
new TimestampSpec("t", null, null),
|
.withDataSource("test")
|
||||||
DimensionsSpec.builder().setDimensions(
|
.withTimestamp(new TimestampSpec("t", null, null))
|
||||||
ImmutableList.of(new StringDimensionSchema("string"),
|
.withDimensions(
|
||||||
new LongDimensionSchema("long"),
|
new StringDimensionSchema("string"),
|
||||||
new DoubleDimensionSchema("double"),
|
new LongDimensionSchema("long"),
|
||||||
new StringDimensionSchema("bool"),
|
new DoubleDimensionSchema("double"),
|
||||||
new AutoTypeColumnSchema("variant", null),
|
new StringDimensionSchema("bool"),
|
||||||
new AutoTypeColumnSchema("array", null),
|
new AutoTypeColumnSchema("variant", null),
|
||||||
new AutoTypeColumnSchema("nested", null)
|
new AutoTypeColumnSchema("array", null),
|
||||||
)
|
new AutoTypeColumnSchema("nested", null)
|
||||||
).build(),
|
)
|
||||||
null,
|
.build();
|
||||||
null,
|
|
||||||
null
|
|
||||||
);
|
|
||||||
final SamplerResponse response = inputSourceSampler.sample(
|
final SamplerResponse response = inputSourceSampler.sample(
|
||||||
inputSource,
|
inputSource,
|
||||||
new JsonInputFormat(null, null, null, null, null),
|
new JsonInputFormat(null, null, null, null, null),
|
||||||
|
|
|
@ -1497,24 +1497,24 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
if (useInputFormatApi) {
|
if (useInputFormatApi) {
|
||||||
return new DataSchema(
|
return DataSchema.builder()
|
||||||
"sampler",
|
.withDataSource("sampler")
|
||||||
timestampSpec,
|
.withTimestamp(timestampSpec)
|
||||||
dimensionsSpec,
|
.withDimensions(dimensionsSpec)
|
||||||
aggregators,
|
.withAggregators(aggregators)
|
||||||
granularitySpec,
|
.withGranularity(granularitySpec)
|
||||||
transformSpec
|
.withTransform(transformSpec)
|
||||||
);
|
.build();
|
||||||
} else {
|
} else {
|
||||||
final Map<String, Object> parserMap = getParserMap(createInputRowParser(timestampSpec, dimensionsSpec));
|
final Map<String, Object> parserMap = getParserMap(createInputRowParser(timestampSpec, dimensionsSpec));
|
||||||
return new DataSchema(
|
return DataSchema.builder()
|
||||||
"sampler",
|
.withDataSource("sampler")
|
||||||
parserMap,
|
.withParserMap(parserMap)
|
||||||
aggregators,
|
.withAggregators(aggregators)
|
||||||
granularitySpec,
|
.withGranularity(granularitySpec)
|
||||||
transformSpec,
|
.withTransform(transformSpec)
|
||||||
OBJECT_MAPPER
|
.withObjectMapper(OBJECT_MAPPER)
|
||||||
);
|
.build();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -33,10 +33,8 @@ import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
|
||||||
import org.apache.druid.indexing.seekablestream.common.StreamPartition;
|
import org.apache.druid.indexing.seekablestream.common.StreamPartition;
|
||||||
import org.apache.druid.java.util.common.DateTimes;
|
import org.apache.druid.java.util.common.DateTimes;
|
||||||
import org.apache.druid.java.util.common.granularity.AllGranularity;
|
import org.apache.druid.java.util.common.granularity.AllGranularity;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
|
import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
|
||||||
import org.apache.druid.segment.transform.TransformSpec;
|
|
||||||
import org.apache.druid.server.security.Access;
|
import org.apache.druid.server.security.Access;
|
||||||
import org.apache.druid.server.security.Action;
|
import org.apache.druid.server.security.Action;
|
||||||
import org.apache.druid.server.security.AuthConfig;
|
import org.apache.druid.server.security.AuthConfig;
|
||||||
|
@ -107,16 +105,13 @@ public class SeekableStreamIndexTaskRunnerAuthTest
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
DataSchema dataSchema = new DataSchema(
|
DataSchema dataSchema =
|
||||||
"datasource",
|
DataSchema.builder()
|
||||||
new TimestampSpec(null, null, null),
|
.withDataSource("datasource")
|
||||||
new DimensionsSpec(Collections.emptyList()),
|
.withTimestamp(new TimestampSpec(null, null, null))
|
||||||
new AggregatorFactory[]{},
|
.withDimensions(new DimensionsSpec(Collections.emptyList()))
|
||||||
new ArbitraryGranularitySpec(new AllGranularity(), Collections.emptyList()),
|
.withGranularity(new ArbitraryGranularitySpec(new AllGranularity(), Collections.emptyList()))
|
||||||
TransformSpec.NONE,
|
.build();
|
||||||
null,
|
|
||||||
null
|
|
||||||
);
|
|
||||||
SeekableStreamIndexTaskTuningConfig tuningConfig = mock(SeekableStreamIndexTaskTuningConfig.class);
|
SeekableStreamIndexTaskTuningConfig tuningConfig = mock(SeekableStreamIndexTaskTuningConfig.class);
|
||||||
SeekableStreamIndexTaskIOConfig<String, String> ioConfig = new TestSeekableStreamIndexTaskIOConfig();
|
SeekableStreamIndexTaskIOConfig<String, String> ioConfig = new TestSeekableStreamIndexTaskIOConfig();
|
||||||
|
|
||||||
|
|
|
@ -95,7 +95,6 @@ import org.apache.druid.query.QueryPlus;
|
||||||
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
|
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
|
||||||
import org.apache.druid.query.Result;
|
import org.apache.druid.query.Result;
|
||||||
import org.apache.druid.query.SegmentDescriptor;
|
import org.apache.druid.query.SegmentDescriptor;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||||
|
@ -164,25 +163,23 @@ public abstract class SeekableStreamIndexTaskTestBase extends EasyMockSupport
|
||||||
|
|
||||||
protected static final ObjectMapper OBJECT_MAPPER;
|
protected static final ObjectMapper OBJECT_MAPPER;
|
||||||
protected static final DataSchema OLD_DATA_SCHEMA;
|
protected static final DataSchema OLD_DATA_SCHEMA;
|
||||||
protected static final DataSchema NEW_DATA_SCHEMA = new DataSchema(
|
protected static final DataSchema NEW_DATA_SCHEMA =
|
||||||
"test_ds",
|
DataSchema.builder()
|
||||||
new TimestampSpec("timestamp", "iso", null),
|
.withDataSource("test_ds")
|
||||||
new DimensionsSpec(
|
.withTimestamp(new TimestampSpec("timestamp", "iso", null))
|
||||||
Arrays.asList(
|
.withDimensions(
|
||||||
new StringDimensionSchema("dim1"),
|
new StringDimensionSchema("dim1"),
|
||||||
new StringDimensionSchema("dim1t"),
|
new StringDimensionSchema("dim1t"),
|
||||||
new StringDimensionSchema("dim2"),
|
new StringDimensionSchema("dim2"),
|
||||||
new LongDimensionSchema("dimLong"),
|
new LongDimensionSchema("dimLong"),
|
||||||
new FloatDimensionSchema("dimFloat")
|
new FloatDimensionSchema("dimFloat")
|
||||||
)
|
)
|
||||||
),
|
.withAggregators(
|
||||||
new AggregatorFactory[]{
|
new DoubleSumAggregatorFactory("met1sum", "met1"),
|
||||||
new DoubleSumAggregatorFactory("met1sum", "met1"),
|
new CountAggregatorFactory("rows")
|
||||||
new CountAggregatorFactory("rows")
|
)
|
||||||
},
|
.withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null))
|
||||||
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
|
.build();
|
||||||
null
|
|
||||||
);
|
|
||||||
protected static final InputFormat INPUT_FORMAT = new JsonInputFormat(
|
protected static final InputFormat INPUT_FORMAT = new JsonInputFormat(
|
||||||
new JSONPathSpec(true, ImmutableList.of()),
|
new JSONPathSpec(true, ImmutableList.of()),
|
||||||
ImmutableMap.of(),
|
ImmutableMap.of(),
|
||||||
|
@ -211,37 +208,38 @@ public abstract class SeekableStreamIndexTaskTestBase extends EasyMockSupport
|
||||||
static {
|
static {
|
||||||
OBJECT_MAPPER = new TestUtils().getTestObjectMapper();
|
OBJECT_MAPPER = new TestUtils().getTestObjectMapper();
|
||||||
OBJECT_MAPPER.registerSubtypes(new NamedType(JSONParseSpec.class, "json"));
|
OBJECT_MAPPER.registerSubtypes(new NamedType(JSONParseSpec.class, "json"));
|
||||||
OLD_DATA_SCHEMA = new DataSchema(
|
OLD_DATA_SCHEMA = DataSchema.builder()
|
||||||
"test_ds",
|
.withDataSource("test_ds")
|
||||||
OBJECT_MAPPER.convertValue(
|
.withParserMap(
|
||||||
new StringInputRowParser(
|
OBJECT_MAPPER.convertValue(
|
||||||
new JSONParseSpec(
|
new StringInputRowParser(
|
||||||
new TimestampSpec("timestamp", "iso", null),
|
new JSONParseSpec(
|
||||||
new DimensionsSpec(
|
new TimestampSpec("timestamp", "iso", null),
|
||||||
Arrays.asList(
|
new DimensionsSpec(
|
||||||
new StringDimensionSchema("dim1"),
|
Arrays.asList(
|
||||||
new StringDimensionSchema("dim1t"),
|
new StringDimensionSchema("dim1"),
|
||||||
new StringDimensionSchema("dim2"),
|
new StringDimensionSchema("dim1t"),
|
||||||
new LongDimensionSchema("dimLong"),
|
new StringDimensionSchema("dim2"),
|
||||||
new FloatDimensionSchema("dimFloat")
|
new LongDimensionSchema("dimLong"),
|
||||||
)
|
new FloatDimensionSchema("dimFloat")
|
||||||
),
|
)
|
||||||
new JSONPathSpec(true, ImmutableList.of()),
|
),
|
||||||
ImmutableMap.of(),
|
new JSONPathSpec(true, ImmutableList.of()),
|
||||||
false
|
ImmutableMap.of(),
|
||||||
),
|
false
|
||||||
StandardCharsets.UTF_8.name()
|
),
|
||||||
),
|
StandardCharsets.UTF_8.name()
|
||||||
Map.class
|
),
|
||||||
),
|
Map.class
|
||||||
new AggregatorFactory[]{
|
)
|
||||||
new DoubleSumAggregatorFactory("met1sum", "met1"),
|
)
|
||||||
new CountAggregatorFactory("rows")
|
.withAggregators(
|
||||||
},
|
new DoubleSumAggregatorFactory("met1sum", "met1"),
|
||||||
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
|
new CountAggregatorFactory("rows")
|
||||||
null,
|
)
|
||||||
OBJECT_MAPPER
|
.withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null))
|
||||||
);
|
.withObjectMapper(OBJECT_MAPPER)
|
||||||
|
.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
public SeekableStreamIndexTaskTestBase(
|
public SeekableStreamIndexTaskTestBase(
|
||||||
|
|
|
@ -48,7 +48,6 @@ import org.apache.druid.jackson.DefaultObjectMapper;
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.java.util.common.parsers.JSONPathSpec;
|
import org.apache.druid.java.util.common.parsers.JSONPathSpec;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
|
@ -106,36 +105,37 @@ public class SeekableStreamSamplerSpecTest extends EasyMockSupport
|
||||||
@Test(timeout = 10_000L)
|
@Test(timeout = 10_000L)
|
||||||
public void testSampleWithInputRowParser() throws Exception
|
public void testSampleWithInputRowParser() throws Exception
|
||||||
{
|
{
|
||||||
final DataSchema dataSchema = new DataSchema(
|
DataSchema dataSchema = DataSchema.builder()
|
||||||
"test_ds",
|
.withDataSource("test_ds")
|
||||||
OBJECT_MAPPER.convertValue(
|
.withParserMap(
|
||||||
new StringInputRowParser(
|
OBJECT_MAPPER.convertValue(
|
||||||
new JSONParseSpec(
|
new StringInputRowParser(
|
||||||
new TimestampSpec("timestamp", "iso", null),
|
new JSONParseSpec(
|
||||||
new DimensionsSpec(
|
new TimestampSpec("timestamp", "iso", null),
|
||||||
Arrays.asList(
|
new DimensionsSpec(
|
||||||
new StringDimensionSchema("dim1"),
|
Arrays.asList(
|
||||||
new StringDimensionSchema("dim1t"),
|
new StringDimensionSchema("dim1"),
|
||||||
new StringDimensionSchema("dim2"),
|
new StringDimensionSchema("dim1t"),
|
||||||
new LongDimensionSchema("dimLong"),
|
new StringDimensionSchema("dim2"),
|
||||||
new FloatDimensionSchema("dimFloat")
|
new LongDimensionSchema("dimLong"),
|
||||||
)
|
new FloatDimensionSchema("dimFloat")
|
||||||
),
|
)
|
||||||
new JSONPathSpec(true, ImmutableList.of()),
|
),
|
||||||
ImmutableMap.of(),
|
new JSONPathSpec(true, ImmutableList.of()),
|
||||||
false
|
ImmutableMap.of(),
|
||||||
)
|
false
|
||||||
),
|
)
|
||||||
Map.class
|
),
|
||||||
),
|
Map.class
|
||||||
new AggregatorFactory[]{
|
)
|
||||||
new DoubleSumAggregatorFactory("met1sum", "met1"),
|
)
|
||||||
new CountAggregatorFactory("rows")
|
.withAggregators(
|
||||||
},
|
new DoubleSumAggregatorFactory("met1sum", "met1"),
|
||||||
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
|
new CountAggregatorFactory("rows")
|
||||||
null,
|
)
|
||||||
OBJECT_MAPPER
|
.withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null))
|
||||||
);
|
.withObjectMapper(OBJECT_MAPPER)
|
||||||
|
.build();
|
||||||
|
|
||||||
final SeekableStreamSupervisorIOConfig supervisorIOConfig = new TestableSeekableStreamSupervisorIOConfig(
|
final SeekableStreamSupervisorIOConfig supervisorIOConfig = new TestableSeekableStreamSupervisorIOConfig(
|
||||||
STREAM,
|
STREAM,
|
||||||
|
|
|
@ -25,7 +25,6 @@ import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import org.apache.druid.data.input.impl.ByteEntity;
|
import org.apache.druid.data.input.impl.ByteEntity;
|
||||||
import org.apache.druid.data.input.impl.DimensionSchema;
|
import org.apache.druid.data.input.impl.DimensionSchema;
|
||||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
|
||||||
import org.apache.druid.data.input.impl.JsonInputFormat;
|
import org.apache.druid.data.input.impl.JsonInputFormat;
|
||||||
import org.apache.druid.data.input.impl.StringDimensionSchema;
|
import org.apache.druid.data.input.impl.StringDimensionSchema;
|
||||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||||
|
@ -59,7 +58,6 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter;
|
||||||
import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig;
|
import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig;
|
||||||
import org.apache.druid.java.util.metrics.StubServiceEmitter;
|
import org.apache.druid.java.util.metrics.StubServiceEmitter;
|
||||||
import org.apache.druid.query.DruidMetrics;
|
import org.apache.druid.query.DruidMetrics;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import org.apache.druid.segment.TestHelper;
|
import org.apache.druid.segment.TestHelper;
|
||||||
import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
|
import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
|
||||||
|
@ -1260,18 +1258,19 @@ public class SeekableStreamSupervisorSpecTest extends EasyMockSupport
|
||||||
dimensions.add(StringDimensionSchema.create("dim1"));
|
dimensions.add(StringDimensionSchema.create("dim1"));
|
||||||
dimensions.add(StringDimensionSchema.create("dim2"));
|
dimensions.add(StringDimensionSchema.create("dim2"));
|
||||||
|
|
||||||
return new DataSchema(
|
return DataSchema.builder()
|
||||||
DATASOURCE,
|
.withDataSource(DATASOURCE)
|
||||||
new TimestampSpec("timestamp", "iso", null),
|
.withTimestamp(new TimestampSpec("timestamp", "iso", null))
|
||||||
new DimensionsSpec(dimensions),
|
.withDimensions(dimensions)
|
||||||
new AggregatorFactory[]{new CountAggregatorFactory("rows")},
|
.withAggregators(new CountAggregatorFactory("rows"))
|
||||||
new UniformGranularitySpec(
|
.withGranularity(
|
||||||
Granularities.HOUR,
|
new UniformGranularitySpec(
|
||||||
Granularities.NONE,
|
Granularities.HOUR,
|
||||||
ImmutableList.of()
|
Granularities.NONE,
|
||||||
),
|
ImmutableList.of()
|
||||||
null
|
)
|
||||||
);
|
)
|
||||||
|
.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
private SeekableStreamSupervisorIOConfig getIOConfig(int taskCount, boolean scaleOut)
|
private SeekableStreamSupervisorIOConfig getIOConfig(int taskCount, boolean scaleOut)
|
||||||
|
|
|
@ -31,7 +31,6 @@ import com.google.common.util.concurrent.Futures;
|
||||||
import com.google.common.util.concurrent.ListenableFuture;
|
import com.google.common.util.concurrent.ListenableFuture;
|
||||||
import org.apache.druid.data.input.impl.ByteEntity;
|
import org.apache.druid.data.input.impl.ByteEntity;
|
||||||
import org.apache.druid.data.input.impl.DimensionSchema;
|
import org.apache.druid.data.input.impl.DimensionSchema;
|
||||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
|
||||||
import org.apache.druid.data.input.impl.JsonInputFormat;
|
import org.apache.druid.data.input.impl.JsonInputFormat;
|
||||||
import org.apache.druid.data.input.impl.StringDimensionSchema;
|
import org.apache.druid.data.input.impl.StringDimensionSchema;
|
||||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||||
|
@ -83,7 +82,6 @@ import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig;
|
||||||
import org.apache.druid.java.util.metrics.StubServiceEmitter;
|
import org.apache.druid.java.util.metrics.StubServiceEmitter;
|
||||||
import org.apache.druid.metadata.PendingSegmentRecord;
|
import org.apache.druid.metadata.PendingSegmentRecord;
|
||||||
import org.apache.druid.query.DruidMetrics;
|
import org.apache.druid.query.DruidMetrics;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import org.apache.druid.segment.TestHelper;
|
import org.apache.druid.segment.TestHelper;
|
||||||
import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
|
import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
|
||||||
|
@ -2562,18 +2560,19 @@ public class SeekableStreamSupervisorStateTest extends EasyMockSupport
|
||||||
dimensions.add(StringDimensionSchema.create("dim1"));
|
dimensions.add(StringDimensionSchema.create("dim1"));
|
||||||
dimensions.add(StringDimensionSchema.create("dim2"));
|
dimensions.add(StringDimensionSchema.create("dim2"));
|
||||||
|
|
||||||
return new DataSchema(
|
return DataSchema.builder()
|
||||||
DATASOURCE,
|
.withDataSource(DATASOURCE)
|
||||||
new TimestampSpec("timestamp", "iso", null),
|
.withTimestamp(new TimestampSpec("timestamp", "iso", null))
|
||||||
new DimensionsSpec(dimensions),
|
.withDimensions(dimensions)
|
||||||
new AggregatorFactory[]{new CountAggregatorFactory("rows")},
|
.withAggregators(new CountAggregatorFactory("rows"))
|
||||||
new UniformGranularitySpec(
|
.withGranularity(
|
||||||
Granularities.HOUR,
|
new UniformGranularitySpec(
|
||||||
Granularities.NONE,
|
Granularities.HOUR,
|
||||||
ImmutableList.of()
|
Granularities.NONE,
|
||||||
),
|
ImmutableList.of()
|
||||||
null
|
)
|
||||||
);
|
)
|
||||||
|
.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
private static SeekableStreamSupervisorIOConfig getIOConfig()
|
private static SeekableStreamSupervisorIOConfig getIOConfig()
|
||||||
|
|
|
@ -28,7 +28,6 @@ import org.apache.druid.indexing.common.task.IndexTask;
|
||||||
import org.apache.druid.indexing.common.task.Task;
|
import org.apache.druid.indexing.common.task.Task;
|
||||||
import org.apache.druid.indexing.common.task.TaskResource;
|
import org.apache.druid.indexing.common.task.TaskResource;
|
||||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
@ -51,7 +50,7 @@ public class TaskAnnouncementTest
|
||||||
"theid",
|
"theid",
|
||||||
new TaskResource("rofl", 2),
|
new TaskResource("rofl", 2),
|
||||||
new IndexTask.IndexIngestionSpec(
|
new IndexTask.IndexIngestionSpec(
|
||||||
new DataSchema("foo", null, new AggregatorFactory[0], null, null, new DefaultObjectMapper()),
|
DataSchema.builder().withDataSource("foo").withObjectMapper(new DefaultObjectMapper()).build(),
|
||||||
ioConfig,
|
ioConfig,
|
||||||
null
|
null
|
||||||
),
|
),
|
||||||
|
|
|
@ -0,0 +1,212 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF 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 org.apache.druid.data.input.impl;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
import org.apache.druid.error.InvalidInput;
|
||||||
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
|
import org.apache.druid.java.util.common.granularity.Granularity;
|
||||||
|
import org.apache.druid.query.OrderBy;
|
||||||
|
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||||
|
import org.apache.druid.segment.AggregateProjectionMetadata;
|
||||||
|
import org.apache.druid.segment.VirtualColumn;
|
||||||
|
import org.apache.druid.segment.VirtualColumns;
|
||||||
|
import org.apache.druid.segment.column.ColumnHolder;
|
||||||
|
import org.apache.druid.utils.CollectionUtils;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* API type to specify an aggregating projection on {@link org.apache.druid.segment.incremental.IncrementalIndexSchema}
|
||||||
|
*
|
||||||
|
* Decorated with {@link JsonTypeInfo} annotations as a future-proofing mechanism in the event we add other types of
|
||||||
|
* projections and need to extract out a base interface from this class.
|
||||||
|
*/
|
||||||
|
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
||||||
|
@JsonTypeName(AggregateProjectionSpec.TYPE_NAME)
|
||||||
|
public class AggregateProjectionSpec
|
||||||
|
{
|
||||||
|
public static final String TYPE_NAME = "aggregate";
|
||||||
|
|
||||||
|
private final String name;
|
||||||
|
private final List<DimensionSchema> groupingColumns;
|
||||||
|
private final VirtualColumns virtualColumns;
|
||||||
|
private final AggregatorFactory[] aggregators;
|
||||||
|
private final List<OrderBy> ordering;
|
||||||
|
@Nullable
|
||||||
|
private final String timeColumnName;
|
||||||
|
|
||||||
|
@JsonCreator
|
||||||
|
public AggregateProjectionSpec(
|
||||||
|
@JsonProperty("name") String name,
|
||||||
|
@JsonProperty("virtualColumns") @Nullable VirtualColumns virtualColumns,
|
||||||
|
@JsonProperty("groupingColumns") @Nullable List<DimensionSchema> groupingColumns,
|
||||||
|
@JsonProperty("aggregators") @Nullable AggregatorFactory[] aggregators
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.name = name;
|
||||||
|
if (CollectionUtils.isNullOrEmpty(groupingColumns)) {
|
||||||
|
throw InvalidInput.exception("groupingColumns must not be null or empty");
|
||||||
|
}
|
||||||
|
this.groupingColumns = groupingColumns;
|
||||||
|
this.virtualColumns = virtualColumns == null ? VirtualColumns.EMPTY : virtualColumns;
|
||||||
|
// in the future this should be expanded to support user specified ordering, but for now we compute it based on
|
||||||
|
// the grouping columns, which is consistent with how rollup ordering works for incremental index base table
|
||||||
|
final ProjectionOrdering ordering = computeOrdering(this.virtualColumns, this.groupingColumns);
|
||||||
|
this.ordering = ordering.ordering;
|
||||||
|
this.timeColumnName = ordering.timeColumnName;
|
||||||
|
this.aggregators = aggregators == null ? new AggregatorFactory[0] : aggregators;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public String getName()
|
||||||
|
{
|
||||||
|
return name;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
|
||||||
|
public VirtualColumns getVirtualColumns()
|
||||||
|
{
|
||||||
|
return virtualColumns;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
|
||||||
|
public List<DimensionSchema> getGroupingColumns()
|
||||||
|
{
|
||||||
|
return groupingColumns;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
|
||||||
|
public AggregatorFactory[] getAggregators()
|
||||||
|
{
|
||||||
|
return aggregators;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public List<OrderBy> getOrdering()
|
||||||
|
{
|
||||||
|
return ordering;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonIgnore
|
||||||
|
public AggregateProjectionMetadata.Schema toMetadataSchema()
|
||||||
|
{
|
||||||
|
return new AggregateProjectionMetadata.Schema(
|
||||||
|
name,
|
||||||
|
timeColumnName,
|
||||||
|
virtualColumns,
|
||||||
|
groupingColumns.stream().map(DimensionSchema::getName).collect(Collectors.toList()),
|
||||||
|
aggregators,
|
||||||
|
ordering
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o)
|
||||||
|
{
|
||||||
|
if (this == o) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
if (o == null || getClass() != o.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
AggregateProjectionSpec that = (AggregateProjectionSpec) o;
|
||||||
|
return Objects.equals(name, that.name)
|
||||||
|
&& Objects.equals(groupingColumns, that.groupingColumns)
|
||||||
|
&& Objects.equals(virtualColumns, that.virtualColumns)
|
||||||
|
&& Objects.deepEquals(aggregators, that.aggregators)
|
||||||
|
&& Objects.equals(ordering, that.ordering);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
return Objects.hash(name, groupingColumns, virtualColumns, Arrays.hashCode(aggregators), ordering);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return "AggregateProjectionSpec{" +
|
||||||
|
"name='" + name + '\'' +
|
||||||
|
", groupingColumns=" + groupingColumns +
|
||||||
|
", virtualColumns=" + virtualColumns +
|
||||||
|
", aggregators=" + Arrays.toString(aggregators) +
|
||||||
|
", ordering=" + ordering +
|
||||||
|
'}';
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
private static ProjectionOrdering computeOrdering(VirtualColumns virtualColumns, List<DimensionSchema> groupingColumns)
|
||||||
|
{
|
||||||
|
final List<OrderBy> ordering = Lists.newArrayListWithCapacity(groupingColumns.size());
|
||||||
|
|
||||||
|
String timeColumnName = null;
|
||||||
|
Granularity granularity = null;
|
||||||
|
// try to find the __time column equivalent, which might be a time_floor expression to model granularity
|
||||||
|
// bucketing. The time column is decided as the finest granularity on __time detected. If the projection does
|
||||||
|
// not have a time-like column, the granularity will be handled as ALL for the projection and all projection
|
||||||
|
// rows will use a synthetic timestamp of the minimum timestamp of the incremental index
|
||||||
|
for (final DimensionSchema dimension : groupingColumns) {
|
||||||
|
ordering.add(OrderBy.ascending(dimension.getName()));
|
||||||
|
if (ColumnHolder.TIME_COLUMN_NAME.equals(dimension.getName())) {
|
||||||
|
timeColumnName = dimension.getName();
|
||||||
|
granularity = Granularities.NONE;
|
||||||
|
} else {
|
||||||
|
final VirtualColumn vc = virtualColumns.getVirtualColumn(dimension.getName());
|
||||||
|
final Granularity maybeGranularity = Granularities.fromVirtualColumn(vc);
|
||||||
|
if (granularity == null && maybeGranularity != null) {
|
||||||
|
granularity = maybeGranularity;
|
||||||
|
timeColumnName = dimension.getName();
|
||||||
|
} else if (granularity != null && maybeGranularity != null && maybeGranularity.isFinerThan(granularity)) {
|
||||||
|
granularity = maybeGranularity;
|
||||||
|
timeColumnName = dimension.getName();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return new ProjectionOrdering(ordering, timeColumnName);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static final class ProjectionOrdering
|
||||||
|
{
|
||||||
|
private final List<OrderBy> ordering;
|
||||||
|
@Nullable
|
||||||
|
private final String timeColumnName;
|
||||||
|
|
||||||
|
private ProjectionOrdering(List<OrderBy> ordering, @Nullable String timeColumnName)
|
||||||
|
{
|
||||||
|
this.ordering = ordering;
|
||||||
|
this.timeColumnName = timeColumnName;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -23,9 +23,11 @@ import com.google.common.base.Preconditions;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
|
import org.apache.druid.math.expr.Expr;
|
||||||
import org.apache.druid.math.expr.ExprMacroTable;
|
import org.apache.druid.math.expr.ExprMacroTable;
|
||||||
import org.apache.druid.query.Query;
|
import org.apache.druid.query.Query;
|
||||||
import org.apache.druid.query.expression.TimestampFloorExprMacro;
|
import org.apache.druid.query.expression.TimestampFloorExprMacro;
|
||||||
|
import org.apache.druid.segment.AggregateProjectionMetadata;
|
||||||
import org.apache.druid.segment.CursorBuildSpec;
|
import org.apache.druid.segment.CursorBuildSpec;
|
||||||
import org.apache.druid.segment.VirtualColumn;
|
import org.apache.druid.segment.VirtualColumn;
|
||||||
import org.apache.druid.segment.VirtualColumns;
|
import org.apache.druid.segment.VirtualColumns;
|
||||||
|
@ -160,4 +162,32 @@ public class Granularities
|
||||||
ExprMacroTable.granularity()
|
ExprMacroTable.granularity()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Converts a virtual column with a single input time column into a {@link Granularity} if it is a
|
||||||
|
* {@link TimestampFloorExprMacro.TimestampFloorExpr}.
|
||||||
|
* <p>
|
||||||
|
* IMPORTANT - this method DOES NOT VERIFY that the virtual column has a single input that is a time column
|
||||||
|
* ({@link ColumnHolder#TIME_COLUMN_NAME} or equivalent projection time column as defined by
|
||||||
|
* {@link AggregateProjectionMetadata.Schema#getTimeColumnName()}). Callers must verify this externally before
|
||||||
|
* calling this method by examining {@link VirtualColumn#requiredColumns()}.
|
||||||
|
* <p>
|
||||||
|
* This method also does not handle other time expressions, or if the virtual column is just an identifier for a
|
||||||
|
* time column
|
||||||
|
*/
|
||||||
|
@Nullable
|
||||||
|
public static Granularity fromVirtualColumn(VirtualColumn virtualColumn)
|
||||||
|
{
|
||||||
|
if (virtualColumn instanceof ExpressionVirtualColumn) {
|
||||||
|
final ExpressionVirtualColumn expressionVirtualColumn = (ExpressionVirtualColumn) virtualColumn;
|
||||||
|
final Expr expr = expressionVirtualColumn.getParsedExpression().get();
|
||||||
|
if (expr instanceof TimestampFloorExprMacro.TimestampFloorExpr) {
|
||||||
|
final TimestampFloorExprMacro.TimestampFloorExpr gran = (TimestampFloorExprMacro.TimestampFloorExpr) expr;
|
||||||
|
if (gran.getArg().getBindingIfIdentifier() != null) {
|
||||||
|
return gran.getGranularity();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -190,6 +190,12 @@ public class DefaultQueryMetrics<QueryType extends Query<?>> implements QueryMet
|
||||||
setDimension("segment", segmentIdentifier);
|
setDimension("segment", segmentIdentifier);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void projection(String projection)
|
||||||
|
{
|
||||||
|
setDimension("projection", projection);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void identity(String identity)
|
public void identity(String identity)
|
||||||
{
|
{
|
||||||
|
|
|
@ -88,6 +88,12 @@ public class QueryContexts
|
||||||
public static final String UNCOVERED_INTERVALS_LIMIT_KEY = "uncoveredIntervalsLimit";
|
public static final String UNCOVERED_INTERVALS_LIMIT_KEY = "uncoveredIntervalsLimit";
|
||||||
public static final String MIN_TOP_N_THRESHOLD = "minTopNThreshold";
|
public static final String MIN_TOP_N_THRESHOLD = "minTopNThreshold";
|
||||||
public static final String CATALOG_VALIDATION_ENABLED = "catalogValidationEnabled";
|
public static final String CATALOG_VALIDATION_ENABLED = "catalogValidationEnabled";
|
||||||
|
|
||||||
|
// projection context keys
|
||||||
|
public static final String NO_PROJECTIONS = "noProjections";
|
||||||
|
public static final String FORCE_PROJECTION = "forceProjections";
|
||||||
|
public static final String USE_PROJECTION = "useProjection";
|
||||||
|
|
||||||
// Unique identifier for the query, that is used to map the global shared resources (specifically merge buffers) to the
|
// Unique identifier for the query, that is used to map the global shared resources (specifically merge buffers) to the
|
||||||
// query's runtime
|
// query's runtime
|
||||||
public static final String QUERY_RESOURCE_ID = "queryResourceId";
|
public static final String QUERY_RESOURCE_ID = "queryResourceId";
|
||||||
|
|
|
@ -243,6 +243,11 @@ public interface QueryMetrics<QueryType extends Query<?>>
|
||||||
|
|
||||||
void segment(String segmentIdentifier);
|
void segment(String segmentIdentifier);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* If a projection was used during segment processing, set its name as the projection dimension
|
||||||
|
*/
|
||||||
|
void projection(String projection);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @deprecated use {@link #filterBundle(FilterBundle.BundleInfo)} instead to collect details about filters which were
|
* @deprecated use {@link #filterBundle(FilterBundle.BundleInfo)} instead to collect details about filters which were
|
||||||
* used to construct {@link org.apache.druid.segment.BitmapOffset} or
|
* used to construct {@link org.apache.druid.segment.BitmapOffset} or
|
||||||
|
|
|
@ -26,6 +26,8 @@ import org.apache.druid.java.util.common.logger.Logger;
|
||||||
import org.apache.druid.query.PerSegmentQueryOptimizationContext;
|
import org.apache.druid.query.PerSegmentQueryOptimizationContext;
|
||||||
import org.apache.druid.segment.ColumnInspector;
|
import org.apache.druid.segment.ColumnInspector;
|
||||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||||
|
import org.apache.druid.segment.CursorBuildSpec;
|
||||||
|
import org.apache.druid.segment.CursorHolder;
|
||||||
import org.apache.druid.segment.column.ColumnType;
|
import org.apache.druid.segment.column.ColumnType;
|
||||||
import org.apache.druid.segment.column.ColumnTypeFactory;
|
import org.apache.druid.segment.column.ColumnTypeFactory;
|
||||||
import org.apache.druid.segment.column.ValueType;
|
import org.apache.druid.segment.column.ValueType;
|
||||||
|
@ -365,6 +367,38 @@ public abstract class AggregatorFactory implements Cacheable
|
||||||
throw new UOE("Cannot change output name for AggregatorFactory[%s].", this.getClass().getName());
|
throw new UOE("Cannot change output name for AggregatorFactory[%s].", this.getClass().getName());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check to see if we can make a 'combining' factory of this aggregator that is suitable to process input from a
|
||||||
|
* selector of values produced by the other {@link AggregatorFactory} representing pre-aggregated data. Typically,
|
||||||
|
* this means that this and the other aggregator have the same inputs ({@link #requiredFields()}, and the same
|
||||||
|
* options for how the data was constructed into the intermediary type. If suitable, this method returns a
|
||||||
|
* 'combining' aggregator factory of this aggregator to use to process the pre-aggregated data which was produced by
|
||||||
|
* the other aggregator.
|
||||||
|
* <p>
|
||||||
|
* This method is used indirectly in service of checking if a
|
||||||
|
* {@link org.apache.druid.segment.projections.QueryableProjection} can be used instead of the base table during
|
||||||
|
* {@link org.apache.druid.segment.CursorFactory#makeCursorHolder(CursorBuildSpec)}, which checks if this
|
||||||
|
* aggregator can be substituted for its combining aggregator if and only if there exists a column that a cursor can
|
||||||
|
* read which was created by an aggregator that satisfies this method. In other words, this aggregator is the 'query'
|
||||||
|
* aggregator defined on the {@link CursorBuildSpec}, the argument to this method is the aggregator which created
|
||||||
|
* some column whose selectors are available to the cursor. If all aggregators on the {@link CursorBuildSpec} can be
|
||||||
|
* paired with aggregators from the underlying table in the cursor factory, then
|
||||||
|
* {@link CursorHolder#isPreAggregated()} will be set to true indicating that query engines should use this
|
||||||
|
* substituted aggregator instead of the original aggregators.
|
||||||
|
*
|
||||||
|
* @param preAggregated {@link AggregatorFactory} which produced the partially aggregated values which are
|
||||||
|
* available in a selector
|
||||||
|
* @return a "combining" {@link AggregatorFactory} to use with the pre-aggregated selector data
|
||||||
|
*/
|
||||||
|
@Nullable
|
||||||
|
public AggregatorFactory substituteCombiningFactory(AggregatorFactory preAggregated)
|
||||||
|
{
|
||||||
|
if (equals(preAggregated.withName(getName()))) {
|
||||||
|
return getCombiningFactory();
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Merges the list of AggregatorFactory[] (presumable from metadata of some segments being merged) and
|
* Merges the list of AggregatorFactory[] (presumable from metadata of some segments being merged) and
|
||||||
* returns merged AggregatorFactory[] (for the metadata for merged segment).
|
* returns merged AggregatorFactory[] (for the metadata for merged segment).
|
||||||
|
|
|
@ -454,13 +454,4 @@ public class AggregatorUtil
|
||||||
}
|
}
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static List<AggregatorFactory> getCombiningAggregators(List<AggregatorFactory> aggs)
|
|
||||||
{
|
|
||||||
List<AggregatorFactory> combining = new ArrayList<>(aggs.size());
|
|
||||||
for (AggregatorFactory agg : aggs) {
|
|
||||||
combining.add(agg.getCombiningFactory());
|
|
||||||
}
|
|
||||||
return combining;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,6 +20,7 @@
|
||||||
package org.apache.druid.query.groupby;
|
package org.apache.druid.query.groupby;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Supplier;
|
import com.google.common.base.Supplier;
|
||||||
import com.google.common.base.Suppliers;
|
import com.google.common.base.Suppliers;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
|
@ -60,7 +61,6 @@ import org.apache.druid.query.QueryWatcher;
|
||||||
import org.apache.druid.query.ResourceLimitExceededException;
|
import org.apache.druid.query.ResourceLimitExceededException;
|
||||||
import org.apache.druid.query.ResultMergeQueryRunner;
|
import org.apache.druid.query.ResultMergeQueryRunner;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||||
import org.apache.druid.query.aggregation.AggregatorUtil;
|
|
||||||
import org.apache.druid.query.aggregation.PostAggregator;
|
import org.apache.druid.query.aggregation.PostAggregator;
|
||||||
import org.apache.druid.query.context.ResponseContext;
|
import org.apache.druid.query.context.ResponseContext;
|
||||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||||
|
@ -509,7 +509,7 @@ public class GroupingEngine
|
||||||
final CursorHolder cursorHolder = closer.register(cursorFactory.makeCursorHolder(buildSpec));
|
final CursorHolder cursorHolder = closer.register(cursorFactory.makeCursorHolder(buildSpec));
|
||||||
|
|
||||||
if (cursorHolder.isPreAggregated()) {
|
if (cursorHolder.isPreAggregated()) {
|
||||||
query = query.withAggregatorSpecs(AggregatorUtil.getCombiningAggregators(query.getAggregatorSpecs()));
|
query = query.withAggregatorSpecs(Preconditions.checkNotNull(cursorHolder.getAggregatorsForPreAggregated()));
|
||||||
}
|
}
|
||||||
final ColumnInspector inspector = query.getVirtualColumns().wrapInspector(cursorFactory);
|
final ColumnInspector inspector = query.getVirtualColumns().wrapInspector(cursorFactory);
|
||||||
|
|
||||||
|
|
|
@ -151,6 +151,12 @@ public class DefaultSearchQueryMetrics implements SearchQueryMetrics
|
||||||
delegateQueryMetrics.segment(segmentIdentifier);
|
delegateQueryMetrics.segment(segmentIdentifier);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void projection(String projection)
|
||||||
|
{
|
||||||
|
delegateQueryMetrics.projection(projection);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void filterBundle(FilterBundle.BundleInfo bundleInfo)
|
public void filterBundle(FilterBundle.BundleInfo bundleInfo)
|
||||||
{
|
{
|
||||||
|
|
|
@ -20,6 +20,7 @@
|
||||||
package org.apache.druid.query.timeseries;
|
package org.apache.druid.query.timeseries;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import org.apache.druid.collections.NonBlockingPool;
|
import org.apache.druid.collections.NonBlockingPool;
|
||||||
|
@ -38,7 +39,6 @@ import org.apache.druid.query.Result;
|
||||||
import org.apache.druid.query.aggregation.Aggregator;
|
import org.apache.druid.query.aggregation.Aggregator;
|
||||||
import org.apache.druid.query.aggregation.AggregatorAdapters;
|
import org.apache.druid.query.aggregation.AggregatorAdapters;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||||
import org.apache.druid.query.aggregation.AggregatorUtil;
|
|
||||||
import org.apache.druid.query.vector.VectorCursorGranularizer;
|
import org.apache.druid.query.vector.VectorCursorGranularizer;
|
||||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||||
import org.apache.druid.segment.Cursor;
|
import org.apache.druid.segment.Cursor;
|
||||||
|
@ -104,7 +104,7 @@ public class TimeseriesQueryEngine
|
||||||
|
|
||||||
final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeCursorBuildSpec(query, timeseriesQueryMetrics));
|
final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeCursorBuildSpec(query, timeseriesQueryMetrics));
|
||||||
if (cursorHolder.isPreAggregated()) {
|
if (cursorHolder.isPreAggregated()) {
|
||||||
query = query.withAggregatorSpecs(AggregatorUtil.getCombiningAggregators(query.getAggregatorSpecs()));
|
query = query.withAggregatorSpecs(Preconditions.checkNotNull(cursorHolder.getAggregatorsForPreAggregated()));
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
final Sequence<Result<TimeseriesResultValue>> result;
|
final Sequence<Result<TimeseriesResultValue>> result;
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package org.apache.druid.query.topn;
|
package org.apache.druid.query.topn;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Predicates;
|
import com.google.common.base.Predicates;
|
||||||
import org.apache.druid.collections.NonBlockingPool;
|
import org.apache.druid.collections.NonBlockingPool;
|
||||||
import org.apache.druid.collections.ResourceHolder;
|
import org.apache.druid.collections.ResourceHolder;
|
||||||
|
@ -30,7 +31,6 @@ import org.apache.druid.query.CursorGranularizer;
|
||||||
import org.apache.druid.query.QueryMetrics;
|
import org.apache.druid.query.QueryMetrics;
|
||||||
import org.apache.druid.query.Result;
|
import org.apache.druid.query.Result;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||||
import org.apache.druid.query.aggregation.AggregatorUtil;
|
|
||||||
import org.apache.druid.query.extraction.ExtractionFn;
|
import org.apache.druid.query.extraction.ExtractionFn;
|
||||||
import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessor;
|
import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessor;
|
||||||
import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessorFactory;
|
import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessorFactory;
|
||||||
|
@ -89,7 +89,7 @@ public class TopNQueryEngine
|
||||||
final CursorBuildSpec buildSpec = makeCursorBuildSpec(query, queryMetrics);
|
final CursorBuildSpec buildSpec = makeCursorBuildSpec(query, queryMetrics);
|
||||||
final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec);
|
final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec);
|
||||||
if (cursorHolder.isPreAggregated()) {
|
if (cursorHolder.isPreAggregated()) {
|
||||||
query = query.withAggregatorSpecs(AggregatorUtil.getCombiningAggregators(query.getAggregatorSpecs()));
|
query = query.withAggregatorSpecs(Preconditions.checkNotNull(cursorHolder.getAggregatorsForPreAggregated()));
|
||||||
}
|
}
|
||||||
final Cursor cursor = cursorHolder.asCursor();
|
final Cursor cursor = cursorHolder.asCursor();
|
||||||
if (cursor == null) {
|
if (cursor == null) {
|
||||||
|
|
|
@ -0,0 +1,463 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF 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 org.apache.druid.segment;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||||
|
import com.google.common.collect.Interner;
|
||||||
|
import com.google.common.collect.Interners;
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
import org.apache.druid.data.input.impl.AggregateProjectionSpec;
|
||||||
|
import org.apache.druid.error.InvalidInput;
|
||||||
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
|
import org.apache.druid.java.util.common.granularity.Granularity;
|
||||||
|
import org.apache.druid.query.OrderBy;
|
||||||
|
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||||
|
import org.apache.druid.segment.column.ColumnHolder;
|
||||||
|
import org.apache.druid.segment.projections.Projections;
|
||||||
|
import org.apache.druid.utils.CollectionUtils;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Comparator;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Aggregate projection schema and row count information to store in {@link Metadata} which itself is stored inside a
|
||||||
|
* segment, defining which projections exist for the segment.
|
||||||
|
* <p>
|
||||||
|
* Decorated with {@link JsonTypeInfo} annotations as a future-proofing mechanism in the event we add other types of
|
||||||
|
* projections and need to extract out a base interface from this class.
|
||||||
|
*/
|
||||||
|
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
||||||
|
@JsonTypeName(AggregateProjectionSpec.TYPE_NAME)
|
||||||
|
public class AggregateProjectionMetadata
|
||||||
|
{
|
||||||
|
private static final Interner<Schema> SCHEMA_INTERNER = Interners.newWeakInterner();
|
||||||
|
|
||||||
|
public static final Comparator<AggregateProjectionMetadata> COMPARATOR = (o1, o2) -> {
|
||||||
|
int rowCompare = Integer.compare(o1.numRows, o2.numRows);
|
||||||
|
if (rowCompare != 0) {
|
||||||
|
return rowCompare;
|
||||||
|
}
|
||||||
|
return Schema.COMPARATOR.compare(o1.getSchema(), o2.getSchema());
|
||||||
|
};
|
||||||
|
|
||||||
|
private final Schema schema;
|
||||||
|
private final int numRows;
|
||||||
|
|
||||||
|
@JsonCreator
|
||||||
|
public AggregateProjectionMetadata(
|
||||||
|
@JsonProperty("schema") Schema schema,
|
||||||
|
@JsonProperty("numRows") int numRows
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.schema = SCHEMA_INTERNER.intern(schema);
|
||||||
|
this.numRows = numRows;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public Schema getSchema()
|
||||||
|
{
|
||||||
|
return schema;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public int getNumRows()
|
||||||
|
{
|
||||||
|
return numRows;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o)
|
||||||
|
{
|
||||||
|
if (this == o) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
if (o == null || getClass() != o.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
AggregateProjectionMetadata that = (AggregateProjectionMetadata) o;
|
||||||
|
return numRows == that.numRows && Objects.equals(schema, that.schema);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
return Objects.hash(schema, numRows);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Schema
|
||||||
|
{
|
||||||
|
/**
|
||||||
|
* It is not likely the best way to find the best matching projections, but it is the one we have for now. This
|
||||||
|
* comparator is used to sort all the projections in a segment "best" first, where best is defined as fewest grouping
|
||||||
|
* columns, most virtual columns and aggregators, as an approximation of likely to have the fewest number of rows to
|
||||||
|
* scan.
|
||||||
|
*/
|
||||||
|
public static final Comparator<Schema> COMPARATOR = (o1, o2) -> {
|
||||||
|
// coarsest granularity first
|
||||||
|
if (o1.getGranularity().isFinerThan(o2.getGranularity())) {
|
||||||
|
return 1;
|
||||||
|
}
|
||||||
|
if (o2.getGranularity().isFinerThan(o1.getGranularity())) {
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
// fewer dimensions first
|
||||||
|
final int dimsCompare = Integer.compare(
|
||||||
|
o1.groupingColumns.size(),
|
||||||
|
o2.groupingColumns.size()
|
||||||
|
);
|
||||||
|
if (dimsCompare != 0) {
|
||||||
|
return dimsCompare;
|
||||||
|
}
|
||||||
|
// more metrics first
|
||||||
|
int metCompare = Integer.compare(o2.aggregators.length, o1.aggregators.length);
|
||||||
|
if (metCompare != 0) {
|
||||||
|
return metCompare;
|
||||||
|
}
|
||||||
|
// more virtual columns first
|
||||||
|
final int virtCompare = Integer.compare(
|
||||||
|
o2.virtualColumns.getVirtualColumns().length,
|
||||||
|
o1.virtualColumns.getVirtualColumns().length
|
||||||
|
);
|
||||||
|
if (virtCompare != 0) {
|
||||||
|
return virtCompare;
|
||||||
|
}
|
||||||
|
return o1.name.compareTo(o2.name);
|
||||||
|
};
|
||||||
|
|
||||||
|
private final String name;
|
||||||
|
@Nullable
|
||||||
|
private final String timeColumnName;
|
||||||
|
private final VirtualColumns virtualColumns;
|
||||||
|
private final List<String> groupingColumns;
|
||||||
|
private final AggregatorFactory[] aggregators;
|
||||||
|
private final List<OrderBy> ordering;
|
||||||
|
private final List<OrderBy> orderingWithTimeSubstitution;
|
||||||
|
|
||||||
|
// computed fields
|
||||||
|
private final int timeColumnPosition;
|
||||||
|
private final Granularity granularity;
|
||||||
|
|
||||||
|
@JsonCreator
|
||||||
|
public Schema(
|
||||||
|
@JsonProperty("name") String name,
|
||||||
|
@JsonProperty("timeColumnName") @Nullable String timeColumnName,
|
||||||
|
@JsonProperty("virtualColumns") @Nullable VirtualColumns virtualColumns,
|
||||||
|
@JsonProperty("groupingColumns") List<String> groupingColumns,
|
||||||
|
@JsonProperty("aggregators") @Nullable AggregatorFactory[] aggregators,
|
||||||
|
@JsonProperty("ordering") List<OrderBy> ordering
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.name = name;
|
||||||
|
if (CollectionUtils.isNullOrEmpty(groupingColumns)) {
|
||||||
|
throw InvalidInput.exception("groupingColumns must not be null or empty");
|
||||||
|
}
|
||||||
|
this.virtualColumns = virtualColumns == null ? VirtualColumns.EMPTY : virtualColumns;
|
||||||
|
this.groupingColumns = groupingColumns;
|
||||||
|
this.aggregators = aggregators == null ? new AggregatorFactory[0] : aggregators;
|
||||||
|
this.ordering = ordering;
|
||||||
|
|
||||||
|
int foundTimePosition = -1;
|
||||||
|
this.orderingWithTimeSubstitution = Lists.newArrayListWithCapacity(ordering.size());
|
||||||
|
Granularity granularity = null;
|
||||||
|
for (int i = 0; i < ordering.size(); i++) {
|
||||||
|
OrderBy orderBy = ordering.get(i);
|
||||||
|
if (orderBy.getColumnName().equals(timeColumnName)) {
|
||||||
|
orderingWithTimeSubstitution.add(new OrderBy(ColumnHolder.TIME_COLUMN_NAME, orderBy.getOrder()));
|
||||||
|
foundTimePosition = i;
|
||||||
|
timeColumnName = groupingColumns.get(foundTimePosition);
|
||||||
|
final VirtualColumn vc = this.virtualColumns.getVirtualColumn(groupingColumns.get(foundTimePosition));
|
||||||
|
if (vc != null) {
|
||||||
|
granularity = Granularities.fromVirtualColumn(vc);
|
||||||
|
} else {
|
||||||
|
granularity = Granularities.NONE;
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
orderingWithTimeSubstitution.add(orderBy);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
this.timeColumnName = timeColumnName;
|
||||||
|
this.timeColumnPosition = foundTimePosition;
|
||||||
|
this.granularity = granularity == null ? Granularities.ALL : granularity;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public String getName()
|
||||||
|
{
|
||||||
|
return name;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
@Nullable
|
||||||
|
public String getTimeColumnName()
|
||||||
|
{
|
||||||
|
return timeColumnName;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
|
||||||
|
public VirtualColumns getVirtualColumns()
|
||||||
|
{
|
||||||
|
return virtualColumns;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
|
||||||
|
public List<String> getGroupingColumns()
|
||||||
|
{
|
||||||
|
return groupingColumns;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
|
||||||
|
public AggregatorFactory[] getAggregators()
|
||||||
|
{
|
||||||
|
return aggregators;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public List<OrderBy> getOrdering()
|
||||||
|
{
|
||||||
|
return ordering;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonIgnore
|
||||||
|
public List<OrderBy> getOrderingWithTimeColumnSubstitution()
|
||||||
|
{
|
||||||
|
return orderingWithTimeSubstitution;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonIgnore
|
||||||
|
public int getTimeColumnPosition()
|
||||||
|
{
|
||||||
|
return timeColumnPosition;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonIgnore
|
||||||
|
public Granularity getGranularity()
|
||||||
|
{
|
||||||
|
return granularity;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check if this projection "matches" a {@link CursorBuildSpec} for a query to see if we can use a projection
|
||||||
|
* instead. For a projection to match, all grouping columns of the build spec must match, virtual columns of the
|
||||||
|
* build spec must either be available as a physical column on the projection, or the inputs to the virtual column
|
||||||
|
* must be available on the projection, and all aggregators must be compatible with pre-aggregated columns of the
|
||||||
|
* projection per {@link AggregatorFactory#substituteCombiningFactory(AggregatorFactory)}. If the projection
|
||||||
|
* matches, this method returns a {@link Projections.ProjectionMatch} which contains an updated
|
||||||
|
* {@link CursorBuildSpec} which has the remaining virtual columns from the original build spec which must still be
|
||||||
|
* computed and the 'combining' aggregator factories to process the pre-aggregated data from the projection, as well
|
||||||
|
* as a mapping of query column names to projection column names.
|
||||||
|
*
|
||||||
|
* @param queryCursorBuildSpec the {@link CursorBuildSpec} that contains the required inputs to build a
|
||||||
|
* {@link CursorHolder} for a query
|
||||||
|
* @param physicalColumnChecker Helper utility which can determine if a physical column required by
|
||||||
|
* queryCursorBuildSpec is available on the projection OR does not exist on the base
|
||||||
|
* table either
|
||||||
|
* @return a {@link Projections.ProjectionMatch} if the {@link CursorBuildSpec} matches the projection, which
|
||||||
|
* contains information such as which
|
||||||
|
*/
|
||||||
|
@Nullable
|
||||||
|
public Projections.ProjectionMatch matches(
|
||||||
|
CursorBuildSpec queryCursorBuildSpec,
|
||||||
|
Projections.PhysicalColumnChecker physicalColumnChecker
|
||||||
|
)
|
||||||
|
{
|
||||||
|
if (!queryCursorBuildSpec.isCompatibleOrdering(orderingWithTimeSubstitution)) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
final List<String> queryGrouping = queryCursorBuildSpec.getGroupingColumns();
|
||||||
|
Projections.ProjectionMatchBuilder matchBuilder = new Projections.ProjectionMatchBuilder();
|
||||||
|
|
||||||
|
if (timeColumnName != null) {
|
||||||
|
matchBuilder.remapColumn(timeColumnName, ColumnHolder.TIME_COLUMN_NAME);
|
||||||
|
}
|
||||||
|
if (queryGrouping != null) {
|
||||||
|
for (String queryColumn : queryGrouping) {
|
||||||
|
matchBuilder = matchRequiredColumn(
|
||||||
|
matchBuilder,
|
||||||
|
queryColumn,
|
||||||
|
queryCursorBuildSpec.getVirtualColumns(),
|
||||||
|
physicalColumnChecker
|
||||||
|
);
|
||||||
|
if (matchBuilder == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (queryCursorBuildSpec.getFilter() != null) {
|
||||||
|
for (String queryColumn : queryCursorBuildSpec.getFilter().getRequiredColumns()) {
|
||||||
|
matchBuilder = matchRequiredColumn(
|
||||||
|
matchBuilder,
|
||||||
|
queryColumn,
|
||||||
|
queryCursorBuildSpec.getVirtualColumns(),
|
||||||
|
physicalColumnChecker
|
||||||
|
);
|
||||||
|
if (matchBuilder == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (!CollectionUtils.isNullOrEmpty(queryCursorBuildSpec.getAggregators())) {
|
||||||
|
boolean allMatch = true;
|
||||||
|
for (AggregatorFactory queryAgg : queryCursorBuildSpec.getAggregators()) {
|
||||||
|
boolean foundMatch = false;
|
||||||
|
for (AggregatorFactory projectionAgg : aggregators) {
|
||||||
|
final AggregatorFactory combining = queryAgg.substituteCombiningFactory(projectionAgg);
|
||||||
|
if (combining != null) {
|
||||||
|
matchBuilder.remapColumn(queryAgg.getName(), projectionAgg.getName()).addPreAggregatedAggregator(combining);
|
||||||
|
foundMatch = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
allMatch = allMatch && foundMatch;
|
||||||
|
}
|
||||||
|
if (!allMatch) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return matchBuilder.build(queryCursorBuildSpec);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Ensure that the projection has the specified column required by a {@link CursorBuildSpec} in one form or another.
|
||||||
|
* If the column is a {@link VirtualColumn} on the build spec, ensure that the projection has an equivalent virtual
|
||||||
|
* column, or has the required inputs to compute the virtual column. If an equivalent virtual column exists, its
|
||||||
|
* name will be added to {@link Projections.ProjectionMatchBuilder#remapColumn(String, String)} so the query
|
||||||
|
* virtual column name can be mapped to the projection physical column name. If no equivalent virtual column exists,
|
||||||
|
* but the inputs are available on the projection to compute it, it will be added to
|
||||||
|
* {@link Projections.ProjectionMatchBuilder#addReferenceedVirtualColumn(VirtualColumn)}.
|
||||||
|
* <p>
|
||||||
|
* Finally, if the column is not a virtual column in the query, it is checked with
|
||||||
|
* {@link Projections.PhysicalColumnChecker} which true if the column is present on the projection OR if the column
|
||||||
|
* is NOT present on the base table (meaning missing columns that do not exist anywhere do not disqualify a
|
||||||
|
* projection from being used).
|
||||||
|
*
|
||||||
|
* @param matchBuilder match state to add mappings of query virtual columns to projection physical columns
|
||||||
|
* and query virtual columns which still must be computed from projection physical
|
||||||
|
* columns
|
||||||
|
* @param column Column name to check
|
||||||
|
* @param queryVirtualColumns {@link VirtualColumns} from the {@link CursorBuildSpec} required by the query
|
||||||
|
* @param physicalColumnChecker Helper to check if the physical column exists on a projection, or does not exist on
|
||||||
|
* the base table
|
||||||
|
* @return {@link Projections.ProjectionMatchBuilder} with updated state per the rules described above, or null
|
||||||
|
* if the column cannot be matched
|
||||||
|
*/
|
||||||
|
@Nullable
|
||||||
|
private Projections.ProjectionMatchBuilder matchRequiredColumn(
|
||||||
|
Projections.ProjectionMatchBuilder matchBuilder,
|
||||||
|
String column,
|
||||||
|
VirtualColumns queryVirtualColumns,
|
||||||
|
Projections.PhysicalColumnChecker physicalColumnChecker
|
||||||
|
)
|
||||||
|
{
|
||||||
|
final VirtualColumn buildSpecVirtualColumn = queryVirtualColumns.getVirtualColumn(column);
|
||||||
|
if (buildSpecVirtualColumn != null) {
|
||||||
|
// check to see if we have an equivalent virtual column defined in the projection, if so we can
|
||||||
|
final VirtualColumn projectionEquivalent = virtualColumns.findEquivalent(buildSpecVirtualColumn);
|
||||||
|
if (projectionEquivalent != null) {
|
||||||
|
if (!buildSpecVirtualColumn.getOutputName().equals(projectionEquivalent.getOutputName())) {
|
||||||
|
matchBuilder.remapColumn(
|
||||||
|
buildSpecVirtualColumn.getOutputName(),
|
||||||
|
projectionEquivalent.getOutputName()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
return matchBuilder;
|
||||||
|
}
|
||||||
|
|
||||||
|
matchBuilder.addReferenceedVirtualColumn(buildSpecVirtualColumn);
|
||||||
|
final List<String> requiredInputs = buildSpecVirtualColumn.requiredColumns();
|
||||||
|
if (requiredInputs.size() == 1 && ColumnHolder.TIME_COLUMN_NAME.equals(requiredInputs.get(0))) {
|
||||||
|
// special handle time granularity. in the future this should be reworked to push this concept into the
|
||||||
|
// virtual column and underlying expression itself, but this will do for now
|
||||||
|
final Granularity virtualGranularity = Granularities.fromVirtualColumn(buildSpecVirtualColumn);
|
||||||
|
if (virtualGranularity != null) {
|
||||||
|
if (virtualGranularity.isFinerThan(granularity)) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
return matchBuilder.remapColumn(column, timeColumnName);
|
||||||
|
} else {
|
||||||
|
// anything else with __time requires none granularity
|
||||||
|
if (Granularities.NONE.equals(granularity)) {
|
||||||
|
return matchBuilder;
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
for (String required : requiredInputs) {
|
||||||
|
matchBuilder = matchRequiredColumn(
|
||||||
|
matchBuilder,
|
||||||
|
required,
|
||||||
|
queryVirtualColumns,
|
||||||
|
physicalColumnChecker
|
||||||
|
);
|
||||||
|
if (matchBuilder == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return matchBuilder;
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
if (physicalColumnChecker.check(name, column)) {
|
||||||
|
return matchBuilder;
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o)
|
||||||
|
{
|
||||||
|
if (this == o) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
if (o == null || getClass() != o.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
Schema schema = (Schema) o;
|
||||||
|
return Objects.equals(name, schema.name)
|
||||||
|
&& Objects.equals(timeColumnName, schema.timeColumnName)
|
||||||
|
&& Objects.equals(virtualColumns, schema.virtualColumns)
|
||||||
|
&& Objects.equals(groupingColumns, schema.groupingColumns)
|
||||||
|
&& Objects.deepEquals(aggregators, schema.aggregators)
|
||||||
|
&& Objects.equals(ordering, schema.ordering);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
return Objects.hash(
|
||||||
|
name,
|
||||||
|
timeColumnName,
|
||||||
|
virtualColumns,
|
||||||
|
groupingColumns,
|
||||||
|
Arrays.hashCode(aggregators),
|
||||||
|
ordering
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -27,6 +27,7 @@ import org.apache.druid.common.guava.GuavaUtils;
|
||||||
import org.apache.druid.data.input.impl.DimensionSchema;
|
import org.apache.druid.data.input.impl.DimensionSchema;
|
||||||
import org.apache.druid.java.util.common.IAE;
|
import org.apache.druid.java.util.common.IAE;
|
||||||
import org.apache.druid.java.util.common.ISE;
|
import org.apache.druid.java.util.common.ISE;
|
||||||
|
import org.apache.druid.java.util.common.Numbers;
|
||||||
import org.apache.druid.java.util.common.UOE;
|
import org.apache.druid.java.util.common.UOE;
|
||||||
import org.apache.druid.java.util.common.parsers.ParseException;
|
import org.apache.druid.java.util.common.parsers.ParseException;
|
||||||
import org.apache.druid.math.expr.Evals;
|
import org.apache.druid.math.expr.Evals;
|
||||||
|
@ -360,8 +361,42 @@ public class AutoTypeColumnIndexer implements DimensionIndexer<StructuredData, S
|
||||||
return rootLiteralSelector;
|
return rootLiteralSelector;
|
||||||
}
|
}
|
||||||
|
|
||||||
return new ObjectColumnSelector<StructuredData>()
|
return new ColumnValueSelector<Object>()
|
||||||
{
|
{
|
||||||
|
@Override
|
||||||
|
public double getDouble()
|
||||||
|
{
|
||||||
|
Object o = StructuredData.unwrap(getObject());
|
||||||
|
return Numbers.tryParseDouble(o, 0.0);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public float getFloat()
|
||||||
|
{
|
||||||
|
Object o = StructuredData.unwrap(getObject());
|
||||||
|
return Numbers.tryParseFloat(o, 0.0f);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getLong()
|
||||||
|
{
|
||||||
|
Object o = StructuredData.unwrap(getObject());
|
||||||
|
return Numbers.tryParseLong(o, 0L);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isNull()
|
||||||
|
{
|
||||||
|
final Object o = StructuredData.unwrap(getObject());
|
||||||
|
if (o instanceof Number) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (o instanceof String) {
|
||||||
|
return GuavaUtils.tryParseLong((String) o) == null && Doubles.tryParse((String) o) == null;
|
||||||
|
}
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||||
{
|
{
|
||||||
|
|
|
@ -75,6 +75,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
||||||
SimpleDictionaryMergingIterator.makePeekingComparator();
|
SimpleDictionaryMergingIterator.makePeekingComparator();
|
||||||
|
|
||||||
private final String name;
|
private final String name;
|
||||||
|
private final String outputName;
|
||||||
private final IndexSpec indexSpec;
|
private final IndexSpec indexSpec;
|
||||||
private final SegmentWriteOutMedium segmentWriteOutMedium;
|
private final SegmentWriteOutMedium segmentWriteOutMedium;
|
||||||
private final Closer closer;
|
private final Closer closer;
|
||||||
|
@ -84,9 +85,23 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
||||||
@Nullable
|
@Nullable
|
||||||
private final ColumnType castToType;
|
private final ColumnType castToType;
|
||||||
private boolean isVariantType = false;
|
private boolean isVariantType = false;
|
||||||
|
private byte variantTypeByte = 0x00;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param name column name
|
||||||
|
* @param outputName output smoosh file name. if this is a base table column, it will be the equivalent to
|
||||||
|
* name, however if this merger is for a projection, this will be prefixed with the
|
||||||
|
* projection name so that multiple projections can store the same column name at
|
||||||
|
* different smoosh file "paths"
|
||||||
|
* @param castToType optional mechanism to enforce that all values are a specific type
|
||||||
|
* @param indexSpec segment level storage options such as compression format and bitmap type
|
||||||
|
* @param segmentWriteOutMedium temporary storage location to stage segment outputs before finalizing into the segment
|
||||||
|
* @param closer resource closer if this merger needs to attach any closables that should be cleaned up
|
||||||
|
* when the segment is finished writing
|
||||||
|
*/
|
||||||
public AutoTypeColumnMerger(
|
public AutoTypeColumnMerger(
|
||||||
String name,
|
String name,
|
||||||
|
String outputName,
|
||||||
@Nullable ColumnType castToType,
|
@Nullable ColumnType castToType,
|
||||||
IndexSpec indexSpec,
|
IndexSpec indexSpec,
|
||||||
SegmentWriteOutMedium segmentWriteOutMedium,
|
SegmentWriteOutMedium segmentWriteOutMedium,
|
||||||
|
@ -95,6 +110,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
||||||
{
|
{
|
||||||
|
|
||||||
this.name = name;
|
this.name = name;
|
||||||
|
this.outputName = outputName;
|
||||||
this.castToType = castToType;
|
this.castToType = castToType;
|
||||||
this.indexSpec = indexSpec;
|
this.indexSpec = indexSpec;
|
||||||
this.segmentWriteOutMedium = segmentWriteOutMedium;
|
this.segmentWriteOutMedium = segmentWriteOutMedium;
|
||||||
|
@ -165,7 +181,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
||||||
if (explicitType == null && !forceNested && ((isConstant && constantValue == null) || numMergeIndex == 0)) {
|
if (explicitType == null && !forceNested && ((isConstant && constantValue == null) || numMergeIndex == 0)) {
|
||||||
logicalType = ColumnType.STRING;
|
logicalType = ColumnType.STRING;
|
||||||
serializer = new ScalarStringColumnSerializer(
|
serializer = new ScalarStringColumnSerializer(
|
||||||
name,
|
outputName,
|
||||||
indexSpec,
|
indexSpec,
|
||||||
segmentWriteOutMedium,
|
segmentWriteOutMedium,
|
||||||
closer
|
closer
|
||||||
|
@ -179,7 +195,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
||||||
switch (logicalType.getType()) {
|
switch (logicalType.getType()) {
|
||||||
case LONG:
|
case LONG:
|
||||||
serializer = new ScalarLongColumnSerializer(
|
serializer = new ScalarLongColumnSerializer(
|
||||||
name,
|
outputName,
|
||||||
indexSpec,
|
indexSpec,
|
||||||
segmentWriteOutMedium,
|
segmentWriteOutMedium,
|
||||||
closer
|
closer
|
||||||
|
@ -187,7 +203,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
||||||
break;
|
break;
|
||||||
case DOUBLE:
|
case DOUBLE:
|
||||||
serializer = new ScalarDoubleColumnSerializer(
|
serializer = new ScalarDoubleColumnSerializer(
|
||||||
name,
|
outputName,
|
||||||
indexSpec,
|
indexSpec,
|
||||||
segmentWriteOutMedium,
|
segmentWriteOutMedium,
|
||||||
closer
|
closer
|
||||||
|
@ -195,7 +211,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
||||||
break;
|
break;
|
||||||
case STRING:
|
case STRING:
|
||||||
serializer = new ScalarStringColumnSerializer(
|
serializer = new ScalarStringColumnSerializer(
|
||||||
name,
|
outputName,
|
||||||
indexSpec,
|
indexSpec,
|
||||||
segmentWriteOutMedium,
|
segmentWriteOutMedium,
|
||||||
closer
|
closer
|
||||||
|
@ -203,7 +219,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
||||||
break;
|
break;
|
||||||
case ARRAY:
|
case ARRAY:
|
||||||
serializer = new VariantColumnSerializer(
|
serializer = new VariantColumnSerializer(
|
||||||
name,
|
outputName,
|
||||||
logicalType,
|
logicalType,
|
||||||
null,
|
null,
|
||||||
indexSpec,
|
indexSpec,
|
||||||
|
@ -222,6 +238,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
||||||
// mixed type column, but only root path, we can use VariantArrayColumnSerializer
|
// mixed type column, but only root path, we can use VariantArrayColumnSerializer
|
||||||
// pick the least restrictive type for the logical type
|
// pick the least restrictive type for the logical type
|
||||||
isVariantType = true;
|
isVariantType = true;
|
||||||
|
variantTypeByte = rootTypes.getByteValue();
|
||||||
for (ColumnType type : FieldTypeInfo.convertToSet(rootTypes.getByteValue())) {
|
for (ColumnType type : FieldTypeInfo.convertToSet(rootTypes.getByteValue())) {
|
||||||
logicalType = ColumnType.leastRestrictiveType(logicalType, type);
|
logicalType = ColumnType.leastRestrictiveType(logicalType, type);
|
||||||
}
|
}
|
||||||
|
@ -230,9 +247,9 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
||||||
logicalType = ColumnTypeFactory.getInstance().ofArray(logicalType);
|
logicalType = ColumnTypeFactory.getInstance().ofArray(logicalType);
|
||||||
}
|
}
|
||||||
serializer = new VariantColumnSerializer(
|
serializer = new VariantColumnSerializer(
|
||||||
name,
|
outputName,
|
||||||
null,
|
null,
|
||||||
rootTypes.getByteValue(),
|
variantTypeByte,
|
||||||
indexSpec,
|
indexSpec,
|
||||||
segmentWriteOutMedium,
|
segmentWriteOutMedium,
|
||||||
closer
|
closer
|
||||||
|
@ -241,7 +258,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
||||||
// all the bells and whistles
|
// all the bells and whistles
|
||||||
logicalType = ColumnType.NESTED_DATA;
|
logicalType = ColumnType.NESTED_DATA;
|
||||||
serializer = new NestedDataColumnSerializer(
|
serializer = new NestedDataColumnSerializer(
|
||||||
name,
|
outputName,
|
||||||
indexSpec,
|
indexSpec,
|
||||||
segmentWriteOutMedium,
|
segmentWriteOutMedium,
|
||||||
closer
|
closer
|
||||||
|
@ -262,7 +279,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
||||||
sortedLookup.getSortedDoubles(),
|
sortedLookup.getSortedDoubles(),
|
||||||
() -> new ArrayDictionaryMergingIterator(
|
() -> new ArrayDictionaryMergingIterator(
|
||||||
sortedArrayLookups,
|
sortedArrayLookups,
|
||||||
serializer.getGlobalLookup()
|
serializer.getDictionaryIdLookup()
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
stringCardinality = sortedLookup.getStringCardinality();
|
stringCardinality = sortedLookup.getStringCardinality();
|
||||||
|
@ -284,7 +301,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
||||||
);
|
);
|
||||||
final ArrayDictionaryMergingIterator arrayIterator = new ArrayDictionaryMergingIterator(
|
final ArrayDictionaryMergingIterator arrayIterator = new ArrayDictionaryMergingIterator(
|
||||||
sortedArrayLookups,
|
sortedArrayLookups,
|
||||||
serializer.getGlobalLookup()
|
serializer.getDictionaryIdLookup()
|
||||||
);
|
);
|
||||||
serializer.serializeDictionaries(
|
serializer.serializeDictionaries(
|
||||||
() -> stringIterator,
|
() -> stringIterator,
|
||||||
|
@ -367,6 +384,76 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
||||||
return descriptorBuilder.build();
|
return descriptorBuilder.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected DictionaryIdLookup getIdLookup()
|
||||||
|
{
|
||||||
|
return serializer.getDictionaryIdLookup();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void attachParent(DimensionMergerV9 parent, List<IndexableAdapter> projectionAdapters) throws IOException
|
||||||
|
{
|
||||||
|
DruidException.conditionalDefensive(
|
||||||
|
parent instanceof AutoTypeColumnMerger,
|
||||||
|
"Projection parent dimension must be same type, got [%s]",
|
||||||
|
parent.getClass()
|
||||||
|
);
|
||||||
|
AutoTypeColumnMerger autoParent = (AutoTypeColumnMerger) parent;
|
||||||
|
logicalType = autoParent.logicalType;
|
||||||
|
isVariantType = autoParent.isVariantType;
|
||||||
|
if (autoParent.serializer instanceof ScalarStringColumnSerializer) {
|
||||||
|
serializer = new ScalarStringColumnSerializer(
|
||||||
|
outputName,
|
||||||
|
indexSpec,
|
||||||
|
segmentWriteOutMedium,
|
||||||
|
closer
|
||||||
|
);
|
||||||
|
} else if (autoParent.serializer instanceof ScalarLongColumnSerializer) {
|
||||||
|
serializer = new ScalarLongColumnSerializer(
|
||||||
|
outputName,
|
||||||
|
indexSpec,
|
||||||
|
segmentWriteOutMedium,
|
||||||
|
closer
|
||||||
|
);
|
||||||
|
} else if (autoParent.serializer instanceof ScalarDoubleColumnSerializer) {
|
||||||
|
serializer = new ScalarDoubleColumnSerializer(
|
||||||
|
outputName,
|
||||||
|
indexSpec,
|
||||||
|
segmentWriteOutMedium,
|
||||||
|
closer
|
||||||
|
);
|
||||||
|
} else if (autoParent.serializer instanceof VariantColumnSerializer) {
|
||||||
|
if (autoParent.isVariantType) {
|
||||||
|
serializer = new VariantColumnSerializer(
|
||||||
|
outputName,
|
||||||
|
null,
|
||||||
|
variantTypeByte,
|
||||||
|
indexSpec,
|
||||||
|
segmentWriteOutMedium,
|
||||||
|
closer
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
serializer = new VariantColumnSerializer(
|
||||||
|
outputName,
|
||||||
|
logicalType,
|
||||||
|
null,
|
||||||
|
indexSpec,
|
||||||
|
segmentWriteOutMedium,
|
||||||
|
closer
|
||||||
|
);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
serializer = new NestedDataColumnSerializer(
|
||||||
|
outputName,
|
||||||
|
indexSpec,
|
||||||
|
segmentWriteOutMedium,
|
||||||
|
closer
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
serializer.setDictionaryIdLookup(autoParent.getIdLookup());
|
||||||
|
serializer.open();
|
||||||
|
}
|
||||||
|
|
||||||
public static class ArrayDictionaryMergingIterator implements Iterator<int[]>
|
public static class ArrayDictionaryMergingIterator implements Iterator<int[]>
|
||||||
{
|
{
|
||||||
private static final Comparator<PeekingIterator<int[]>> PEEKING_ITERATOR_COMPARATOR =
|
private static final Comparator<PeekingIterator<int[]>> PEEKING_ITERATOR_COMPARATOR =
|
||||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.druid.query.QueryContext;
|
||||||
import org.apache.druid.query.QueryMetrics;
|
import org.apache.druid.query.QueryMetrics;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||||
import org.apache.druid.query.filter.Filter;
|
import org.apache.druid.query.filter.Filter;
|
||||||
|
import org.apache.druid.utils.CollectionUtils;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
@ -54,10 +55,12 @@ public class CursorBuildSpec
|
||||||
private final VirtualColumns virtualColumns;
|
private final VirtualColumns virtualColumns;
|
||||||
@Nullable
|
@Nullable
|
||||||
private final List<AggregatorFactory> aggregators;
|
private final List<AggregatorFactory> aggregators;
|
||||||
private final List<OrderBy> orderByColumns;
|
private final List<OrderBy> preferredOrdering;
|
||||||
|
|
||||||
private final QueryContext queryContext;
|
private final QueryContext queryContext;
|
||||||
|
|
||||||
|
private final boolean isAggregate;
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
private final QueryMetrics<?> queryMetrics;
|
private final QueryMetrics<?> queryMetrics;
|
||||||
|
|
||||||
|
@ -77,9 +80,10 @@ public class CursorBuildSpec
|
||||||
this.groupingColumns = groupingColumns;
|
this.groupingColumns = groupingColumns;
|
||||||
this.virtualColumns = Preconditions.checkNotNull(virtualColumns, "virtualColumns");
|
this.virtualColumns = Preconditions.checkNotNull(virtualColumns, "virtualColumns");
|
||||||
this.aggregators = aggregators;
|
this.aggregators = aggregators;
|
||||||
this.orderByColumns = Preconditions.checkNotNull(preferredOrdering, "preferredOrdering");
|
this.preferredOrdering = Preconditions.checkNotNull(preferredOrdering, "preferredOrdering");
|
||||||
this.queryContext = Preconditions.checkNotNull(queryContext, "queryContext");
|
this.queryContext = Preconditions.checkNotNull(queryContext, "queryContext");
|
||||||
this.queryMetrics = queryMetrics;
|
this.queryMetrics = queryMetrics;
|
||||||
|
this.isAggregate = !CollectionUtils.isNullOrEmpty(groupingColumns) || !CollectionUtils.isNullOrEmpty(aggregators);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -142,7 +146,7 @@ public class CursorBuildSpec
|
||||||
*/
|
*/
|
||||||
public List<OrderBy> getPreferredOrdering()
|
public List<OrderBy> getPreferredOrdering()
|
||||||
{
|
{
|
||||||
return orderByColumns;
|
return preferredOrdering;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -166,6 +170,39 @@ public class CursorBuildSpec
|
||||||
return queryMetrics;
|
return queryMetrics;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns true if {@link #getGroupingColumns()} is not null or empty and/or {@link #getAggregators()} is not null or
|
||||||
|
* empty. This method is useful for quickly checking if it is worth considering if a {@link CursorFactory} should
|
||||||
|
* attempt to produce a {@link CursorHolder} that is {@link CursorHolder#isPreAggregated()} to satisfy the build spec.
|
||||||
|
*/
|
||||||
|
public boolean isAggregate()
|
||||||
|
{
|
||||||
|
return isAggregate;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns true if the supplied ordering matches {@link #getPreferredOrdering()}, meaning that the supplied ordering
|
||||||
|
* has everything which is in the preferred ordering in the same direction and order. The supplied ordering may have
|
||||||
|
* additional columns beyond the preferred ordering and still satisify this method.
|
||||||
|
*/
|
||||||
|
public boolean isCompatibleOrdering(List<OrderBy> ordering)
|
||||||
|
{
|
||||||
|
// if the build spec doesn't prefer an ordering, any order is ok
|
||||||
|
if (preferredOrdering.isEmpty()) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
// all columns must be present in ordering if the build spec specifies them
|
||||||
|
if (ordering.size() < preferredOrdering.size()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
for (int i = 0; i < preferredOrdering.size(); i++) {
|
||||||
|
if (!ordering.get(i).equals(preferredOrdering.get(i))) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
public static class CursorBuildSpecBuilder
|
public static class CursorBuildSpecBuilder
|
||||||
{
|
{
|
||||||
@Nullable
|
@Nullable
|
||||||
|
@ -195,7 +232,7 @@ public class CursorBuildSpec
|
||||||
this.groupingColumns = buildSpec.groupingColumns;
|
this.groupingColumns = buildSpec.groupingColumns;
|
||||||
this.virtualColumns = buildSpec.virtualColumns;
|
this.virtualColumns = buildSpec.virtualColumns;
|
||||||
this.aggregators = buildSpec.aggregators;
|
this.aggregators = buildSpec.aggregators;
|
||||||
this.preferredOrdering = buildSpec.orderByColumns;
|
this.preferredOrdering = buildSpec.preferredOrdering;
|
||||||
this.queryContext = buildSpec.queryContext;
|
this.queryContext = buildSpec.queryContext;
|
||||||
this.queryMetrics = buildSpec.queryMetrics;
|
this.queryMetrics = buildSpec.queryMetrics;
|
||||||
}
|
}
|
||||||
|
|
|
@ -75,6 +75,18 @@ public interface CursorHolder extends Closeable
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a set of replacement {@link AggregatorFactory} if and only if {@link #isPreAggregated()} is true. The
|
||||||
|
* query engine should replace the query aggregators with these aggregators, which are combining aggregators derived
|
||||||
|
* from the {@link CursorBuildSpec} passed into {@link CursorFactory#makeCursorHolder(CursorBuildSpec)}. If
|
||||||
|
* {@link #isPreAggregated()} is not true, this method returns null
|
||||||
|
*/
|
||||||
|
@Nullable
|
||||||
|
default List<AggregatorFactory> getAggregatorsForPreAggregated()
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns cursor ordering, which may or may not match {@link CursorBuildSpec#getPreferredOrdering()}. If returns
|
* Returns cursor ordering, which may or may not match {@link CursorBuildSpec#getPreferredOrdering()}. If returns
|
||||||
* an empty list then the cursor has no defined ordering.
|
* an empty list then the cursor has no defined ordering.
|
||||||
|
|
|
@ -70,6 +70,7 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
|
||||||
private static final Logger log = new Logger(DictionaryEncodedColumnMerger.class);
|
private static final Logger log = new Logger(DictionaryEncodedColumnMerger.class);
|
||||||
|
|
||||||
protected final String dimensionName;
|
protected final String dimensionName;
|
||||||
|
protected final String outputName;
|
||||||
protected final ProgressIndicator progress;
|
protected final ProgressIndicator progress;
|
||||||
protected final Closer closer;
|
protected final Closer closer;
|
||||||
protected final IndexSpec indexSpec;
|
protected final IndexSpec indexSpec;
|
||||||
|
@ -81,6 +82,7 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
|
||||||
protected int rowCount = 0;
|
protected int rowCount = 0;
|
||||||
protected int cardinality = 0;
|
protected int cardinality = 0;
|
||||||
protected boolean hasNull = false;
|
protected boolean hasNull = false;
|
||||||
|
protected boolean writeDictionary = true;
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
protected GenericIndexedWriter<ImmutableBitmap> bitmapWriter;
|
protected GenericIndexedWriter<ImmutableBitmap> bitmapWriter;
|
||||||
|
@ -102,6 +104,7 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
|
||||||
|
|
||||||
public DictionaryEncodedColumnMerger(
|
public DictionaryEncodedColumnMerger(
|
||||||
String dimensionName,
|
String dimensionName,
|
||||||
|
String outputName,
|
||||||
IndexSpec indexSpec,
|
IndexSpec indexSpec,
|
||||||
SegmentWriteOutMedium segmentWriteOutMedium,
|
SegmentWriteOutMedium segmentWriteOutMedium,
|
||||||
ColumnCapabilities capabilities,
|
ColumnCapabilities capabilities,
|
||||||
|
@ -110,6 +113,7 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.dimensionName = dimensionName;
|
this.dimensionName = dimensionName;
|
||||||
|
this.outputName = outputName;
|
||||||
this.indexSpec = indexSpec;
|
this.indexSpec = indexSpec;
|
||||||
this.capabilities = capabilities;
|
this.capabilities = capabilities;
|
||||||
this.segmentWriteOutMedium = segmentWriteOutMedium;
|
this.segmentWriteOutMedium = segmentWriteOutMedium;
|
||||||
|
@ -171,8 +175,9 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
|
||||||
numMergeIndex++;
|
numMergeIndex++;
|
||||||
}
|
}
|
||||||
|
|
||||||
String dictFilename = StringUtils.format("%s.dim_values", dimensionName);
|
String dictFilename = StringUtils.format("%s.dim_values", outputName);
|
||||||
dictionaryWriter = makeDictionaryWriter(dictFilename);
|
dictionaryWriter = makeDictionaryWriter(dictFilename);
|
||||||
|
|
||||||
firstDictionaryValue = null;
|
firstDictionaryValue = null;
|
||||||
dictionarySize = 0;
|
dictionarySize = 0;
|
||||||
dictionaryWriter.open();
|
dictionaryWriter.open();
|
||||||
|
@ -338,7 +343,7 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
|
||||||
long dimStartTime = System.currentTimeMillis();
|
long dimStartTime = System.currentTimeMillis();
|
||||||
final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory();
|
final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory();
|
||||||
|
|
||||||
String bmpFilename = StringUtils.format("%s.inverted", dimensionName);
|
String bmpFilename = StringUtils.format("%s.inverted", outputName);
|
||||||
bitmapWriter = new GenericIndexedWriter<>(
|
bitmapWriter = new GenericIndexedWriter<>(
|
||||||
segmentWriteOutMedium,
|
segmentWriteOutMedium,
|
||||||
bmpFilename,
|
bmpFilename,
|
||||||
|
@ -402,11 +407,11 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
|
||||||
{
|
{
|
||||||
final CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression();
|
final CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression();
|
||||||
|
|
||||||
String filenameBase = StringUtils.format("%s.forward_dim", dimensionName);
|
String filenameBase = StringUtils.format("%s.forward_dim", outputName);
|
||||||
if (capabilities.hasMultipleValues().isTrue()) {
|
if (capabilities.hasMultipleValues().isTrue()) {
|
||||||
if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) {
|
if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) {
|
||||||
encodedValueSerializer = V3CompressedVSizeColumnarMultiIntsSerializer.create(
|
encodedValueSerializer = V3CompressedVSizeColumnarMultiIntsSerializer.create(
|
||||||
dimensionName,
|
outputName,
|
||||||
segmentWriteOutMedium,
|
segmentWriteOutMedium,
|
||||||
filenameBase,
|
filenameBase,
|
||||||
cardinality,
|
cardinality,
|
||||||
|
@ -414,12 +419,12 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
encodedValueSerializer =
|
encodedValueSerializer =
|
||||||
new VSizeColumnarMultiIntsSerializer(dimensionName, segmentWriteOutMedium, cardinality);
|
new VSizeColumnarMultiIntsSerializer(outputName, segmentWriteOutMedium, cardinality);
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) {
|
if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) {
|
||||||
encodedValueSerializer = CompressedVSizeColumnarIntsSerializer.create(
|
encodedValueSerializer = CompressedVSizeColumnarIntsSerializer.create(
|
||||||
dimensionName,
|
outputName,
|
||||||
segmentWriteOutMedium,
|
segmentWriteOutMedium,
|
||||||
filenameBase,
|
filenameBase,
|
||||||
cardinality,
|
cardinality,
|
||||||
|
|
|
@ -106,13 +106,17 @@ public interface DimensionHandler
|
||||||
*
|
*
|
||||||
* See {@link DimensionMergerV9} interface for more information.
|
* See {@link DimensionMergerV9} interface for more information.
|
||||||
*
|
*
|
||||||
* @param indexSpec Specification object for the index merge
|
* @param outputName Output "file" name for the column to use for serializers, to control where it is
|
||||||
* @param segmentWriteOutMedium this SegmentWriteOutMedium object could be used internally in the created merger, if needed
|
* stored in the segments internal files
|
||||||
* @param capabilities The ColumnCapabilities of the dimension represented by this DimensionHandler
|
* @param indexSpec Specification object for the index merge
|
||||||
* @param progress ProgressIndicator used by the merging process
|
* @param segmentWriteOutMedium this SegmentWriteOutMedium object could be used internally in the created merger, if
|
||||||
|
* needed
|
||||||
|
* @param capabilities The ColumnCapabilities of the dimension represented by this DimensionHandler
|
||||||
|
* @param progress ProgressIndicator used by the merging process
|
||||||
* @return A new DimensionMergerV9 object.
|
* @return A new DimensionMergerV9 object.
|
||||||
*/
|
*/
|
||||||
DimensionMergerV9 makeMerger(
|
DimensionMergerV9 makeMerger(
|
||||||
|
String outputName,
|
||||||
IndexSpec indexSpec,
|
IndexSpec indexSpec,
|
||||||
SegmentWriteOutMedium segmentWriteOutMedium,
|
SegmentWriteOutMedium segmentWriteOutMedium,
|
||||||
ColumnCapabilities capabilities,
|
ColumnCapabilities capabilities,
|
||||||
|
|
|
@ -21,6 +21,9 @@ package org.apache.druid.segment;
|
||||||
|
|
||||||
import org.apache.druid.segment.column.ColumnDescriptor;
|
import org.apache.druid.segment.column.ColumnDescriptor;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Processing related interface
|
* Processing related interface
|
||||||
*
|
*
|
||||||
|
@ -35,4 +38,13 @@ public interface DimensionMergerV9 extends DimensionMerger
|
||||||
* @return ColumnDescriptor that IndexMergerV9 will use to build a column.
|
* @return ColumnDescriptor that IndexMergerV9 will use to build a column.
|
||||||
*/
|
*/
|
||||||
ColumnDescriptor makeColumnDescriptor();
|
ColumnDescriptor makeColumnDescriptor();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Attaches the {@link DimensionMergerV9} of a "projection" parent column so that stuff like value dictionaries can
|
||||||
|
* be shared between parent and child
|
||||||
|
*/
|
||||||
|
default void attachParent(DimensionMergerV9 parent, List<IndexableAdapter> projectionAdapters) throws IOException
|
||||||
|
{
|
||||||
|
// do nothing
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -77,6 +77,7 @@ public class DoubleDimensionHandler implements DimensionHandler<Double, Double,
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DimensionMergerV9 makeMerger(
|
public DimensionMergerV9 makeMerger(
|
||||||
|
String outputName,
|
||||||
IndexSpec indexSpec,
|
IndexSpec indexSpec,
|
||||||
SegmentWriteOutMedium segmentWriteOutMedium,
|
SegmentWriteOutMedium segmentWriteOutMedium,
|
||||||
ColumnCapabilities capabilities,
|
ColumnCapabilities capabilities,
|
||||||
|
@ -85,7 +86,7 @@ public class DoubleDimensionHandler implements DimensionHandler<Double, Double,
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return new DoubleDimensionMergerV9(
|
return new DoubleDimensionMergerV9(
|
||||||
dimensionName,
|
outputName,
|
||||||
indexSpec,
|
indexSpec,
|
||||||
segmentWriteOutMedium
|
segmentWriteOutMedium
|
||||||
);
|
);
|
||||||
|
|
|
@ -27,15 +27,15 @@ import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
|
||||||
public class DoubleDimensionMergerV9 extends NumericDimensionMergerV9
|
public class DoubleDimensionMergerV9 extends NumericDimensionMergerV9
|
||||||
{
|
{
|
||||||
|
|
||||||
DoubleDimensionMergerV9(String dimensionName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium)
|
DoubleDimensionMergerV9(String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium)
|
||||||
{
|
{
|
||||||
super(dimensionName, indexSpec, segmentWriteOutMedium);
|
super(outputName, indexSpec, segmentWriteOutMedium);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
GenericColumnSerializer setupEncodedValueWriter()
|
GenericColumnSerializer setupEncodedValueWriter()
|
||||||
{
|
{
|
||||||
return IndexMergerV9.createDoubleColumnSerializer(segmentWriteOutMedium, dimensionName, indexSpec);
|
return IndexMergerV9.createDoubleColumnSerializer(segmentWriteOutMedium, outputName, indexSpec);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -77,6 +77,7 @@ public class FloatDimensionHandler implements DimensionHandler<Float, Float, Flo
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DimensionMergerV9 makeMerger(
|
public DimensionMergerV9 makeMerger(
|
||||||
|
String outputName,
|
||||||
IndexSpec indexSpec,
|
IndexSpec indexSpec,
|
||||||
SegmentWriteOutMedium segmentWriteOutMedium,
|
SegmentWriteOutMedium segmentWriteOutMedium,
|
||||||
ColumnCapabilities capabilities,
|
ColumnCapabilities capabilities,
|
||||||
|
@ -85,7 +86,7 @@ public class FloatDimensionHandler implements DimensionHandler<Float, Float, Flo
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return new FloatDimensionMergerV9(
|
return new FloatDimensionMergerV9(
|
||||||
dimensionName,
|
outputName,
|
||||||
indexSpec,
|
indexSpec,
|
||||||
segmentWriteOutMedium
|
segmentWriteOutMedium
|
||||||
);
|
);
|
||||||
|
|
|
@ -27,15 +27,15 @@ import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
|
||||||
public class FloatDimensionMergerV9 extends NumericDimensionMergerV9
|
public class FloatDimensionMergerV9 extends NumericDimensionMergerV9
|
||||||
{
|
{
|
||||||
|
|
||||||
FloatDimensionMergerV9(String dimensionName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium)
|
FloatDimensionMergerV9(String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium)
|
||||||
{
|
{
|
||||||
super(dimensionName, indexSpec, segmentWriteOutMedium);
|
super(outputName, indexSpec, segmentWriteOutMedium);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
GenericColumnSerializer setupEncodedValueWriter()
|
GenericColumnSerializer setupEncodedValueWriter()
|
||||||
{
|
{
|
||||||
return IndexMergerV9.createFloatColumnSerializer(segmentWriteOutMedium, dimensionName, indexSpec);
|
return IndexMergerV9.createFloatColumnSerializer(segmentWriteOutMedium, outputName, indexSpec);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -48,6 +48,7 @@ import org.apache.druid.java.util.common.io.smoosh.Smoosh;
|
||||||
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
|
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
|
||||||
import org.apache.druid.java.util.common.logger.Logger;
|
import org.apache.druid.java.util.common.logger.Logger;
|
||||||
import org.apache.druid.java.util.emitter.EmittingLogger;
|
import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||||
|
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||||
import org.apache.druid.segment.column.ColumnBuilder;
|
import org.apache.druid.segment.column.ColumnBuilder;
|
||||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||||
import org.apache.druid.segment.column.ColumnConfig;
|
import org.apache.druid.segment.column.ColumnConfig;
|
||||||
|
@ -64,6 +65,7 @@ import org.apache.druid.segment.data.Indexed;
|
||||||
import org.apache.druid.segment.data.IndexedIterable;
|
import org.apache.druid.segment.data.IndexedIterable;
|
||||||
import org.apache.druid.segment.data.ListIndexed;
|
import org.apache.druid.segment.data.ListIndexed;
|
||||||
import org.apache.druid.segment.data.VSizeColumnarMultiInts;
|
import org.apache.druid.segment.data.VSizeColumnarMultiInts;
|
||||||
|
import org.apache.druid.segment.projections.Projections;
|
||||||
import org.apache.druid.segment.serde.ComplexColumnPartSupplier;
|
import org.apache.druid.segment.serde.ComplexColumnPartSupplier;
|
||||||
import org.apache.druid.segment.serde.FloatNumericColumnSupplier;
|
import org.apache.druid.segment.serde.FloatNumericColumnSupplier;
|
||||||
import org.apache.druid.segment.serde.LongNumericColumnSupplier;
|
import org.apache.druid.segment.serde.LongNumericColumnSupplier;
|
||||||
|
@ -621,6 +623,7 @@ public class IndexIO
|
||||||
mapper,
|
mapper,
|
||||||
timeBuffer,
|
timeBuffer,
|
||||||
smooshedFiles,
|
smooshedFiles,
|
||||||
|
null,
|
||||||
loadFailed
|
loadFailed
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -643,6 +646,22 @@ public class IndexIO
|
||||||
smooshedFiles,
|
smooshedFiles,
|
||||||
loadFailed
|
loadFailed
|
||||||
);
|
);
|
||||||
|
final Map<String, Map<String, Supplier<ColumnHolder>>> projectionsColumns = new LinkedHashMap<>();
|
||||||
|
final Metadata metadata = getMetdata(smooshedFiles, mapper, inDir);
|
||||||
|
if (metadata != null && metadata.getProjections() != null) {
|
||||||
|
for (AggregateProjectionMetadata projectionSpec : metadata.getProjections()) {
|
||||||
|
final Map<String, Supplier<ColumnHolder>> projectionColumns = readProjectionColumns(
|
||||||
|
mapper,
|
||||||
|
loadFailed,
|
||||||
|
projectionSpec,
|
||||||
|
smooshedFiles,
|
||||||
|
columns,
|
||||||
|
dataInterval
|
||||||
|
);
|
||||||
|
|
||||||
|
projectionsColumns.put(projectionSpec.getSchema().getName(), projectionColumns);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
final QueryableIndex index = new SimpleQueryableIndex(
|
final QueryableIndex index = new SimpleQueryableIndex(
|
||||||
dataInterval,
|
dataInterval,
|
||||||
|
@ -650,30 +669,15 @@ public class IndexIO
|
||||||
segmentBitmapSerdeFactory.getBitmapFactory(),
|
segmentBitmapSerdeFactory.getBitmapFactory(),
|
||||||
columns,
|
columns,
|
||||||
smooshedFiles,
|
smooshedFiles,
|
||||||
lazy
|
lazy,
|
||||||
|
metadata,
|
||||||
|
projectionsColumns
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public Metadata getMetadata()
|
public Metadata getMetadata()
|
||||||
{
|
{
|
||||||
try {
|
return getMetdata(smooshedFiles, mapper, inDir);
|
||||||
ByteBuffer metadataBB = smooshedFiles.mapFile("metadata.drd");
|
|
||||||
if (metadataBB != null) {
|
|
||||||
return mapper.readValue(
|
|
||||||
SERIALIZER_UTILS.readBytes(metadataBB, metadataBB.remaining()),
|
|
||||||
Metadata.class
|
|
||||||
);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
catch (JsonParseException | JsonMappingException ex) {
|
|
||||||
// Any jackson deserialization errors are ignored e.g. if metadata contains some aggregator which
|
|
||||||
// is no longer supported then it is OK to not use the metadata instead of failing segment loading
|
|
||||||
log.warn(ex, "Failed to load metadata for segment [%s]", inDir);
|
|
||||||
}
|
|
||||||
catch (IOException ex) {
|
|
||||||
log.warn(ex, "Failed to read metadata for segment [%s]", inDir);
|
|
||||||
}
|
|
||||||
return null;
|
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -682,6 +686,87 @@ public class IndexIO
|
||||||
return index;
|
return index;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private Map<String, Supplier<ColumnHolder>> readProjectionColumns(
|
||||||
|
ObjectMapper mapper,
|
||||||
|
SegmentLazyLoadFailCallback loadFailed,
|
||||||
|
AggregateProjectionMetadata projectionSpec,
|
||||||
|
SmooshedFileMapper smooshedFiles,
|
||||||
|
Map<String, Supplier<ColumnHolder>> columns,
|
||||||
|
Interval dataInterval
|
||||||
|
) throws IOException
|
||||||
|
{
|
||||||
|
final Map<String, Supplier<ColumnHolder>> projectionColumns = new LinkedHashMap<>();
|
||||||
|
for (String groupingColumn : projectionSpec.getSchema().getGroupingColumns()) {
|
||||||
|
final String smooshName = Projections.getProjectionSmooshV9FileName(projectionSpec, groupingColumn);
|
||||||
|
final ByteBuffer colBuffer = smooshedFiles.mapFile(smooshName);
|
||||||
|
|
||||||
|
final ColumnHolder parentColumn;
|
||||||
|
if (columns.containsKey(groupingColumn)) {
|
||||||
|
parentColumn = columns.get(groupingColumn).get();
|
||||||
|
} else {
|
||||||
|
parentColumn = null;
|
||||||
|
}
|
||||||
|
registerColumnHolder(
|
||||||
|
true,
|
||||||
|
projectionColumns,
|
||||||
|
groupingColumn,
|
||||||
|
mapper,
|
||||||
|
colBuffer,
|
||||||
|
smooshedFiles,
|
||||||
|
parentColumn,
|
||||||
|
loadFailed
|
||||||
|
);
|
||||||
|
|
||||||
|
if (groupingColumn.equals(projectionSpec.getSchema().getTimeColumnName())) {
|
||||||
|
projectionColumns.put(ColumnHolder.TIME_COLUMN_NAME, projectionColumns.get(groupingColumn));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
for (AggregatorFactory aggregator : projectionSpec.getSchema().getAggregators()) {
|
||||||
|
final String smooshName = Projections.getProjectionSmooshV9FileName(projectionSpec, aggregator.getName());
|
||||||
|
final ByteBuffer aggBuffer = smooshedFiles.mapFile(smooshName);
|
||||||
|
registerColumnHolder(
|
||||||
|
true,
|
||||||
|
projectionColumns,
|
||||||
|
aggregator.getName(),
|
||||||
|
mapper,
|
||||||
|
aggBuffer,
|
||||||
|
smooshedFiles,
|
||||||
|
null,
|
||||||
|
loadFailed
|
||||||
|
);
|
||||||
|
}
|
||||||
|
if (projectionSpec.getSchema().getTimeColumnName() == null) {
|
||||||
|
projectionColumns.put(
|
||||||
|
ColumnHolder.TIME_COLUMN_NAME,
|
||||||
|
Projections.makeConstantTimeSupplier(projectionSpec.getNumRows(), dataInterval.getStartMillis())
|
||||||
|
);
|
||||||
|
}
|
||||||
|
return projectionColumns;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
private Metadata getMetdata(SmooshedFileMapper smooshedFiles, ObjectMapper mapper, File inDir)
|
||||||
|
{
|
||||||
|
try {
|
||||||
|
ByteBuffer metadataBB = smooshedFiles.mapFile("metadata.drd");
|
||||||
|
if (metadataBB != null) {
|
||||||
|
return mapper.readValue(
|
||||||
|
SERIALIZER_UTILS.readBytes(metadataBB, metadataBB.remaining()),
|
||||||
|
Metadata.class
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (JsonParseException | JsonMappingException ex) {
|
||||||
|
// Any jackson deserialization errors are ignored e.g. if metadata contains some aggregator which
|
||||||
|
// is no longer supported then it is OK to not use the metadata instead of failing segment loading
|
||||||
|
log.warn(ex, "Failed to load metadata for segment [%s]", inDir);
|
||||||
|
}
|
||||||
|
catch (IOException ex) {
|
||||||
|
log.warn(ex, "Failed to read metadata for segment [%s]", inDir);
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Return a list of columns that contains given inputs merged. The returned column names are in
|
* Return a list of columns that contains given inputs merged. The returned column names are in
|
||||||
* the original order that is used when this segment is created.
|
* the original order that is used when this segment is created.
|
||||||
|
@ -740,6 +825,7 @@ public class IndexIO
|
||||||
mapper,
|
mapper,
|
||||||
colBuffer,
|
colBuffer,
|
||||||
smooshedFiles,
|
smooshedFiles,
|
||||||
|
null,
|
||||||
loadFailed
|
loadFailed
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -752,6 +838,7 @@ public class IndexIO
|
||||||
ObjectMapper mapper,
|
ObjectMapper mapper,
|
||||||
ByteBuffer colBuffer,
|
ByteBuffer colBuffer,
|
||||||
SmooshedFileMapper smooshedFiles,
|
SmooshedFileMapper smooshedFiles,
|
||||||
|
@Nullable ColumnHolder parentColumn,
|
||||||
SegmentLazyLoadFailCallback loadFailed
|
SegmentLazyLoadFailCallback loadFailed
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
|
@ -767,7 +854,8 @@ public class IndexIO
|
||||||
internedColumnName,
|
internedColumnName,
|
||||||
mapper,
|
mapper,
|
||||||
colBuffer,
|
colBuffer,
|
||||||
smooshedFiles
|
smooshedFiles,
|
||||||
|
parentColumn
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (IOException | RuntimeException e) {
|
catch (IOException | RuntimeException e) {
|
||||||
|
@ -782,7 +870,8 @@ public class IndexIO
|
||||||
internedColumnName,
|
internedColumnName,
|
||||||
mapper,
|
mapper,
|
||||||
colBuffer,
|
colBuffer,
|
||||||
smooshedFiles
|
smooshedFiles,
|
||||||
|
parentColumn
|
||||||
);
|
);
|
||||||
columns.put(internedColumnName, () -> columnHolder);
|
columns.put(internedColumnName, () -> columnHolder);
|
||||||
}
|
}
|
||||||
|
@ -797,11 +886,12 @@ public class IndexIO
|
||||||
String columnName, // columnName is not used in this method, but used in tests.
|
String columnName, // columnName is not used in this method, but used in tests.
|
||||||
ObjectMapper mapper,
|
ObjectMapper mapper,
|
||||||
ByteBuffer byteBuffer,
|
ByteBuffer byteBuffer,
|
||||||
SmooshedFileMapper smooshedFiles
|
SmooshedFileMapper smooshedFiles,
|
||||||
|
@Nullable ColumnHolder parentColumn
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
ColumnDescriptor serde = mapper.readValue(SERIALIZER_UTILS.readString(byteBuffer), ColumnDescriptor.class);
|
ColumnDescriptor serde = mapper.readValue(SERIALIZER_UTILS.readString(byteBuffer), ColumnDescriptor.class);
|
||||||
return serde.read(byteBuffer, columnConfig, smooshedFiles);
|
return serde.read(byteBuffer, columnConfig, smooshedFiles, parentColumn);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -354,7 +354,7 @@ public interface IndexMerger
|
||||||
// type of column doesn't have any kind of special per-index encoding that needs to be converted to the "global"
|
// type of column doesn't have any kind of special per-index encoding that needs to be converted to the "global"
|
||||||
// encoding. E. g. it's always true for subclasses of NumericDimensionMergerV9.
|
// encoding. E. g. it's always true for subclasses of NumericDimensionMergerV9.
|
||||||
//noinspection ObjectEquality
|
//noinspection ObjectEquality
|
||||||
anySelectorChanged |= convertedDimensionSelector != sourceDimensionSelector;
|
anySelectorChanged = anySelectorChanged || convertedDimensionSelector != sourceDimensionSelector;
|
||||||
|
|
||||||
convertedMarkedDimensionSelectors[i] = mergers.get(i).convertSortedSegmentRowValuesToMergedRowValues(
|
convertedMarkedDimensionSelectors[i] = mergers.get(i).convertSortedSegmentRowValuesToMergedRowValues(
|
||||||
indexNumber,
|
indexNumber,
|
||||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
|
import com.google.common.collect.Maps;
|
||||||
import com.google.common.io.Files;
|
import com.google.common.io.Files;
|
||||||
import com.google.common.primitives.Ints;
|
import com.google.common.primitives.Ints;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
|
@ -51,6 +52,7 @@ import org.apache.druid.segment.incremental.IncrementalIndex;
|
||||||
import org.apache.druid.segment.incremental.IncrementalIndexAdapter;
|
import org.apache.druid.segment.incremental.IncrementalIndexAdapter;
|
||||||
import org.apache.druid.segment.loading.MMappedQueryableSegmentizerFactory;
|
import org.apache.druid.segment.loading.MMappedQueryableSegmentizerFactory;
|
||||||
import org.apache.druid.segment.loading.SegmentizerFactory;
|
import org.apache.druid.segment.loading.SegmentizerFactory;
|
||||||
|
import org.apache.druid.segment.projections.Projections;
|
||||||
import org.apache.druid.segment.serde.ColumnPartSerde;
|
import org.apache.druid.segment.serde.ColumnPartSerde;
|
||||||
import org.apache.druid.segment.serde.ComplexColumnPartSerde;
|
import org.apache.druid.segment.serde.ComplexColumnPartSerde;
|
||||||
import org.apache.druid.segment.serde.ComplexMetricSerde;
|
import org.apache.druid.segment.serde.ComplexMetricSerde;
|
||||||
|
@ -64,6 +66,7 @@ import org.apache.druid.segment.serde.LongNumericColumnPartSerdeV2;
|
||||||
import org.apache.druid.segment.serde.NullColumnPartSerde;
|
import org.apache.druid.segment.serde.NullColumnPartSerde;
|
||||||
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
|
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
|
||||||
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
|
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
|
||||||
|
import org.apache.druid.utils.CollectionUtils;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
@ -138,7 +141,6 @@ public class IndexMergerV9 implements IndexMerger
|
||||||
final DimensionsSpecInspector dimensionsSpecInspector,
|
final DimensionsSpecInspector dimensionsSpecInspector,
|
||||||
final List<String> mergedMetrics,
|
final List<String> mergedMetrics,
|
||||||
final Function<List<TransformableRowIterator>, TimeAndDimsIterator> rowMergerFn,
|
final Function<List<TransformableRowIterator>, TimeAndDimsIterator> rowMergerFn,
|
||||||
final boolean fillRowNumConversions,
|
|
||||||
final IndexSpec indexSpec,
|
final IndexSpec indexSpec,
|
||||||
final @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
|
final @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
|
||||||
) throws IOException
|
) throws IOException
|
||||||
|
@ -154,7 +156,7 @@ public class IndexMergerV9 implements IndexMerger
|
||||||
.filter(dim -> !ColumnHolder.TIME_COLUMN_NAME.equals(dim))
|
.filter(dim -> !ColumnHolder.TIME_COLUMN_NAME.equals(dim))
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
|
|
||||||
final Metadata segmentMetadata;
|
Metadata segmentMetadata;
|
||||||
if (metricAggs != null) {
|
if (metricAggs != null) {
|
||||||
AggregatorFactory[] combiningMetricAggs = new AggregatorFactory[metricAggs.length];
|
AggregatorFactory[] combiningMetricAggs = new AggregatorFactory[metricAggs.length];
|
||||||
for (int i = 0; i < metricAggs.length; i++) {
|
for (int i = 0; i < metricAggs.length; i++) {
|
||||||
|
@ -215,18 +217,20 @@ public class IndexMergerV9 implements IndexMerger
|
||||||
mergeFormat(adapters, mergedDimensions, metricFormats, dimFormats);
|
mergeFormat(adapters, mergedDimensions, metricFormats, dimFormats);
|
||||||
|
|
||||||
final Map<String, DimensionHandler> handlers = makeDimensionHandlers(mergedDimensions, dimFormats);
|
final Map<String, DimensionHandler> handlers = makeDimensionHandlers(mergedDimensions, dimFormats);
|
||||||
|
final Map<String, DimensionMergerV9> mergersMap = Maps.newHashMapWithExpectedSize(mergedDimensions.size());
|
||||||
final List<DimensionMergerV9> mergers = new ArrayList<>();
|
final List<DimensionMergerV9> mergers = new ArrayList<>();
|
||||||
for (int i = 0; i < mergedDimensions.size(); i++) {
|
for (int i = 0; i < mergedDimensions.size(); i++) {
|
||||||
DimensionHandler handler = handlers.get(mergedDimensions.get(i));
|
DimensionHandler handler = handlers.get(mergedDimensions.get(i));
|
||||||
mergers.add(
|
DimensionMergerV9 merger = handler.makeMerger(
|
||||||
handler.makeMerger(
|
mergedDimensions.get(i),
|
||||||
indexSpec,
|
indexSpec,
|
||||||
segmentWriteOutMedium,
|
segmentWriteOutMedium,
|
||||||
dimFormats.get(i).toColumnCapabilities(),
|
dimFormats.get(i).toColumnCapabilities(),
|
||||||
progress,
|
progress,
|
||||||
closer
|
closer
|
||||||
)
|
|
||||||
);
|
);
|
||||||
|
mergers.add(merger);
|
||||||
|
mergersMap.put(mergedDimensions.get(i), merger);
|
||||||
}
|
}
|
||||||
|
|
||||||
/************* Setup Dim Conversions **************/
|
/************* Setup Dim Conversions **************/
|
||||||
|
@ -255,8 +259,7 @@ public class IndexMergerV9 implements IndexMerger
|
||||||
timeAndDimsIterator,
|
timeAndDimsIterator,
|
||||||
timeWriter,
|
timeWriter,
|
||||||
metricWriters,
|
metricWriters,
|
||||||
mergers,
|
mergers
|
||||||
fillRowNumConversions
|
|
||||||
);
|
);
|
||||||
|
|
||||||
/************ Create Inverted Indexes and Finalize Build Columns *************/
|
/************ Create Inverted Indexes and Finalize Build Columns *************/
|
||||||
|
@ -291,6 +294,20 @@ public class IndexMergerV9 implements IndexMerger
|
||||||
|
|
||||||
progress.stopSection(section);
|
progress.stopSection(section);
|
||||||
|
|
||||||
|
if (segmentMetadata != null && !CollectionUtils.isNullOrEmpty(segmentMetadata.getProjections())) {
|
||||||
|
segmentMetadata = makeProjections(
|
||||||
|
v9Smoosher,
|
||||||
|
segmentMetadata.getProjections(),
|
||||||
|
adapters,
|
||||||
|
indexSpec,
|
||||||
|
segmentWriteOutMedium,
|
||||||
|
progress,
|
||||||
|
closer,
|
||||||
|
mergersMap,
|
||||||
|
segmentMetadata
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
/************* Make index.drd & metadata.drd files **************/
|
/************* Make index.drd & metadata.drd files **************/
|
||||||
progress.progress();
|
progress.progress();
|
||||||
makeIndexBinary(
|
makeIndexBinary(
|
||||||
|
@ -332,6 +349,193 @@ public class IndexMergerV9 implements IndexMerger
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private Metadata makeProjections(
|
||||||
|
final FileSmoosher smoosher,
|
||||||
|
final List<AggregateProjectionMetadata> projections,
|
||||||
|
final List<IndexableAdapter> adapters,
|
||||||
|
final IndexSpec indexSpec,
|
||||||
|
final SegmentWriteOutMedium segmentWriteOutMedium,
|
||||||
|
final ProgressIndicator progress,
|
||||||
|
final Closer closer,
|
||||||
|
final Map<String, DimensionMergerV9> parentMergers,
|
||||||
|
final Metadata segmentMetadata
|
||||||
|
) throws IOException
|
||||||
|
{
|
||||||
|
final List<AggregateProjectionMetadata> projectionMetadata = Lists.newArrayListWithCapacity(projections.size());
|
||||||
|
for (AggregateProjectionMetadata spec : projections) {
|
||||||
|
final List<IndexableAdapter> projectionAdapters = Lists.newArrayListWithCapacity(adapters.size());
|
||||||
|
final AggregateProjectionMetadata.Schema projectionSchema = spec.getSchema();
|
||||||
|
for (IndexableAdapter adapter : adapters) {
|
||||||
|
projectionAdapters.add(adapter.getProjectionAdapter(projectionSchema.getName()));
|
||||||
|
}
|
||||||
|
// we can use the first adapter to get the dimensions and metrics because the projection schema should be
|
||||||
|
// identical across all segments. This is validated by segment metadata merging
|
||||||
|
final List<String> dimensions = projectionAdapters.get(0).getDimensionNames(false);
|
||||||
|
final List<String> metrics = Arrays.stream(projectionSchema.getAggregators())
|
||||||
|
.map(AggregatorFactory::getName)
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
|
||||||
|
|
||||||
|
final List<DimensionMergerV9> mergers = new ArrayList<>();
|
||||||
|
final Map<String, ColumnFormat> columnFormats = Maps.newLinkedHashMapWithExpectedSize(dimensions.size() + metrics.size());
|
||||||
|
|
||||||
|
for (String dimension : dimensions) {
|
||||||
|
final ColumnFormat dimensionFormat = projectionAdapters.get(0).getFormat(dimension);
|
||||||
|
columnFormats.put(dimension, dimensionFormat);
|
||||||
|
DimensionHandler handler = dimensionFormat.getColumnHandler(dimension);
|
||||||
|
DimensionMergerV9 merger = handler.makeMerger(
|
||||||
|
Projections.getProjectionSmooshV9FileName(spec, dimension),
|
||||||
|
indexSpec,
|
||||||
|
segmentWriteOutMedium,
|
||||||
|
dimensionFormat.toColumnCapabilities(),
|
||||||
|
progress,
|
||||||
|
closer
|
||||||
|
);
|
||||||
|
if (parentMergers.containsKey(dimension)) {
|
||||||
|
merger.attachParent(parentMergers.get(dimension), projectionAdapters);
|
||||||
|
} else {
|
||||||
|
merger.writeMergedValueDictionary(projectionAdapters);
|
||||||
|
}
|
||||||
|
mergers.add(merger);
|
||||||
|
}
|
||||||
|
for (String metric : metrics) {
|
||||||
|
columnFormats.put(metric, projectionAdapters.get(0).getFormat(metric));
|
||||||
|
}
|
||||||
|
|
||||||
|
final GenericColumnSerializer timeWriter;
|
||||||
|
if (projectionSchema.getTimeColumnName() != null) {
|
||||||
|
timeWriter = setupTimeWriter(segmentWriteOutMedium, indexSpec);
|
||||||
|
} else {
|
||||||
|
timeWriter = null;
|
||||||
|
}
|
||||||
|
final ArrayList<GenericColumnSerializer> metricWriters =
|
||||||
|
setupMetricsWriters(
|
||||||
|
segmentWriteOutMedium,
|
||||||
|
metrics,
|
||||||
|
columnFormats,
|
||||||
|
indexSpec,
|
||||||
|
Projections.getProjectionSmooshV9Prefix(spec)
|
||||||
|
);
|
||||||
|
|
||||||
|
Function<List<TransformableRowIterator>, TimeAndDimsIterator> rowMergerFn =
|
||||||
|
rowIterators -> new RowCombiningTimeAndDimsIterator(rowIterators, projectionSchema.getAggregators(), metrics);
|
||||||
|
|
||||||
|
List<TransformableRowIterator> perIndexRowIterators = Lists.newArrayListWithCapacity(projectionAdapters.size());
|
||||||
|
for (int i = 0; i < projectionAdapters.size(); ++i) {
|
||||||
|
final IndexableAdapter adapter = projectionAdapters.get(i);
|
||||||
|
TransformableRowIterator target = adapter.getRows();
|
||||||
|
perIndexRowIterators.add(IndexMerger.toMergedIndexRowIterator(target, i, mergers));
|
||||||
|
}
|
||||||
|
final TimeAndDimsIterator timeAndDimsIterator = rowMergerFn.apply(perIndexRowIterators);
|
||||||
|
closer.register(timeAndDimsIterator);
|
||||||
|
|
||||||
|
int rowCount = 0;
|
||||||
|
List<IntBuffer> rowNumConversions = new ArrayList<>(projectionAdapters.size());
|
||||||
|
for (IndexableAdapter adapter : projectionAdapters) {
|
||||||
|
int[] arr = new int[adapter.getNumRows()];
|
||||||
|
Arrays.fill(arr, INVALID_ROW);
|
||||||
|
rowNumConversions.add(IntBuffer.wrap(arr));
|
||||||
|
}
|
||||||
|
|
||||||
|
final String section = "walk through and merge projection[" + projectionSchema.getName() + "] rows";
|
||||||
|
progress.startSection(section);
|
||||||
|
long startTime = System.currentTimeMillis();
|
||||||
|
long time = startTime;
|
||||||
|
while (timeAndDimsIterator.moveToNext()) {
|
||||||
|
progress.progress();
|
||||||
|
TimeAndDimsPointer timeAndDims = timeAndDimsIterator.getPointer();
|
||||||
|
if (timeWriter != null) {
|
||||||
|
timeWriter.serialize(timeAndDims.timestampSelector);
|
||||||
|
}
|
||||||
|
|
||||||
|
for (int metricIndex = 0; metricIndex < timeAndDims.getNumMetrics(); metricIndex++) {
|
||||||
|
metricWriters.get(metricIndex).serialize(timeAndDims.getMetricSelector(metricIndex));
|
||||||
|
}
|
||||||
|
|
||||||
|
for (int dimIndex = 0; dimIndex < timeAndDims.getNumDimensions(); dimIndex++) {
|
||||||
|
DimensionMergerV9 merger = mergers.get(dimIndex);
|
||||||
|
if (merger.hasOnlyNulls()) {
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
merger.processMergedRow(timeAndDims.getDimensionSelector(dimIndex));
|
||||||
|
}
|
||||||
|
|
||||||
|
RowCombiningTimeAndDimsIterator comprisedRows = (RowCombiningTimeAndDimsIterator) timeAndDimsIterator;
|
||||||
|
|
||||||
|
for (int originalIteratorIndex = comprisedRows.nextCurrentlyCombinedOriginalIteratorIndex(0);
|
||||||
|
originalIteratorIndex >= 0;
|
||||||
|
originalIteratorIndex =
|
||||||
|
comprisedRows.nextCurrentlyCombinedOriginalIteratorIndex(originalIteratorIndex + 1)) {
|
||||||
|
|
||||||
|
IntBuffer conversionBuffer = rowNumConversions.get(originalIteratorIndex);
|
||||||
|
int minRowNum = comprisedRows.getMinCurrentlyCombinedRowNumByOriginalIteratorIndex(originalIteratorIndex);
|
||||||
|
int maxRowNum = comprisedRows.getMaxCurrentlyCombinedRowNumByOriginalIteratorIndex(originalIteratorIndex);
|
||||||
|
|
||||||
|
for (int rowNum = minRowNum; rowNum <= maxRowNum; rowNum++) {
|
||||||
|
while (conversionBuffer.position() < rowNum) {
|
||||||
|
conversionBuffer.put(INVALID_ROW);
|
||||||
|
}
|
||||||
|
conversionBuffer.put(rowCount);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if ((++rowCount % 500000) == 0) {
|
||||||
|
log.debug(
|
||||||
|
"walked 500,000/%d rows of projection[%s] in %,d millis.",
|
||||||
|
rowCount,
|
||||||
|
projectionSchema.getName(),
|
||||||
|
System.currentTimeMillis() - time
|
||||||
|
);
|
||||||
|
time = System.currentTimeMillis();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
for (IntBuffer rowNumConversion : rowNumConversions) {
|
||||||
|
rowNumConversion.rewind();
|
||||||
|
}
|
||||||
|
log.debug(
|
||||||
|
"completed walk through of %,d rows of projection[%s] in %,d millis.",
|
||||||
|
rowCount,
|
||||||
|
projectionSchema.getName(),
|
||||||
|
System.currentTimeMillis() - startTime
|
||||||
|
);
|
||||||
|
progress.stopSection(section);
|
||||||
|
|
||||||
|
final String section2 = "build projection[" + projectionSchema.getName() + "] inverted index and columns";
|
||||||
|
progress.startSection(section2);
|
||||||
|
if (projectionSchema.getTimeColumnName() != null) {
|
||||||
|
makeTimeColumn(
|
||||||
|
smoosher,
|
||||||
|
progress,
|
||||||
|
timeWriter,
|
||||||
|
indexSpec,
|
||||||
|
Projections.getProjectionSmooshV9FileName(spec, projectionSchema.getTimeColumnName())
|
||||||
|
);
|
||||||
|
}
|
||||||
|
makeMetricsColumns(
|
||||||
|
smoosher,
|
||||||
|
progress,
|
||||||
|
metrics,
|
||||||
|
columnFormats,
|
||||||
|
metricWriters,
|
||||||
|
indexSpec,
|
||||||
|
Projections.getProjectionSmooshV9Prefix(spec)
|
||||||
|
);
|
||||||
|
|
||||||
|
for (int i = 0; i < dimensions.size(); i++) {
|
||||||
|
final String dimension = dimensions.get(i);
|
||||||
|
DimensionMergerV9 merger = mergers.get(i);
|
||||||
|
merger.writeIndexes(rowNumConversions);
|
||||||
|
if (!merger.hasOnlyNulls()) {
|
||||||
|
ColumnDescriptor columnDesc = merger.makeColumnDescriptor();
|
||||||
|
makeColumn(smoosher, Projections.getProjectionSmooshV9FileName(spec, dimension), columnDesc);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
progress.stopSection(section2);
|
||||||
|
projectionMetadata.add(new AggregateProjectionMetadata(projectionSchema, rowCount));
|
||||||
|
}
|
||||||
|
return segmentMetadata.withProjections(projectionMetadata);
|
||||||
|
}
|
||||||
|
|
||||||
private void makeIndexBinary(
|
private void makeIndexBinary(
|
||||||
final FileSmoosher v9Smoosher,
|
final FileSmoosher v9Smoosher,
|
||||||
final List<IndexableAdapter> adapters,
|
final List<IndexableAdapter> adapters,
|
||||||
|
@ -348,7 +552,7 @@ public class IndexMergerV9 implements IndexMerger
|
||||||
columnSet.addAll(mergedMetrics);
|
columnSet.addAll(mergedMetrics);
|
||||||
Preconditions.checkState(
|
Preconditions.checkState(
|
||||||
columnSet.size() == mergedDimensions.size() + mergedMetrics.size(),
|
columnSet.size() == mergedDimensions.size() + mergedMetrics.size(),
|
||||||
"column names are not unique in dims%s and mets%s",
|
"column names are not unique in dims[%s] and mets[%s]",
|
||||||
mergedDimensions,
|
mergedDimensions,
|
||||||
mergedMetrics
|
mergedMetrics
|
||||||
);
|
);
|
||||||
|
@ -444,6 +648,18 @@ public class IndexMergerV9 implements IndexMerger
|
||||||
final List<GenericColumnSerializer> metWriters,
|
final List<GenericColumnSerializer> metWriters,
|
||||||
final IndexSpec indexSpec
|
final IndexSpec indexSpec
|
||||||
) throws IOException
|
) throws IOException
|
||||||
|
{
|
||||||
|
makeMetricsColumns(v9Smoosher, progress, mergedMetrics, metricsTypes, metWriters, indexSpec, "");
|
||||||
|
}
|
||||||
|
private void makeMetricsColumns(
|
||||||
|
final FileSmoosher v9Smoosher,
|
||||||
|
final ProgressIndicator progress,
|
||||||
|
final List<String> mergedMetrics,
|
||||||
|
final Map<String, ColumnFormat> metricsTypes,
|
||||||
|
final List<GenericColumnSerializer> metWriters,
|
||||||
|
final IndexSpec indexSpec,
|
||||||
|
final String namePrefix
|
||||||
|
) throws IOException
|
||||||
{
|
{
|
||||||
final String section = "make metric columns";
|
final String section = "make metric columns";
|
||||||
progress.startSection(section);
|
progress.startSection(section);
|
||||||
|
@ -483,8 +699,9 @@ public class IndexMergerV9 implements IndexMerger
|
||||||
default:
|
default:
|
||||||
throw new ISE("Unknown type[%s]", type);
|
throw new ISE("Unknown type[%s]", type);
|
||||||
}
|
}
|
||||||
makeColumn(v9Smoosher, metric, builder.build());
|
final String columnName = namePrefix + metric;
|
||||||
log.debug("Completed metric column[%s] in %,d millis.", metric, System.currentTimeMillis() - metricStartTime);
|
makeColumn(v9Smoosher, columnName, builder.build());
|
||||||
|
log.debug("Completed metric column[%s] in %,d millis.", columnName, System.currentTimeMillis() - metricStartTime);
|
||||||
}
|
}
|
||||||
log.debug("Completed metric columns in %,d millis.", System.currentTimeMillis() - startTime);
|
log.debug("Completed metric columns in %,d millis.", System.currentTimeMillis() - startTime);
|
||||||
progress.stopSection(section);
|
progress.stopSection(section);
|
||||||
|
@ -547,6 +764,17 @@ public class IndexMergerV9 implements IndexMerger
|
||||||
final GenericColumnSerializer timeWriter,
|
final GenericColumnSerializer timeWriter,
|
||||||
final IndexSpec indexSpec
|
final IndexSpec indexSpec
|
||||||
) throws IOException
|
) throws IOException
|
||||||
|
{
|
||||||
|
makeTimeColumn(v9Smoosher, progress, timeWriter, indexSpec, ColumnHolder.TIME_COLUMN_NAME);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void makeTimeColumn(
|
||||||
|
final FileSmoosher v9Smoosher,
|
||||||
|
final ProgressIndicator progress,
|
||||||
|
final GenericColumnSerializer timeWriter,
|
||||||
|
final IndexSpec indexSpec,
|
||||||
|
final String name
|
||||||
|
) throws IOException
|
||||||
{
|
{
|
||||||
final String section = "make time column";
|
final String section = "make time column";
|
||||||
progress.startSection(section);
|
progress.startSection(section);
|
||||||
|
@ -557,7 +785,7 @@ public class IndexMergerV9 implements IndexMerger
|
||||||
.setValueType(ValueType.LONG)
|
.setValueType(ValueType.LONG)
|
||||||
.addSerde(createLongColumnPartSerde(timeWriter, indexSpec))
|
.addSerde(createLongColumnPartSerde(timeWriter, indexSpec))
|
||||||
.build();
|
.build();
|
||||||
makeColumn(v9Smoosher, ColumnHolder.TIME_COLUMN_NAME, serdeficator);
|
makeColumn(v9Smoosher, name, serdeficator);
|
||||||
log.debug("Completed time column in %,d millis.", System.currentTimeMillis() - startTime);
|
log.debug("Completed time column in %,d millis.", System.currentTimeMillis() - startTime);
|
||||||
progress.stopSection(section);
|
progress.stopSection(section);
|
||||||
}
|
}
|
||||||
|
@ -601,23 +829,19 @@ public class IndexMergerV9 implements IndexMerger
|
||||||
final TimeAndDimsIterator timeAndDimsIterator,
|
final TimeAndDimsIterator timeAndDimsIterator,
|
||||||
final GenericColumnSerializer timeWriter,
|
final GenericColumnSerializer timeWriter,
|
||||||
final ArrayList<GenericColumnSerializer> metricWriters,
|
final ArrayList<GenericColumnSerializer> metricWriters,
|
||||||
final List<DimensionMergerV9> mergers,
|
final List<DimensionMergerV9> mergers
|
||||||
final boolean fillRowNumConversions
|
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
final String section = "walk through and merge rows";
|
final String section = "walk through and merge rows";
|
||||||
progress.startSection(section);
|
progress.startSection(section);
|
||||||
long startTime = System.currentTimeMillis();
|
long startTime = System.currentTimeMillis();
|
||||||
|
|
||||||
List<IntBuffer> rowNumConversions = null;
|
|
||||||
int rowCount = 0;
|
int rowCount = 0;
|
||||||
if (fillRowNumConversions) {
|
List<IntBuffer> rowNumConversions = new ArrayList<>(adapters.size());
|
||||||
rowNumConversions = new ArrayList<>(adapters.size());
|
for (IndexableAdapter adapter : adapters) {
|
||||||
for (IndexableAdapter adapter : adapters) {
|
int[] arr = new int[adapter.getNumRows()];
|
||||||
int[] arr = new int[adapter.getNumRows()];
|
Arrays.fill(arr, INVALID_ROW);
|
||||||
Arrays.fill(arr, INVALID_ROW);
|
rowNumConversions.add(IntBuffer.wrap(arr));
|
||||||
rowNumConversions.add(IntBuffer.wrap(arr));
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
long time = System.currentTimeMillis();
|
long time = System.currentTimeMillis();
|
||||||
|
@ -656,9 +880,7 @@ public class IndexMergerV9 implements IndexMerger
|
||||||
}
|
}
|
||||||
conversionBuffer.put(rowCount);
|
conversionBuffer.put(rowCount);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
} else if (timeAndDimsIterator instanceof MergingRowIterator) {
|
} else if (timeAndDimsIterator instanceof MergingRowIterator) {
|
||||||
RowPointer rowPointer = (RowPointer) timeAndDims;
|
RowPointer rowPointer = (RowPointer) timeAndDims;
|
||||||
IntBuffer conversionBuffer = rowNumConversions.get(rowPointer.getIndexNum());
|
IntBuffer conversionBuffer = rowNumConversions.get(rowPointer.getIndexNum());
|
||||||
|
@ -668,11 +890,9 @@ public class IndexMergerV9 implements IndexMerger
|
||||||
}
|
}
|
||||||
conversionBuffer.put(rowCount);
|
conversionBuffer.put(rowCount);
|
||||||
} else {
|
} else {
|
||||||
if (fillRowNumConversions) {
|
throw new IllegalStateException(
|
||||||
throw new IllegalStateException(
|
"Filling row num conversions is supported only with RowCombining and Merging iterators"
|
||||||
"Filling row num conversions is supported only with RowCombining and Merging iterators"
|
);
|
||||||
);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if ((++rowCount % 500000) == 0) {
|
if ((++rowCount % 500000) == 0) {
|
||||||
|
@ -680,10 +900,8 @@ public class IndexMergerV9 implements IndexMerger
|
||||||
time = System.currentTimeMillis();
|
time = System.currentTimeMillis();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (rowNumConversions != null) {
|
for (IntBuffer rowNumConversion : rowNumConversions) {
|
||||||
for (IntBuffer rowNumConversion : rowNumConversions) {
|
rowNumConversion.rewind();
|
||||||
rowNumConversion.rewind();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
log.debug("completed walk through of %,d rows in %,d millis.", rowCount, System.currentTimeMillis() - startTime);
|
log.debug("completed walk through of %,d rows in %,d millis.", rowCount, System.currentTimeMillis() - startTime);
|
||||||
progress.stopSection(section);
|
progress.stopSection(section);
|
||||||
|
@ -711,28 +929,40 @@ public class IndexMergerV9 implements IndexMerger
|
||||||
final Map<String, ColumnFormat> metricsTypes,
|
final Map<String, ColumnFormat> metricsTypes,
|
||||||
final IndexSpec indexSpec
|
final IndexSpec indexSpec
|
||||||
) throws IOException
|
) throws IOException
|
||||||
|
{
|
||||||
|
return setupMetricsWriters(segmentWriteOutMedium, mergedMetrics, metricsTypes, indexSpec, "");
|
||||||
|
}
|
||||||
|
|
||||||
|
private ArrayList<GenericColumnSerializer> setupMetricsWriters(
|
||||||
|
final SegmentWriteOutMedium segmentWriteOutMedium,
|
||||||
|
final List<String> mergedMetrics,
|
||||||
|
final Map<String, ColumnFormat> metricsTypes,
|
||||||
|
final IndexSpec indexSpec,
|
||||||
|
final String prefix
|
||||||
|
) throws IOException
|
||||||
{
|
{
|
||||||
ArrayList<GenericColumnSerializer> metWriters = Lists.newArrayListWithCapacity(mergedMetrics.size());
|
ArrayList<GenericColumnSerializer> metWriters = Lists.newArrayListWithCapacity(mergedMetrics.size());
|
||||||
|
|
||||||
for (String metric : mergedMetrics) {
|
for (String metric : mergedMetrics) {
|
||||||
TypeSignature<ValueType> type = metricsTypes.get(metric).getLogicalType();
|
TypeSignature<ValueType> type = metricsTypes.get(metric).getLogicalType();
|
||||||
|
final String outputName = prefix + metric;
|
||||||
GenericColumnSerializer writer;
|
GenericColumnSerializer writer;
|
||||||
switch (type.getType()) {
|
switch (type.getType()) {
|
||||||
case LONG:
|
case LONG:
|
||||||
writer = createLongColumnSerializer(segmentWriteOutMedium, metric, indexSpec);
|
writer = createLongColumnSerializer(segmentWriteOutMedium, outputName, indexSpec);
|
||||||
break;
|
break;
|
||||||
case FLOAT:
|
case FLOAT:
|
||||||
writer = createFloatColumnSerializer(segmentWriteOutMedium, metric, indexSpec);
|
writer = createFloatColumnSerializer(segmentWriteOutMedium, outputName, indexSpec);
|
||||||
break;
|
break;
|
||||||
case DOUBLE:
|
case DOUBLE:
|
||||||
writer = createDoubleColumnSerializer(segmentWriteOutMedium, metric, indexSpec);
|
writer = createDoubleColumnSerializer(segmentWriteOutMedium, outputName, indexSpec);
|
||||||
break;
|
break;
|
||||||
case COMPLEX:
|
case COMPLEX:
|
||||||
ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(type.getComplexTypeName());
|
ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(type.getComplexTypeName());
|
||||||
if (serde == null) {
|
if (serde == null) {
|
||||||
throw new ISE("Unknown type[%s]", type.getComplexTypeName());
|
throw new ISE("Unknown type[%s]", type.getComplexTypeName());
|
||||||
}
|
}
|
||||||
writer = serde.getSerializer(segmentWriteOutMedium, metric, indexSpec);
|
writer = serde.getSerializer(segmentWriteOutMedium, outputName, indexSpec);
|
||||||
break;
|
break;
|
||||||
default:
|
default:
|
||||||
throw new ISE("Unknown type[%s]", type);
|
throw new ISE("Unknown type[%s]", type);
|
||||||
|
@ -891,7 +1121,7 @@ public class IndexMergerV9 implements IndexMerger
|
||||||
|
|
||||||
FileUtils.mkdirp(outDir);
|
FileUtils.mkdirp(outDir);
|
||||||
|
|
||||||
log.debug("Starting persist for interval[%s], rows[%,d]", dataInterval, index.size());
|
log.debug("Starting persist for interval[%s], rows[%,d]", dataInterval, index.numRows());
|
||||||
return multiphaseMerge(
|
return multiphaseMerge(
|
||||||
Collections.singletonList(
|
Collections.singletonList(
|
||||||
new IncrementalIndexAdapter(
|
new IncrementalIndexAdapter(
|
||||||
|
@ -1179,7 +1409,6 @@ public class IndexMergerV9 implements IndexMerger
|
||||||
new DimensionsSpecInspector(storeEmptyColumns, dimensionsSpec),
|
new DimensionsSpecInspector(storeEmptyColumns, dimensionsSpec),
|
||||||
mergedMetrics,
|
mergedMetrics,
|
||||||
rowMergerFn,
|
rowMergerFn,
|
||||||
true,
|
|
||||||
indexSpec,
|
indexSpec,
|
||||||
segmentWriteOutMediumFactory
|
segmentWriteOutMediumFactory
|
||||||
);
|
);
|
||||||
|
|
|
@ -64,6 +64,8 @@ public interface IndexableAdapter
|
||||||
|
|
||||||
TransformableRowIterator getRows();
|
TransformableRowIterator getRows();
|
||||||
|
|
||||||
|
IndexableAdapter getProjectionAdapter(String projection);
|
||||||
|
|
||||||
BitmapValues getBitmapValues(String dimension, int dictId);
|
BitmapValues getBitmapValues(String dimension, int dictId);
|
||||||
|
|
||||||
ColumnCapabilities getCapabilities(String column);
|
ColumnCapabilities getCapabilities(String column);
|
||||||
|
|
|
@ -77,6 +77,7 @@ public class LongDimensionHandler implements DimensionHandler<Long, Long, Long>
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DimensionMergerV9 makeMerger(
|
public DimensionMergerV9 makeMerger(
|
||||||
|
String outputName,
|
||||||
IndexSpec indexSpec,
|
IndexSpec indexSpec,
|
||||||
SegmentWriteOutMedium segmentWriteOutMedium,
|
SegmentWriteOutMedium segmentWriteOutMedium,
|
||||||
ColumnCapabilities capabilities,
|
ColumnCapabilities capabilities,
|
||||||
|
@ -85,7 +86,7 @@ public class LongDimensionHandler implements DimensionHandler<Long, Long, Long>
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return new LongDimensionMergerV9(
|
return new LongDimensionMergerV9(
|
||||||
dimensionName,
|
outputName,
|
||||||
indexSpec,
|
indexSpec,
|
||||||
segmentWriteOutMedium
|
segmentWriteOutMedium
|
||||||
);
|
);
|
||||||
|
|
|
@ -27,15 +27,15 @@ import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
|
||||||
public class LongDimensionMergerV9 extends NumericDimensionMergerV9
|
public class LongDimensionMergerV9 extends NumericDimensionMergerV9
|
||||||
{
|
{
|
||||||
|
|
||||||
LongDimensionMergerV9(String dimensionName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium)
|
LongDimensionMergerV9(String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium)
|
||||||
{
|
{
|
||||||
super(dimensionName, indexSpec, segmentWriteOutMedium);
|
super(outputName, indexSpec, segmentWriteOutMedium);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
GenericColumnSerializer setupEncodedValueWriter()
|
GenericColumnSerializer setupEncodedValueWriter()
|
||||||
{
|
{
|
||||||
return IndexMergerV9.createLongColumnSerializer(segmentWriteOutMedium, dimensionName, indexSpec);
|
return IndexMergerV9.createLongColumnSerializer(segmentWriteOutMedium, outputName, indexSpec);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -22,6 +22,7 @@ package org.apache.druid.segment;
|
||||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||||
|
import org.apache.druid.error.DruidException;
|
||||||
import org.apache.druid.guice.annotations.PublicApi;
|
import org.apache.druid.guice.annotations.PublicApi;
|
||||||
import org.apache.druid.java.util.common.IAE;
|
import org.apache.druid.java.util.common.IAE;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularity;
|
import org.apache.druid.java.util.common.granularity.Granularity;
|
||||||
|
@ -56,6 +57,8 @@ public class Metadata
|
||||||
private final Boolean rollup;
|
private final Boolean rollup;
|
||||||
@Nullable
|
@Nullable
|
||||||
private final List<OrderBy> ordering;
|
private final List<OrderBy> ordering;
|
||||||
|
@Nullable
|
||||||
|
private final List<AggregateProjectionMetadata> projections;
|
||||||
|
|
||||||
public Metadata(
|
public Metadata(
|
||||||
@JsonProperty("container") @Nullable Map<String, Object> container,
|
@JsonProperty("container") @Nullable Map<String, Object> container,
|
||||||
|
@ -63,7 +66,8 @@ public class Metadata
|
||||||
@JsonProperty("timestampSpec") @Nullable TimestampSpec timestampSpec,
|
@JsonProperty("timestampSpec") @Nullable TimestampSpec timestampSpec,
|
||||||
@JsonProperty("queryGranularity") @Nullable Granularity queryGranularity,
|
@JsonProperty("queryGranularity") @Nullable Granularity queryGranularity,
|
||||||
@JsonProperty("rollup") @Nullable Boolean rollup,
|
@JsonProperty("rollup") @Nullable Boolean rollup,
|
||||||
@JsonProperty("ordering") @Nullable List<OrderBy> ordering
|
@JsonProperty("ordering") @Nullable List<OrderBy> ordering,
|
||||||
|
@JsonProperty("projections") @Nullable List<AggregateProjectionMetadata> projections
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.container = container == null ? new ConcurrentHashMap<>() : container;
|
this.container = container == null ? new ConcurrentHashMap<>() : container;
|
||||||
|
@ -72,6 +76,7 @@ public class Metadata
|
||||||
this.queryGranularity = queryGranularity;
|
this.queryGranularity = queryGranularity;
|
||||||
this.rollup = rollup;
|
this.rollup = rollup;
|
||||||
this.ordering = ordering;
|
this.ordering = ordering;
|
||||||
|
this.projections = projections;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
|
@ -124,6 +129,27 @@ public class Metadata
|
||||||
return ordering;
|
return ordering;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@JsonProperty
|
||||||
|
@JsonInclude(JsonInclude.Include.NON_EMPTY)
|
||||||
|
public List<AggregateProjectionMetadata> getProjections()
|
||||||
|
{
|
||||||
|
return projections;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Metadata withProjections(List<AggregateProjectionMetadata> projections)
|
||||||
|
{
|
||||||
|
return new Metadata(
|
||||||
|
container,
|
||||||
|
aggregators,
|
||||||
|
timestampSpec,
|
||||||
|
queryGranularity,
|
||||||
|
rollup,
|
||||||
|
ordering,
|
||||||
|
projections
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
public Metadata putAll(@Nullable Map<String, Object> other)
|
public Metadata putAll(@Nullable Map<String, Object> other)
|
||||||
{
|
{
|
||||||
if (other != null) {
|
if (other != null) {
|
||||||
|
@ -155,6 +181,7 @@ public class Metadata
|
||||||
List<Granularity> gransToMerge = new ArrayList<>();
|
List<Granularity> gransToMerge = new ArrayList<>();
|
||||||
List<Boolean> rollupToMerge = new ArrayList<>();
|
List<Boolean> rollupToMerge = new ArrayList<>();
|
||||||
List<List<OrderBy>> orderingsToMerge = new ArrayList<>();
|
List<List<OrderBy>> orderingsToMerge = new ArrayList<>();
|
||||||
|
List<List<AggregateProjectionMetadata>> projectionsToMerge = new ArrayList<>();
|
||||||
|
|
||||||
for (Metadata metadata : toBeMerged) {
|
for (Metadata metadata : toBeMerged) {
|
||||||
if (metadata != null) {
|
if (metadata != null) {
|
||||||
|
@ -176,6 +203,7 @@ public class Metadata
|
||||||
}
|
}
|
||||||
|
|
||||||
orderingsToMerge.add(metadata.getOrdering());
|
orderingsToMerge.add(metadata.getOrdering());
|
||||||
|
projectionsToMerge.add(metadata.getProjections());
|
||||||
mergedContainer.putAll(metadata.container);
|
mergedContainer.putAll(metadata.container);
|
||||||
} else {
|
} else {
|
||||||
//if metadata and hence aggregators and queryGranularity for some segment being merged are unknown then
|
//if metadata and hence aggregators and queryGranularity for some segment being merged are unknown then
|
||||||
|
@ -204,6 +232,7 @@ public class Metadata
|
||||||
Granularity.mergeGranularities(gransToMerge);
|
Granularity.mergeGranularities(gransToMerge);
|
||||||
|
|
||||||
final List<OrderBy> mergedOrdering = mergeOrderings(orderingsToMerge);
|
final List<OrderBy> mergedOrdering = mergeOrderings(orderingsToMerge);
|
||||||
|
validateProjections(projectionsToMerge);
|
||||||
|
|
||||||
Boolean rollup = null;
|
Boolean rollup = null;
|
||||||
if (rollupToMerge != null && !rollupToMerge.isEmpty()) {
|
if (rollupToMerge != null && !rollupToMerge.isEmpty()) {
|
||||||
|
@ -227,7 +256,8 @@ public class Metadata
|
||||||
mergedTimestampSpec,
|
mergedTimestampSpec,
|
||||||
mergedGranularity,
|
mergedGranularity,
|
||||||
rollup,
|
rollup,
|
||||||
mergedOrdering
|
mergedOrdering,
|
||||||
|
projectionsToMerge.get(0) // we're going to replace this later with updated rowcount
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -246,13 +276,14 @@ public class Metadata
|
||||||
Objects.equals(timestampSpec, metadata.timestampSpec) &&
|
Objects.equals(timestampSpec, metadata.timestampSpec) &&
|
||||||
Objects.equals(queryGranularity, metadata.queryGranularity) &&
|
Objects.equals(queryGranularity, metadata.queryGranularity) &&
|
||||||
Objects.equals(rollup, metadata.rollup) &&
|
Objects.equals(rollup, metadata.rollup) &&
|
||||||
Objects.equals(ordering, metadata.ordering);
|
Objects.equals(ordering, metadata.ordering) &&
|
||||||
|
Objects.equals(projections, metadata.projections);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int hashCode()
|
public int hashCode()
|
||||||
{
|
{
|
||||||
return Objects.hash(container, Arrays.hashCode(aggregators), timestampSpec, queryGranularity, rollup);
|
return Objects.hash(container, Arrays.hashCode(aggregators), timestampSpec, queryGranularity, rollup, ordering, projections);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -265,6 +296,7 @@ public class Metadata
|
||||||
", queryGranularity=" + queryGranularity +
|
", queryGranularity=" + queryGranularity +
|
||||||
", rollup=" + rollup +
|
", rollup=" + rollup +
|
||||||
", ordering=" + ordering +
|
", ordering=" + ordering +
|
||||||
|
", projections=" + projections +
|
||||||
'}';
|
'}';
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -308,4 +340,34 @@ public class Metadata
|
||||||
mergedOrdering.add(orderBy);
|
mergedOrdering.add(orderBy);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static void validateProjections(List<List<AggregateProjectionMetadata>> projectionsToMerge)
|
||||||
|
{
|
||||||
|
final Map<String, AggregateProjectionMetadata> projectionsMap = new HashMap<>();
|
||||||
|
// dedupe by name, fail if somehow incompatible projections are defined
|
||||||
|
int nullCount = 0;
|
||||||
|
int expectedSize = -1;
|
||||||
|
for (List<AggregateProjectionMetadata> projections : projectionsToMerge) {
|
||||||
|
if (projections == null) {
|
||||||
|
nullCount++;
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
if (expectedSize < 0) {
|
||||||
|
expectedSize = projections.size();
|
||||||
|
} else if (projections.size() != expectedSize) {
|
||||||
|
throw DruidException.defensive("Unable to merge projections: mismatched projections count");
|
||||||
|
}
|
||||||
|
for (AggregateProjectionMetadata projection : projections) {
|
||||||
|
AggregateProjectionMetadata prev = projectionsMap.putIfAbsent(projection.getSchema().getName(), projection);
|
||||||
|
if (prev != null && !prev.getSchema().equals(projection.getSchema())) {
|
||||||
|
throw DruidException.defensive("Unable to merge projections: mismatched projections [%s] and [%s]", prev, projection);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (nullCount > 0) {
|
||||||
|
if (nullCount != projectionsToMerge.size()) {
|
||||||
|
throw DruidException.defensive("Unable to merge projections: some projections were null");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -77,6 +77,7 @@ public class NestedCommonFormatColumnHandler implements DimensionHandler<Structu
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DimensionMergerV9 makeMerger(
|
public DimensionMergerV9 makeMerger(
|
||||||
|
String outputName,
|
||||||
IndexSpec indexSpec,
|
IndexSpec indexSpec,
|
||||||
SegmentWriteOutMedium segmentWriteOutMedium,
|
SegmentWriteOutMedium segmentWriteOutMedium,
|
||||||
ColumnCapabilities capabilities,
|
ColumnCapabilities capabilities,
|
||||||
|
@ -84,7 +85,7 @@ public class NestedCommonFormatColumnHandler implements DimensionHandler<Structu
|
||||||
Closer closer
|
Closer closer
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return new AutoTypeColumnMerger(name, castTo, indexSpec, segmentWriteOutMedium, closer);
|
return new AutoTypeColumnMerger(name, outputName, castTo, indexSpec, segmentWriteOutMedium, closer);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -73,6 +73,7 @@ public class NestedDataColumnHandlerV4 implements DimensionHandler<StructuredDat
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DimensionMergerV9 makeMerger(
|
public DimensionMergerV9 makeMerger(
|
||||||
|
String outputName,
|
||||||
IndexSpec indexSpec,
|
IndexSpec indexSpec,
|
||||||
SegmentWriteOutMedium segmentWriteOutMedium,
|
SegmentWriteOutMedium segmentWriteOutMedium,
|
||||||
ColumnCapabilities capabilities,
|
ColumnCapabilities capabilities,
|
||||||
|
@ -80,7 +81,7 @@ public class NestedDataColumnHandlerV4 implements DimensionHandler<StructuredDat
|
||||||
Closer closer
|
Closer closer
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return new NestedDataColumnMergerV4(name, indexSpec, segmentWriteOutMedium, closer);
|
return new NestedDataColumnMergerV4(outputName, indexSpec, segmentWriteOutMedium, closer);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -31,19 +31,19 @@ import java.util.List;
|
||||||
*/
|
*/
|
||||||
public abstract class NumericDimensionMergerV9 implements DimensionMergerV9
|
public abstract class NumericDimensionMergerV9 implements DimensionMergerV9
|
||||||
{
|
{
|
||||||
protected final String dimensionName;
|
protected final String outputName;
|
||||||
protected final IndexSpec indexSpec;
|
protected final IndexSpec indexSpec;
|
||||||
protected final SegmentWriteOutMedium segmentWriteOutMedium;
|
protected final SegmentWriteOutMedium segmentWriteOutMedium;
|
||||||
|
|
||||||
protected final GenericColumnSerializer serializer;
|
protected final GenericColumnSerializer serializer;
|
||||||
|
|
||||||
NumericDimensionMergerV9(
|
NumericDimensionMergerV9(
|
||||||
String dimensionName,
|
String outputName,
|
||||||
IndexSpec indexSpec,
|
IndexSpec indexSpec,
|
||||||
SegmentWriteOutMedium segmentWriteOutMedium
|
SegmentWriteOutMedium segmentWriteOutMedium
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.dimensionName = dimensionName;
|
this.outputName = outputName;
|
||||||
this.indexSpec = indexSpec;
|
this.indexSpec = indexSpec;
|
||||||
this.segmentWriteOutMedium = segmentWriteOutMedium;
|
this.segmentWriteOutMedium = segmentWriteOutMedium;
|
||||||
|
|
||||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.druid.query.OrderBy;
|
||||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||||
import org.apache.druid.segment.column.ColumnHolder;
|
import org.apache.druid.segment.column.ColumnHolder;
|
||||||
import org.apache.druid.segment.data.Indexed;
|
import org.apache.druid.segment.data.Indexed;
|
||||||
|
import org.apache.druid.segment.projections.QueryableProjection;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
@ -85,4 +86,16 @@ public interface QueryableIndex extends Closeable, ColumnInspector
|
||||||
//@Deprecated // This is still required for SimpleQueryableIndex. It should not go away until SimpleQueryableIndex is fixed
|
//@Deprecated // This is still required for SimpleQueryableIndex. It should not go away until SimpleQueryableIndex is fixed
|
||||||
@Override
|
@Override
|
||||||
void close();
|
void close();
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
default QueryableProjection<QueryableIndex> getProjection(CursorBuildSpec cursorBuildSpec)
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
default QueryableIndex getProjectionQueryableIndex(String name)
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,13 +20,19 @@
|
||||||
package org.apache.druid.segment;
|
package org.apache.druid.segment;
|
||||||
|
|
||||||
import org.apache.druid.query.OrderBy;
|
import org.apache.druid.query.OrderBy;
|
||||||
|
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||||
import org.apache.druid.segment.column.ColumnHolder;
|
import org.apache.druid.segment.column.ColumnHolder;
|
||||||
import org.apache.druid.segment.column.ColumnType;
|
import org.apache.druid.segment.column.ColumnType;
|
||||||
import org.apache.druid.segment.column.RowSignature;
|
import org.apache.druid.segment.column.RowSignature;
|
||||||
|
import org.apache.druid.segment.data.Offset;
|
||||||
|
import org.apache.druid.segment.projections.QueryableProjection;
|
||||||
|
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||||
|
import org.apache.druid.segment.vector.VectorOffset;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.util.LinkedHashSet;
|
import java.util.LinkedHashSet;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
public class QueryableIndexCursorFactory implements CursorFactory
|
public class QueryableIndexCursorFactory implements CursorFactory
|
||||||
{
|
{
|
||||||
|
@ -40,6 +46,46 @@ public class QueryableIndexCursorFactory implements CursorFactory
|
||||||
@Override
|
@Override
|
||||||
public CursorHolder makeCursorHolder(CursorBuildSpec spec)
|
public CursorHolder makeCursorHolder(CursorBuildSpec spec)
|
||||||
{
|
{
|
||||||
|
QueryableProjection<QueryableIndex> projection = index.getProjection(spec);
|
||||||
|
if (projection != null) {
|
||||||
|
return new QueryableIndexCursorHolder(projection.getRowSelector(), projection.getCursorBuildSpec())
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
protected ColumnSelectorFactory makeColumnSelectorFactoryForOffset(
|
||||||
|
ColumnCache columnCache,
|
||||||
|
Offset baseOffset
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return projection.wrapColumnSelectorFactory(
|
||||||
|
super.makeColumnSelectorFactoryForOffset(columnCache, baseOffset)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected VectorColumnSelectorFactory makeVectorColumnSelectorFactoryForOffset(
|
||||||
|
ColumnCache columnCache,
|
||||||
|
VectorOffset baseOffset
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return projection.wrapVectorColumnSelectorFactory(
|
||||||
|
super.makeVectorColumnSelectorFactoryForOffset(columnCache, baseOffset)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isPreAggregated()
|
||||||
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public List<AggregatorFactory> getAggregatorsForPreAggregated()
|
||||||
|
{
|
||||||
|
return projection.getCursorBuildSpec().getAggregators();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
return new QueryableIndexCursorHolder(index, CursorBuildSpec.builder(spec).build());
|
return new QueryableIndexCursorHolder(index, CursorBuildSpec.builder(spec).build());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -214,11 +214,9 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
||||||
}
|
}
|
||||||
|
|
||||||
final Offset baseCursorOffset = offset.clone();
|
final Offset baseCursorOffset = offset.clone();
|
||||||
final ColumnSelectorFactory columnSelectorFactory = new QueryableIndexColumnSelectorFactory(
|
final ColumnSelectorFactory columnSelectorFactory = makeColumnSelectorFactoryForOffset(
|
||||||
virtualColumns,
|
columnCache,
|
||||||
Cursors.getTimeOrdering(ordering),
|
baseCursorOffset
|
||||||
baseCursorOffset.getBaseReadableOffset(),
|
|
||||||
columnCache
|
|
||||||
);
|
);
|
||||||
// filterBundle will only be null if the filter itself is null, otherwise check to see if the filter
|
// filterBundle will only be null if the filter itself is null, otherwise check to see if the filter
|
||||||
// needs to use a value matcher
|
// needs to use a value matcher
|
||||||
|
@ -327,7 +325,20 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
private VectorColumnSelectorFactory makeVectorColumnSelectorFactoryForOffset(
|
protected ColumnSelectorFactory makeColumnSelectorFactoryForOffset(
|
||||||
|
ColumnCache columnCache,
|
||||||
|
Offset baseOffset
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return new QueryableIndexColumnSelectorFactory(
|
||||||
|
virtualColumns,
|
||||||
|
Cursors.getTimeOrdering(ordering),
|
||||||
|
baseOffset.getBaseReadableOffset(),
|
||||||
|
columnCache
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected VectorColumnSelectorFactory makeVectorColumnSelectorFactoryForOffset(
|
||||||
ColumnCache columnCache,
|
ColumnCache columnCache,
|
||||||
VectorOffset baseOffset
|
VectorOffset baseOffset
|
||||||
)
|
)
|
||||||
|
|
|
@ -262,6 +262,14 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
||||||
return new RowIteratorImpl();
|
return new RowIteratorImpl();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IndexableAdapter getProjectionAdapter(String projection)
|
||||||
|
{
|
||||||
|
QueryableIndex projectionIndex = input.getProjectionQueryableIndex(projection);
|
||||||
|
DruidException.conditionalDefensive(projectionIndex != null, "Projection[%s] was not found", projection);
|
||||||
|
return new QueryableIndexIndexableAdapter(projectionIndex);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* On {@link #moveToNext()} and {@link #mark()}, this class copies all column values into a set of {@link
|
* On {@link #moveToNext()} and {@link #mark()}, this class copies all column values into a set of {@link
|
||||||
* SettableColumnValueSelector} instances. Alternative approach was to save only offset in column and use the same
|
* SettableColumnValueSelector} instances. Alternative approach was to save only offset in column and use the same
|
||||||
|
|
|
@ -0,0 +1,73 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF 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 org.apache.druid.segment;
|
||||||
|
|
||||||
|
import org.apache.druid.math.expr.ExpressionType;
|
||||||
|
import org.apache.druid.query.dimension.DimensionSpec;
|
||||||
|
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
public class RemapColumnSelectorFactory implements ColumnSelectorFactory
|
||||||
|
{
|
||||||
|
private final ColumnSelectorFactory delegate;
|
||||||
|
private final Map<String, String> remap;
|
||||||
|
|
||||||
|
public RemapColumnSelectorFactory(ColumnSelectorFactory delegate, Map<String, String> remap)
|
||||||
|
{
|
||||||
|
this.delegate = delegate;
|
||||||
|
this.remap = remap;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec)
|
||||||
|
{
|
||||||
|
DimensionSpec remapDimensionSpec = dimensionSpec.withDimension(remap.getOrDefault(dimensionSpec.getDimension(), dimensionSpec.getDimension()));
|
||||||
|
return delegate.makeDimensionSelector(remapDimensionSpec);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ColumnValueSelector makeColumnValueSelector(String columnName)
|
||||||
|
{
|
||||||
|
return delegate.makeColumnValueSelector(remap.getOrDefault(columnName, columnName));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Nullable
|
||||||
|
public ColumnCapabilities getColumnCapabilities(String column)
|
||||||
|
{
|
||||||
|
return delegate.getColumnCapabilities(remap.getOrDefault(column, column));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public RowIdSupplier getRowIdSupplier()
|
||||||
|
{
|
||||||
|
return delegate.getRowIdSupplier();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public ExpressionType getType(String name)
|
||||||
|
{
|
||||||
|
return delegate.getType(remap.getOrDefault(name, name));
|
||||||
|
}
|
||||||
|
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue