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:
Clint Wylie 2024-10-05 10:33:41 -07:00 committed by GitHub
parent 1435b9f4bd
commit 7b3fc4e768
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
191 changed files with 7470 additions and 2599 deletions

View File

@ -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(),

View File

@ -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(

View File

@ -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(

View File

@ -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

View File

@ -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()
{ {

View File

@ -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()
{ {

View File

@ -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

View File

@ -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,

View File

@ -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(

View File

@ -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,

View File

@ -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

View File

@ -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,

View File

@ -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();
} }

View File

@ -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(

View File

@ -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(

View File

@ -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();
} }

View File

@ -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,

View File

@ -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",

View File

@ -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",

View File

@ -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",

View File

@ -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"),

View File

@ -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(

View File

@ -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,

View File

@ -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",

View File

@ -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,

View File

@ -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",

View File

@ -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());

View File

@ -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",

View File

@ -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
); );

View File

@ -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);

View File

@ -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

View File

@ -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),

View File

@ -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
), ),

View File

@ -1913,6 +1913,7 @@ public class CompactionTaskTest
null, null,
null, null,
null, null,
null,
null null
); );
} }

View File

@ -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,

View File

@ -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,

View File

@ -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),

View File

@ -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,

View File

@ -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
); );

View File

@ -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
); );

View File

@ -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,

View File

@ -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()
); );

View File

@ -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()
); );

View File

@ -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()

View File

@ -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
); );

View File

@ -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(

View File

@ -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
); );

View File

@ -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),

View File

@ -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),

View File

@ -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();

View File

@ -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());

View File

@ -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)

View File

@ -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",

View File

@ -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",

View File

@ -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),

View File

@ -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();
} }
} }

View File

@ -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();

View File

@ -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(

View File

@ -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,

View File

@ -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)

View File

@ -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()

View File

@ -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
), ),

View File

@ -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;
}
}
}

View File

@ -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;
}
} }

View File

@ -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)
{ {

View File

@ -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";

View File

@ -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

View File

@ -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).

View File

@ -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;
}
} }

View File

@ -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);

View File

@ -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)
{ {

View File

@ -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;

View File

@ -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) {

View File

@ -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
);
}
}
}

View File

@ -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)
{ {

View File

@ -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 =

View File

@ -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;
} }

View File

@ -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.

View File

@ -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,

View File

@ -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,

View File

@ -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
}
} }

View File

@ -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
); );

View File

@ -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

View File

@ -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
); );

View File

@ -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

View File

@ -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);
} }
} }

View File

@ -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,

View File

@ -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
); );

View File

@ -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);

View File

@ -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
); );

View File

@ -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

View File

@ -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");
}
}
}
} }

View File

@ -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

View File

@ -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

View File

@ -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;

View File

@ -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;
}
} }

View File

@ -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());
} }

View File

@ -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
) )

View File

@ -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

View File

@ -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