Introduce SegmentId class (#6370)

* Introduce SegmentId class

* tmp

* Fix SelectQueryRunnerTest

* Fix indentation

* Fixes

* Remove Comparators.inverse() tests

* Refinements

* Fix tests

* Fix more tests

* Remove duplicate DataSegmentTest, fixes #6064

* SegmentDescriptor doc

* Fix SQLMetadataStorageUpdaterJobHandler

* Fix DataSegment deserialization for ignoring id

* Add comments

* More comments

* Address more comments

* Fix compilation

* Restore segment2 in SystemSchemaTest according to a comment

* Fix style

* fix testServerSegmentsTable

* Fix compilation

* Add comments about why SegmentId and SegmentIdWithShardSpec are separate classes

* Fix SystemSchemaTest

* Fix style

* Compare SegmentDescriptor with SegmentId in Javadoc and comments rather than with DataSegment

* Remove a link, see https://youtrack.jetbrains.com/issue/IDEA-205164

* Fix compilation
This commit is contained in:
Roman Leventov 2019-01-22 02:11:10 +07:00 committed by Jihoon Son
parent 8ba33b2505
commit 8eae26fd4e
320 changed files with 3925 additions and 3923 deletions

View File

@ -65,6 +65,7 @@
<option name="ignoreUntypedCollections" value="false" />
</inspection_tool>
<inspection_tool class="ForLoopThatDoesntUseLoopVariable" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="Guava" enabled="false" level="WARNING" enabled_by_default="false" />
<inspection_tool class="HashCodeUsesNonFinalVariable" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="ImplicitArrayToString" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="IndexOfReplaceableByContains" enabled="true" level="ERROR" enabled_by_default="true" />
@ -167,6 +168,12 @@
<constraint name="x" nameOfFormalType="java\.util\.Random" exceptedTypes="java.util.Random" exprTypeWithinHierarchy="true" formalTypeWithinHierarchy="true" within="" contains="" />
<constraint name="a" within="" contains="" />
</searchConfiguration>
<searchConfiguration name="Use ZKPaths.makePath() with many arguments" created="1537504371779" text="org.apache.curator.utils.ZKPaths.makePath(org.apache.curator.utils.ZKPaths.makePath($x$, $y$), $z$)" recursive="false" caseInsensitive="true" type="JAVA">
<constraint name="__context__" target="true" within="" contains="" />
<constraint name="x" within="" contains="" />
<constraint name="y" within="" contains="" />
<constraint name="z" within="" contains="" />
</searchConfiguration>
<searchConfiguration name="Use RE (a Druid's class)" created="1539352150701" text="new $E$(org.apache.druid.java.util.common.StringUtils.format($x$))" recursive="false" caseInsensitive="true" type="JAVA">
<constraint name="__context__" target="true" within="" contains="" />
<constraint name="E" regexp="java\.lang\.RuntimeException" within="" contains="" />

View File

@ -71,6 +71,7 @@ import org.apache.druid.segment.column.ColumnConfig;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.timeline.SegmentId;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
@ -266,8 +267,8 @@ public class FilteredAggregatorBenchmark
{
QueryRunner<Result<TimeseriesResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"incIndex",
new IncrementalIndexSegment(incIndex, "incIndex")
SegmentId.dummy("incIndex"),
new IncrementalIndexSegment(incIndex, SegmentId.dummy("incIndex"))
);
List<Result<TimeseriesResultValue>> results = FilteredAggregatorBenchmark.runQuery(factory, runner, query);
@ -283,8 +284,8 @@ public class FilteredAggregatorBenchmark
{
final QueryRunner<Result<TimeseriesResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndex)
SegmentId.dummy("qIndex"),
new QueryableIndexSegment(qIndex, SegmentId.dummy("qIndex"))
);
List<Result<TimeseriesResultValue>> results = FilteredAggregatorBenchmark.runQuery(factory, runner, query);

View File

@ -73,6 +73,7 @@ import org.apache.druid.segment.column.ColumnConfig;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.timeline.SegmentId;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
@ -108,6 +109,8 @@ import java.util.concurrent.TimeUnit;
@Measurement(iterations = 30)
public class GroupByTypeInterfaceBenchmark
{
private static final SegmentId Q_INDEX_SEGMENT_ID = SegmentId.dummy("qIndex");
@Param({"4"})
private int numSegments;
@ -462,8 +465,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@ -480,8 +483,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@ -498,8 +501,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@ -516,8 +519,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longFloatQuery);
@ -534,8 +537,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longFloatQuery);
@ -546,8 +549,8 @@ public class GroupByTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@ -565,8 +568,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@ -577,8 +580,8 @@ public class GroupByTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@ -595,8 +598,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@ -607,8 +610,8 @@ public class GroupByTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@ -625,8 +628,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@ -637,8 +640,8 @@ public class GroupByTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longFloatQuery);
@ -655,8 +658,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@ -667,8 +670,8 @@ public class GroupByTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@ -685,8 +688,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@ -697,8 +700,8 @@ public class GroupByTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@ -715,8 +718,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@ -727,8 +730,8 @@ public class GroupByTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@ -746,8 +749,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@ -758,8 +761,8 @@ public class GroupByTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@ -776,8 +779,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@ -788,8 +791,8 @@ public class GroupByTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@ -806,8 +809,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@ -818,8 +821,8 @@ public class GroupByTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);

View File

@ -68,6 +68,7 @@ import org.apache.druid.segment.column.ColumnConfig;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.timeline.SegmentId;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
@ -99,6 +100,8 @@ import java.util.concurrent.TimeUnit;
@Measurement(iterations = 25)
public class TopNTypeInterfaceBenchmark
{
private static final SegmentId Q_INDEX_SEGMENT_ID = SegmentId.dummy("qIndex");
@Param({"1"})
private int numSegments;
@ -330,8 +333,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@ -347,8 +350,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@ -358,8 +361,8 @@ public class TopNTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@ -375,8 +378,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@ -386,8 +389,8 @@ public class TopNTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@ -403,8 +406,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@ -414,8 +417,8 @@ public class TopNTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@ -431,8 +434,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@ -448,8 +451,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@ -459,8 +462,8 @@ public class TopNTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@ -476,8 +479,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@ -487,8 +490,8 @@ public class TopNTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@ -504,8 +507,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@ -515,8 +518,8 @@ public class TopNTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@ -532,8 +535,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@ -549,8 +552,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@ -560,8 +563,8 @@ public class TopNTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@ -577,8 +580,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@ -588,8 +591,8 @@ public class TopNTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@ -605,8 +608,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@ -616,8 +619,8 @@ public class TopNTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
Q_INDEX_SEGMENT_ID,
new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);

View File

@ -46,6 +46,7 @@ import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import java.io.Closeable;
import java.io.File;
@ -131,7 +132,7 @@ public class SegmentGenerator implements Closeable
}
if (rows.size() % MAX_ROWS_IN_MEMORY == 0) {
indexes.add(makeIndex(dataSegment.getIdentifier(), indexes.size(), rows, indexSchema));
indexes.add(makeIndex(dataSegment.getId(), indexes.size(), rows, indexSchema));
rows.clear();
}
}
@ -139,7 +140,7 @@ public class SegmentGenerator implements Closeable
log.info("%,d/%,d rows generated.", numRows, numRows);
if (rows.size() > 0) {
indexes.add(makeIndex(dataSegment.getIdentifier(), indexes.size(), rows, indexSchema));
indexes.add(makeIndex(dataSegment.getId(), indexes.size(), rows, indexSchema));
rows.clear();
}
@ -181,7 +182,7 @@ public class SegmentGenerator implements Closeable
}
private QueryableIndex makeIndex(
final String identifier,
final SegmentId identifier,
final int indexNumber,
final List<InputRow> rows,
final IncrementalIndexSchema indexSchema
@ -190,7 +191,7 @@ public class SegmentGenerator implements Closeable
return IndexBuilder
.create()
.schema(indexSchema)
.tmpDir(new File(new File(tempDir, identifier), String.valueOf(indexNumber)))
.tmpDir(new File(new File(tempDir, identifier.toString()), String.valueOf(indexNumber)))
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
.rows(rows)
.buildMMappedIndex();

View File

@ -81,6 +81,7 @@ import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.timeline.SegmentId;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
@ -557,8 +558,8 @@ public class GroupByBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"incIndex",
new IncrementalIndexSegment(anIncrementalIndex, "incIndex")
SegmentId.dummy("incIndex"),
new IncrementalIndexSegment(anIncrementalIndex, SegmentId.dummy("incIndex"))
);
List<Row> results = GroupByBenchmark.runQuery(factory, runner, query);
@ -575,8 +576,8 @@ public class GroupByBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
SegmentId.dummy("qIndex"),
new QueryableIndexSegment(queryableIndexes.get(0), SegmentId.dummy("qIndex"))
);
List<Row> results = GroupByBenchmark.runQuery(factory, runner, query);
@ -665,8 +666,8 @@ public class GroupByBenchmark
String segmentName = "qIndex" + i;
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
segmentName,
new QueryableIndexSegment(segmentName, queryableIndexes.get(i))
SegmentId.dummy(segmentName),
new QueryableIndexSegment(queryableIndexes.get(i), SegmentId.dummy(segmentName))
);
runners.add(factory.getToolchest().preMergeQueryDecoration(runner));
}

View File

@ -31,6 +31,7 @@ import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryWatcher;
import org.apache.druid.segment.Segment;
import org.apache.druid.timeline.SegmentId;
import java.util.Map;
@ -38,7 +39,7 @@ public class QueryBenchmarkUtil
{
public static <T, QueryType extends Query<T>> QueryRunner<T> makeQueryRunner(
QueryRunnerFactory<T, QueryType> factory,
String segmentId,
SegmentId segmentId,
Segment adapter
)
{

View File

@ -77,6 +77,7 @@ import org.apache.druid.segment.column.ColumnConfig;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.timeline.SegmentId;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
@ -412,8 +413,8 @@ public class SearchBenchmark
{
QueryRunner<SearchHit> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"incIndex",
new IncrementalIndexSegment(incIndexes.get(0), "incIndex")
SegmentId.dummy("incIndex"),
new IncrementalIndexSegment(incIndexes.get(0), SegmentId.dummy("incIndex"))
);
List<Result<SearchResultValue>> results = SearchBenchmark.runQuery(factory, runner, query);
@ -430,8 +431,8 @@ public class SearchBenchmark
{
final QueryRunner<Result<SearchResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
SegmentId.dummy("qIndex"),
new QueryableIndexSegment(qIndexes.get(0), SegmentId.dummy("qIndex"))
);
List<Result<SearchResultValue>> results = SearchBenchmark.runQuery(factory, runner, query);
@ -453,8 +454,8 @@ public class SearchBenchmark
String segmentName = "qIndex" + i;
final QueryRunner<Result<SearchResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
segmentName,
new QueryableIndexSegment(segmentName, qIndexes.get(i))
SegmentId.dummy(segmentName),
new QueryableIndexSegment(qIndexes.get(i), SegmentId.dummy(segmentName))
);
singleSegmentRunners.add(toolChest.preMergeQueryDecoration(runner));
}

View File

@ -66,6 +66,7 @@ import org.apache.druid.segment.column.ColumnConfig;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.timeline.SegmentId;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
@ -268,8 +269,10 @@ public class SelectBenchmark
return queryResult.toList();
}
// don't run this benchmark with a query that doesn't use QueryGranularities.ALL,
// this pagination function probably doesn't work correctly in that case.
/**
* Don't run this benchmark with a query that doesn't use {@link Granularities#ALL},
* this pagination function probably doesn't work correctly in that case.
*/
private SelectQuery incrementQueryPagination(SelectQuery query, SelectResultValue prevResult)
{
Map<String, Integer> pagingIdentifiers = prevResult.getPagingIdentifiers();
@ -290,7 +293,7 @@ public class SelectBenchmark
{
SelectQuery queryCopy = query.withPagingSpec(PagingSpec.newSpec(pagingThreshold));
String segmentId = "incIndex";
SegmentId segmentId = SegmentId.dummy("incIndex");
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
segmentId,
@ -320,11 +323,11 @@ public class SelectBenchmark
{
SelectQuery queryCopy = query.withPagingSpec(PagingSpec.newSpec(pagingThreshold));
String segmentId = "qIndex";
SegmentId segmentId = SegmentId.dummy("qIndex");
QueryRunner<Result<SelectResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
segmentId,
new QueryableIndexSegment(segmentId, qIndexes.get(0))
new QueryableIndexSegment(qIndexes.get(0), segmentId)
);
boolean done = false;
@ -350,15 +353,14 @@ public class SelectBenchmark
{
SelectQuery queryCopy = query.withPagingSpec(PagingSpec.newSpec(pagingThreshold));
String segmentName;
List<QueryRunner<Result<SelectResultValue>>> singleSegmentRunners = new ArrayList<>();
QueryToolChest toolChest = factory.getToolchest();
for (int i = 0; i < numSegments; i++) {
segmentName = "qIndex" + i;
SegmentId segmentId = SegmentId.dummy("qIndex" + i);
QueryRunner<Result<SelectResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
segmentName,
new QueryableIndexSegment(segmentName, qIndexes.get(i))
segmentId,
new QueryableIndexSegment(qIndexes.get(i), segmentId)
);
singleSegmentRunners.add(toolChest.preMergeQueryDecoration(runner));
}

View File

@ -71,6 +71,7 @@ import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.timeline.SegmentId;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
@ -336,8 +337,8 @@ public class TimeseriesBenchmark
{
QueryRunner<Result<TimeseriesResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"incIndex",
new IncrementalIndexSegment(incIndexes.get(0), "incIndex")
SegmentId.dummy("incIndex"),
new IncrementalIndexSegment(incIndexes.get(0), SegmentId.dummy("incIndex"))
);
List<Result<TimeseriesResultValue>> results = TimeseriesBenchmark.runQuery(factory, runner, query);
@ -353,8 +354,8 @@ public class TimeseriesBenchmark
{
final QueryRunner<Result<TimeseriesResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
SegmentId.dummy("qIndex"),
new QueryableIndexSegment(qIndexes.get(0), SegmentId.dummy("qIndex"))
);
List<Result<TimeseriesResultValue>> results = TimeseriesBenchmark.runQuery(factory, runner, query);
@ -370,8 +371,8 @@ public class TimeseriesBenchmark
{
final QueryRunner<Result<TimeseriesResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
SegmentId.dummy("qIndex"),
new QueryableIndexSegment(qIndexes.get(0), SegmentId.dummy("qIndex"))
);
DimFilter filter = new SelectorDimFilter("dimSequential", "399", null);
@ -391,11 +392,11 @@ public class TimeseriesBenchmark
List<QueryRunner<Result<TimeseriesResultValue>>> singleSegmentRunners = new ArrayList<>();
QueryToolChest toolChest = factory.getToolchest();
for (int i = 0; i < numSegments; i++) {
String segmentName = "qIndex" + i;
SegmentId segmentId = SegmentId.dummy("qIndex" + i);
QueryRunner<Result<TimeseriesResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
segmentName,
new QueryableIndexSegment(segmentName, qIndexes.get(i))
segmentId,
new QueryableIndexSegment(qIndexes.get(i), segmentId)
);
singleSegmentRunners.add(toolChest.preMergeQueryDecoration(runner));
}

View File

@ -68,6 +68,7 @@ import org.apache.druid.segment.column.ColumnConfig;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.timeline.SegmentId;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
@ -317,8 +318,8 @@ public class TopNBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"incIndex",
new IncrementalIndexSegment(incIndexes.get(0), "incIndex")
SegmentId.dummy("incIndex"),
new IncrementalIndexSegment(incIndexes.get(0), SegmentId.dummy("incIndex"))
);
List<Result<TopNResultValue>> results = TopNBenchmark.runQuery(factory, runner, query);
@ -334,8 +335,8 @@ public class TopNBenchmark
{
final QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
SegmentId.dummy("qIndex"),
new QueryableIndexSegment(qIndexes.get(0), SegmentId.dummy("qIndex"))
);
List<Result<TopNResultValue>> results = TopNBenchmark.runQuery(factory, runner, query);
@ -352,11 +353,11 @@ public class TopNBenchmark
List<QueryRunner<Result<TopNResultValue>>> singleSegmentRunners = new ArrayList<>();
QueryToolChest toolChest = factory.getToolchest();
for (int i = 0; i < numSegments; i++) {
String segmentName = "qIndex" + i;
SegmentId segmentId = SegmentId.dummy("qIndex" + i);
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
segmentName,
new QueryableIndexSegment(segmentName, qIndexes.get(i))
segmentId,
new QueryableIndexSegment(qIndexes.get(i), segmentId)
);
singleSegmentRunners.add(toolChest.preMergeQueryDecoration(runner));
}

View File

@ -75,6 +75,7 @@ import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.timeline.SegmentId;
import org.joda.time.Interval;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
@ -346,11 +347,11 @@ public class TimeCompareBenchmark
List<QueryRunner<Result<TopNResultValue>>> singleSegmentRunners = new ArrayList<>();
QueryToolChest toolChest = topNFactory.getToolchest();
for (int i = 0; i < numSegments; i++) {
String segmentName = "qIndex" + i;
SegmentId segmentId = SegmentId.dummy("qIndex" + i);
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
topNFactory,
segmentName,
new QueryableIndexSegment(segmentName, qIndexes.get(i))
segmentId,
new QueryableIndexSegment(qIndexes.get(i), segmentId)
);
singleSegmentRunners.add(
new PerSegmentOptimizingQueryRunner<>(
@ -372,11 +373,11 @@ public class TimeCompareBenchmark
List<QueryRunner<Result<TimeseriesResultValue>>> singleSegmentRunnersT = new ArrayList<>();
QueryToolChest toolChestT = timeseriesFactory.getToolchest();
for (int i = 0; i < numSegments; i++) {
String segmentName = "qIndex" + i;
SegmentId segmentId = SegmentId.dummy("qIndex" + i);
QueryRunner<Result<TimeseriesResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
timeseriesFactory,
segmentName,
new QueryableIndexSegment(segmentName, qIndexes.get(i))
segmentId,
new QueryableIndexSegment(qIndexes.get(i), segmentId)
);
singleSegmentRunnersT.add(
new PerSegmentOptimizingQueryRunner<>(

View File

@ -8,7 +8,6 @@ org.joda.time.DateTime#<init>(int, int, int, int, int, int, int)
org.joda.time.DateTime#now()
org.joda.time.DateTime#parse(java.lang.String)
org.joda.time.DateTime#parse(java.lang.String, org.joda.time.format.DateTimeFormatter)
org.joda.time.DateTimeZone#forID(java.lang.String)
org.joda.time.base.BaseDateTime#<init>()
org.joda.time.base.BaseDateTime#<init>(long)
org.joda.time.base.BaseDateTime#<init>(int, int, int, int, int, int, int)
@ -59,6 +58,8 @@ org.joda.time.MutableInterval#<init>(long, long)
org.joda.time.MutableInterval#<init>(java.lang.Object)
org.joda.time.MutableInterval#parse(java.lang.String)
org.joda.time.DateTimeZone#forID(java.lang.String) @ Use DateTimes.inferTzFromString() instead
@defaultMessage Uses default time zone, use DateTimes.UtcFormatter to parse.
org.joda.time.format.DateTimeFormatter#parseInto(org.joda.time.ReadWritableInstant, java.lang.String, int)
org.joda.time.format.DateTimeFormatter#parseDateTime(java.lang.String)

View File

@ -36,9 +36,8 @@ import java.util.stream.Collectors;
/**
*/
public class Rows
public final class Rows
{
/**
* @param timeStamp rollup up timestamp to be used to create group key
* @param inputRow input row
@ -124,4 +123,6 @@ public class Rows
}
return metricValueString;
}
private Rows() {}
}

View File

@ -28,6 +28,7 @@ import org.joda.time.format.DateTimeFormatter;
import org.joda.time.format.ISODateTimeFormat;
import java.util.TimeZone;
import java.util.regex.Pattern;
public final class DateTimes
{
@ -41,8 +42,18 @@ public final class DateTimes
ISODateTimeFormat.dateTimeParser().withOffsetParsed()
);
/**
* This pattern aims to match strings, produced by {@link DateTime#toString()}. It's not rigorous: it could accept
* some strings that couldn't be obtained by calling toString() on any {@link DateTime} object, and also it could
* not match some valid DateTime string. Use for heuristic purposes only.
*/
public static final Pattern COMMON_DATE_TIME_PATTERN = Pattern.compile(
//year month day hour minute second millis time zone
"[0-9]{4}-[01][0-9]-[0-3][0-9]T[0-2][0-9]:[0-5][0-9]:[0-5][0-9]\\.[0-9]{3}(Z|[+\\-][0-9]{2}(:[0-9]{2}))"
);
@SuppressForbidden(reason = "DateTimeZone#forID")
public static DateTimeZone inferTzfromString(String tzId)
public static DateTimeZone inferTzFromString(String tzId)
{
try {
return DateTimeZone.forID(tzId);

View File

@ -20,7 +20,7 @@
package org.apache.druid.java.util.common.granularity;
/**
* This class was created b/c sometimes static initializers of a class that use a subclass can deadlock.
* This class was created b/c sometimes static initializers of a class that use a subclass can deadlock.
* See: #2979, #3979
*/
public class Granularities

View File

@ -20,7 +20,6 @@
package org.apache.druid.java.util.common.guava;
import com.google.common.collect.Ordering;
import com.google.common.primitives.Longs;
import org.joda.time.DateTimeComparator;
import org.joda.time.Interval;
@ -31,7 +30,6 @@ import java.util.Comparator;
*/
public class Comparators
{
private static final Ordering<Object> ALWAYS_EQUAL = new Ordering<Object>()
{
@SuppressWarnings("ComparatorMethodParameterNotUsed")
@ -59,25 +57,6 @@ public class Comparators
return NATURAL_NULLS_FIRST;
}
/**
* Use Guava Ordering.natural() instead
*
* @param <T>
* @return
*/
@Deprecated
public static <T extends Comparable> Comparator<T> comparable()
{
return new Comparator<T>()
{
@Override
public int compare(T t, T t1)
{
return t.compareTo(t1);
}
};
}
private static final Comparator<Interval> INTERVAL_BY_START_THEN_END = new Comparator<Interval>()
{
private final DateTimeComparator dateTimeComp = DateTimeComparator.getInstance();
@ -86,9 +65,9 @@ public class Comparators
public int compare(Interval lhs, Interval rhs)
{
if (lhs.getChronology().equals(rhs.getChronology())) {
int compare = Longs.compare(lhs.getStartMillis(), rhs.getStartMillis());
int compare = Long.compare(lhs.getStartMillis(), rhs.getStartMillis());
if (compare == 0) {
return Longs.compare(lhs.getEndMillis(), rhs.getEndMillis());
return Long.compare(lhs.getEndMillis(), rhs.getEndMillis());
}
return compare;
}
@ -108,9 +87,9 @@ public class Comparators
public int compare(Interval lhs, Interval rhs)
{
if (lhs.getChronology().equals(rhs.getChronology())) {
int compare = Longs.compare(lhs.getEndMillis(), rhs.getEndMillis());
int compare = Long.compare(lhs.getEndMillis(), rhs.getEndMillis());
if (compare == 0) {
return Longs.compare(lhs.getStartMillis(), rhs.getStartMillis());
return Long.compare(lhs.getStartMillis(), rhs.getStartMillis());
}
return compare;
}

View File

@ -81,9 +81,7 @@ public interface Sequence<T>
return new LimitedSequence<>(this, limit);
}
default <R> Sequence<R> flatMap(
Function<? super T, ? extends Sequence<? extends R>> mapper
)
default <R> Sequence<R> flatMap(Function<? super T, ? extends Sequence<? extends R>> mapper)
{
return new ConcatSequence<>(this.map(mapper));
}

View File

@ -24,7 +24,14 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import org.joda.time.Interval;
/**
*/
* The difference between this class and {@link org.apache.druid.timeline.SegmentId} is that this class is a "light"
* version of {@link org.apache.druid.timeline.SegmentId}, that only contains the interval, version, and partition
* number. It's used where the data source, another essential part of {@link org.apache.druid.timeline.SegmentId}
* is determined by the context (e. g. in org.apache.druid.client.CachingClusteredClient, where SegmentDescriptor is
* used when Brokers tell data servers which segments to include for a particular query) and where having lean JSON
* representations is important, because it's actively transferred detween Druid nodes. It's also for this reason that
* the JSON field names of SegmentDescriptor are abbreviated.
*/
public class SegmentDescriptor
{
private final Interval interval;

View File

@ -23,6 +23,7 @@ import org.apache.druid.guice.annotations.ExtensionPoint;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import java.util.Map;
import java.util.Set;
@ -61,16 +62,16 @@ public interface DataSegmentFinder
* @param segmentModifiedAt segment modified timestamp
*/
static void putInMapRetainingNewest(
Map<String, Pair<DataSegment, Long>> timestampedSegments,
Map<SegmentId, Pair<DataSegment, Long>> timestampedSegments,
DataSegment dataSegment,
long segmentModifiedAt
)
{
timestampedSegments.merge(
dataSegment.getIdentifier(),
dataSegment.getId(),
Pair.of(dataSegment, segmentModifiedAt),
(previous, current) -> {
log.warn("Multiple copies of segmentId [%s] found, using newest version", current.lhs.getIdentifier());
log.warn("Multiple copies of segmentId [%s] found, using newest version", current.lhs.getId());
return previous.rhs > current.rhs ? previous : current;
}
);

View File

@ -50,41 +50,18 @@ import java.util.Map;
import java.util.stream.Collectors;
/**
* Metadata of Druid's data segment. An immutable object.
*
* DataSegment's equality ({@link #equals}/{@link #hashCode}) and {@link #compareTo} methods consider only the
* {@link SegmentId} of the segment.
*/
@PublicApi
public class DataSegment implements Comparable<DataSegment>
{
public static String delimiter = "_";
private final Integer binaryVersion;
private static final Interner<String> STRING_INTERNER = Interners.newWeakInterner();
private static final Interner<List<String>> DIMENSIONS_INTERNER = Interners.newWeakInterner();
private static final Interner<List<String>> METRICS_INTERNER = Interners.newWeakInterner();
private static final Map<String, Object> PRUNED_LOAD_SPEC = ImmutableMap.of(
"load spec is pruned, because it's not needed on Brokers, but eats a lot of heap space",
""
);
public static String makeDataSegmentIdentifier(
String dataSource,
DateTime start,
DateTime end,
String version,
ShardSpec shardSpec
)
{
StringBuilder sb = new StringBuilder();
sb.append(dataSource).append(delimiter)
.append(start).append(delimiter)
.append(end).append(delimiter)
.append(version);
if (shardSpec.getPartitionNum() != 0) {
sb.append(delimiter).append(shardSpec.getPartitionNum());
}
return sb.toString();
}
/*
* The difference between this class and org.apache.druid.segment.Segment is that this class contains the segment
* metadata only, while org.apache.druid.segment.Segment represents the actual body of segment data, queryable.
*/
/**
* This class is needed for optional injection of pruneLoadSpec, see
@ -99,16 +76,22 @@ public class DataSegment implements Comparable<DataSegment>
@Inject(optional = true) @PruneLoadSpec boolean pruneLoadSpec = false;
}
private final String dataSource;
private final Interval interval;
private final String version;
private static final Interner<String> STRING_INTERNER = Interners.newWeakInterner();
private static final Interner<List<String>> DIMENSIONS_INTERNER = Interners.newWeakInterner();
private static final Interner<List<String>> METRICS_INTERNER = Interners.newWeakInterner();
private static final Map<String, Object> PRUNED_LOAD_SPEC = ImmutableMap.of(
"load spec is pruned, because it's not needed on Brokers, but eats a lot of heap space",
""
);
private final Integer binaryVersion;
private final SegmentId id;
@Nullable
private final Map<String, Object> loadSpec;
private final List<String> dimensions;
private final List<String> metrics;
private final ShardSpec shardSpec;
private final long size;
private final String identifier;
public DataSegment(
String dataSource,
@ -157,12 +140,8 @@ public class DataSegment implements Comparable<DataSegment>
@JacksonInject PruneLoadSpecHolder pruneLoadSpecHolder
)
{
// dataSource, dimensions & metrics are stored as canonical string values to decrease memory required for storing
// large numbers of segments.
this.dataSource = STRING_INTERNER.intern(dataSource);
this.interval = interval;
this.id = SegmentId.of(dataSource, interval, version, shardSpec);
this.loadSpec = pruneLoadSpecHolder.pruneLoadSpec ? PRUNED_LOAD_SPEC : prepareLoadSpec(loadSpec);
this.version = version;
// Deduplicating dimensions and metrics lists as a whole because they are very likely the same for the same
// dataSource
this.dimensions = prepareDimensionsOrMetrics(dimensions, DIMENSIONS_INTERNER);
@ -170,14 +149,6 @@ public class DataSegment implements Comparable<DataSegment>
this.shardSpec = (shardSpec == null) ? NoneShardSpec.instance() : shardSpec;
this.binaryVersion = binaryVersion;
this.size = size;
this.identifier = makeDataSegmentIdentifier(
this.dataSource,
this.interval.getStart(),
this.interval.getEnd(),
this.version,
this.shardSpec
);
}
@Nullable
@ -202,6 +173,8 @@ public class DataSegment implements Comparable<DataSegment>
List<String> result = list
.stream()
.filter(s -> !Strings.isNullOrEmpty(s))
// dimensions & metrics are stored as canonical string values to decrease memory required for storing
// large numbers of segments.
.map(STRING_INTERNER::intern)
// TODO replace with ImmutableList.toImmutableList() when updated to Guava 21+
.collect(Collectors.collectingAndThen(Collectors.toList(), ImmutableList::copyOf));
@ -217,13 +190,13 @@ public class DataSegment implements Comparable<DataSegment>
@JsonProperty
public String getDataSource()
{
return dataSource;
return id.getDataSource();
}
@JsonProperty
public Interval getInterval()
{
return interval;
return id.getInterval();
}
@Nullable
@ -236,7 +209,7 @@ public class DataSegment implements Comparable<DataSegment>
@JsonProperty
public String getVersion()
{
return version;
return id.getVersion();
}
@JsonProperty
@ -271,15 +244,16 @@ public class DataSegment implements Comparable<DataSegment>
return size;
}
@JsonProperty
public String getIdentifier()
// "identifier" for backward compatibility of JSON API
@JsonProperty(value = "identifier", access = JsonProperty.Access.READ_ONLY)
public SegmentId getId()
{
return identifier;
return id;
}
public SegmentDescriptor toDescriptor()
{
return new SegmentDescriptor(interval, version, shardSpec.getPartitionNum());
return new SegmentDescriptor(getInterval(), getVersion(), shardSpec.getPartitionNum());
}
public DataSegment withLoadSpec(Map<String, Object> loadSpec)
@ -315,14 +289,14 @@ public class DataSegment implements Comparable<DataSegment>
@Override
public int compareTo(DataSegment dataSegment)
{
return getIdentifier().compareTo(dataSegment.getIdentifier());
return getId().compareTo(dataSegment.getId());
}
@Override
public boolean equals(Object o)
{
if (o instanceof DataSegment) {
return getIdentifier().equals(((DataSegment) o).getIdentifier());
return getId().equals(((DataSegment) o).getId());
}
return false;
}
@ -330,7 +304,7 @@ public class DataSegment implements Comparable<DataSegment>
@Override
public int hashCode()
{
return getIdentifier().hashCode();
return getId().hashCode();
}
@Override
@ -341,10 +315,10 @@ public class DataSegment implements Comparable<DataSegment>
", shardSpec=" + shardSpec +
", metrics=" + metrics +
", dimensions=" + dimensions +
", version='" + version + '\'' +
", version='" + getVersion() + '\'' +
", loadSpec=" + loadSpec +
", interval=" + interval +
", dataSource='" + dataSource + '\'' +
", interval=" + getInterval() +
", dataSource='" + getDataSource() + '\'' +
", binaryVersion='" + binaryVersion + '\'' +
'}';
}
@ -472,7 +446,7 @@ public class DataSegment implements Comparable<DataSegment>
public DataSegment build()
{
// Check stuff that goes into the identifier, at least.
// Check stuff that goes into the id, at least.
Preconditions.checkNotNull(dataSource, "dataSource");
Preconditions.checkNotNull(interval, "interval");
Preconditions.checkNotNull(version, "version");

View File

@ -1,217 +0,0 @@
/*
* 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.timeline;
import com.google.common.base.Function;
import org.apache.druid.guice.annotations.PublicApi;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import java.util.Collection;
import java.util.Objects;
import java.util.stream.Collectors;
@PublicApi
public class DataSegmentUtils
{
private static final Logger LOGGER = new Logger(DataSegmentUtils.class);
public static Function<String, Interval> INTERVAL_EXTRACTOR(final String datasource)
{
return new Function<String, Interval>()
{
@Override
public Interval apply(String identifier)
{
SegmentIdentifierParts segmentIdentifierParts = valueOf(datasource, identifier);
if (segmentIdentifierParts == null) {
throw new IAE("Invalid identifier [%s]", identifier);
}
return segmentIdentifierParts.getInterval();
}
};
}
/**
* Parses a segment identifier into its components: dataSource, interval, version, and any trailing tags. Ignores
* shard spec.
*
* It is possible that this method may incorrectly parse an identifier, for example if the dataSource name in the
* identifier contains a DateTime parseable string such as 'datasource_2000-01-01T00:00:00.000Z' and dataSource was
* provided as 'datasource'. The desired behavior in this case would be to return null since the identifier does not
* actually belong to the provided dataSource but a non-null result would be returned. This is an edge case that would
* currently only affect paged select queries with a union dataSource of two similarly-named dataSources as in the
* given example.
*
* @param dataSource the dataSource corresponding to this identifier
* @param identifier segment identifier
* @return a {@link SegmentIdentifierParts} object if the identifier could be parsed, null otherwise
*/
public static SegmentIdentifierParts valueOf(String dataSource, String identifier)
{
if (!identifier.startsWith(StringUtils.format("%s_", dataSource))) {
return null;
}
String remaining = identifier.substring(dataSource.length() + 1);
String[] splits = remaining.split(DataSegment.delimiter);
if (splits.length < 3) {
return null;
}
try {
DateTime start = DateTimes.ISO_DATE_TIME.parse(splits[0]);
DateTime end = DateTimes.ISO_DATE_TIME.parse(splits[1]);
String version = splits[2];
String trail = splits.length > 3 ? join(splits, DataSegment.delimiter, 3, splits.length) : null;
return new SegmentIdentifierParts(
dataSource,
new Interval(start, end),
version,
trail
);
}
catch (IllegalArgumentException e) {
return null;
}
}
public static String withInterval(final String dataSource, final String identifier, Interval newInterval)
{
SegmentIdentifierParts segmentDesc = DataSegmentUtils.valueOf(dataSource, identifier);
if (segmentDesc == null) {
// happens for test segments which has invalid segment id.. ignore for now
LOGGER.warn("Invalid segment identifier " + identifier);
return identifier;
}
return segmentDesc.withInterval(newInterval).toString();
}
/**
* Creates a comma delimited list of segment identifiers
* @param segments
* @return
*/
public static String getIdentifiersString(Collection<DataSegment> segments)
{
return segments.stream().map(DataSegment::getIdentifier).collect(Collectors.joining(", "));
}
public static class SegmentIdentifierParts
{
private final String dataSource;
private final Interval interval;
private final String version;
private final String trail;
public SegmentIdentifierParts(String dataSource, Interval interval, String version, String trail)
{
this.dataSource = dataSource;
this.interval = interval;
this.version = version;
this.trail = trail;
}
@PublicApi
public String getDataSource()
{
return dataSource;
}
public Interval getInterval()
{
return interval;
}
@PublicApi
public String getVersion()
{
return version;
}
public SegmentIdentifierParts withInterval(Interval interval)
{
return new SegmentIdentifierParts(dataSource, interval, version, trail);
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
SegmentIdentifierParts that = (SegmentIdentifierParts) o;
if (!Objects.equals(dataSource, that.dataSource)) {
return false;
}
if (!Objects.equals(interval, that.interval)) {
return false;
}
if (!Objects.equals(version, that.version)) {
return false;
}
if (!Objects.equals(trail, that.trail)) {
return false;
}
return true;
}
@Override
public int hashCode()
{
return Objects.hash(dataSource, interval, version, trail);
}
@Override
public String toString()
{
return join(
new Object[]{dataSource, interval.getStart(), interval.getEnd(), version, trail},
DataSegment.delimiter, 0, version == null ? 3 : trail == null ? 4 : 5
);
}
}
private static String join(Object[] input, String delimiter, int start, int end)
{
StringBuilder builder = new StringBuilder();
for (int i = start; i < end; i++) {
if (i > start) {
builder.append(delimiter);
}
if (input[i] != null) {
builder.append(input[i]);
}
}
return builder.toString();
}
}

View File

@ -0,0 +1,408 @@
/*
* 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.timeline;
import com.fasterxml.jackson.annotation.JsonValue;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Joiner;
import com.google.common.base.Splitter;
import com.google.common.collect.Interner;
import com.google.common.collect.Interners;
import com.google.common.collect.Iterables;
import com.google.common.primitives.Ints;
import org.apache.druid.guice.annotations.PublicApi;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.timeline.partition.ShardSpec;
import org.joda.time.Chronology;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
import java.util.function.Function;
import java.util.regex.Matcher;
import java.util.stream.IntStream;
/**
* Identifier of {@link DataSegment}.
*/
@PublicApi
public final class SegmentId implements Comparable<SegmentId>
{
/*
* Implementation note: this class must be optimized for resident memory footprint, because segment data consumes
* a lot of heap memory on Druid Broker and Coordinator nodes.
*
* This class is separate from org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec
* because in a lot of places segment ids are transmitted as "segment id strings" that don't contain enough
* information to deconstruct the ShardSpec. Also, even a single extra field is important for SegmentIds, because it
* adds to the memory footprint considerably.
*
* The difference between this class and {@link org.apache.druid.query.SegmentDescriptor} is that the latter is
* a "light" version of SegmentId, that only contains the interval, version, and partition number. It's used where the
* data source, another essential part of SegmentId is determined by the context (e. g. in {@link
* org.apache.druid.client.CachingClusteredClient}, where SegmentDescriptor is used when Brokers tell data servers
* which segments to include for a particular query) and where having lean JSON representations is important, because
* it's actively transferred detween Druid nodes. It's also for this reason that the JSON field names of
* SegmentDescriptor are abbreviated.
*
* API design note: "SegmentId" is chosen as the name for this class instead of more verbose "SegmentIdentifier" or
* "DataSegmentIdentifier" because it's used very frequently and a long class name adds noticeable clutter. Variables
* of SegmentId type are recommended to be named "segmentId" rather than "identifier" or "segmentIdentifier".
*/
/**
* {@link #dataSource} field values are stored as canonical strings to decrease memory required for large numbers of
* segment identifiers.
*/
private static final Interner<String> STRING_INTERNER = Interners.newWeakInterner();
private static final char DELIMITER = '_';
private static final Splitter DELIMITER_SPLITTER = Splitter.on(DELIMITER);
private static final Joiner DELIMITER_JOINER = Joiner.on(DELIMITER);
private static final int DATE_TIME_SIZE_UPPER_LIMIT = "yyyy-MM-ddTHH:mm:ss.SSS+00:00".length();
public static SegmentId of(String dataSource, Interval interval, String version, int partitionNum)
{
return new SegmentId(dataSource, interval, version, partitionNum);
}
public static SegmentId of(String dataSource, Interval interval, String version, @Nullable ShardSpec shardSpec)
{
return of(dataSource, interval, version, shardSpec != null ? shardSpec.getPartitionNum() : 0);
}
/**
* Tries to parse a segment id from the given String representation, or returns null on failure. If returns a non-null
* {@code SegmentId} object, calling {@link #toString()} on the latter is guaranteed to return a string equal to the
* argument string of the {@code tryParse()} call.
*
* It is possible that this method may incorrectly parse a segment id, for example if the dataSource name in the
* segment id contains a DateTime parseable string such as 'datasource_2000-01-01T00:00:00.000Z' and dataSource was
* provided as 'datasource'. The desired behavior in this case would be to return null since the identifier does not
* actually belong to the provided dataSource but a non-null result would be returned. This is an edge case that would
* currently only affect paged select queries with a union dataSource of two similarly-named dataSources as in the
* given example.
*
* Another source of ambiguity is the end of a segment id like '_123' - it could always be interpreted either as the
* partitionNum of the segment id, or as the end of the version, with the implicit partitionNum of 0. This method
* prefers the first iterpretation. To iterate all possible parsings of a segment id, use {@link
* #iteratePossibleParsingsWithDataSource}.
*
* @param dataSource the dataSource corresponding to this segment id
* @param segmentId segment id
* @return a {@link SegmentId} object if the segment id could be parsed, null otherwise
*/
@Nullable
public static SegmentId tryParse(String dataSource, String segmentId)
{
List<SegmentId> possibleParsings = iteratePossibleParsingsWithDataSource(dataSource, segmentId);
return possibleParsings.isEmpty() ? null : possibleParsings.get(0);
}
/**
* Returns a (potentially empty) lazy iteration of all possible valid parsings of the given segment id string into
* {@code SegmentId} objects.
*
* Warning: most of the parsing work is repeated each time {@link Iterable#iterator()} of this iterable is consumed,
* so it should be consumed only once if possible.
*/
public static Iterable<SegmentId> iterateAllPossibleParsings(String segmentId)
{
List<String> splits = DELIMITER_SPLITTER.splitToList(segmentId);
String probableDataSource = tryExtractMostProbableDataSource(segmentId);
// Iterate parsings with the most probably data source first to allow the users of iterateAllPossibleParsings() to
// break from the iteration earlier with higher probability.
if (probableDataSource != null) {
List<SegmentId> probableParsings = iteratePossibleParsingsWithDataSource(probableDataSource, segmentId);
Iterable<SegmentId> otherPossibleParsings = () -> IntStream
.range(1, splits.size() - 3)
.mapToObj(dataSourceDelimiterOrder -> DELIMITER_JOINER.join(splits.subList(0, dataSourceDelimiterOrder)))
.filter(dataSource -> dataSource.length() != probableDataSource.length())
.flatMap(dataSource -> iteratePossibleParsingsWithDataSource(dataSource, segmentId).stream())
.iterator();
return Iterables.concat(probableParsings, otherPossibleParsings);
} else {
return () -> IntStream
.range(1, splits.size() - 3)
.mapToObj(dataSourceDelimiterOrder -> {
String dataSource = DELIMITER_JOINER.join(splits.subList(0, dataSourceDelimiterOrder));
return iteratePossibleParsingsWithDataSource(dataSource, segmentId);
})
.flatMap(List::stream)
.iterator();
}
}
/**
* Returns a list of either 0, 1 or 2 elements containing possible parsings if the given segment id String
* representation with the given data source name. Returns an empty list when parsing into a valid {@code SegmentId}
* object is impossible. Returns a list of a single element when the given segment id doesn't end with
* '_[any positive number]', that means that the implicit partitionNum is 0. Otherwise the end of the segment id
* is interpreted in two ways: with the explicit partitionNum (the first element in the returned list), and with the
* implicit partitionNum of 0 and the version that ends with '_[any positive number]' (the second element in the
* returned list).
*/
public static List<SegmentId> iteratePossibleParsingsWithDataSource(String dataSource, String segmentId)
{
if (!segmentId.startsWith(dataSource) || segmentId.charAt(dataSource.length()) != DELIMITER) {
return Collections.emptyList();
}
String remaining = segmentId.substring(dataSource.length() + 1);
List<String> splits = DELIMITER_SPLITTER.splitToList(remaining);
if (splits.size() < 3) {
return Collections.emptyList();
}
DateTime start;
DateTime end;
try {
start = DateTimes.ISO_DATE_TIME.parse(splits.get(0));
end = DateTimes.ISO_DATE_TIME.parse(splits.get(1));
}
catch (IllegalArgumentException e) {
return Collections.emptyList();
}
if (start.compareTo(end) >= 0) {
return Collections.emptyList();
}
List<SegmentId> possibleParsings = new ArrayList<>(2);
String version = DELIMITER_JOINER.join(splits.subList(2, Math.max(splits.size() - 1, 3)));
String trail = splits.size() > 3 ? splits.get(splits.size() - 1) : null;
if (trail != null) {
Integer possiblePartitionNum = Ints.tryParse(trail);
if (possiblePartitionNum != null && possiblePartitionNum > 0) {
possibleParsings.add(of(dataSource, new Interval(start, end), version, possiblePartitionNum));
}
version = version + '_' + trail;
}
possibleParsings.add(of(dataSource, new Interval(start, end), version, 0));
return possibleParsings;
}
/**
* Heuristically tries to extract the most probable data source from a String segment id representation, or returns
* null on failure.
*
* This method is not guaranteed to return a non-null data source given a valid String segment id representation.
*/
@VisibleForTesting
@Nullable
static String tryExtractMostProbableDataSource(String segmentId)
{
Matcher dateTimeMatcher = DateTimes.COMMON_DATE_TIME_PATTERN.matcher(segmentId);
while (true) {
if (!dateTimeMatcher.find()) {
return null;
}
int dataSourceEnd = dateTimeMatcher.start() - 1;
if (segmentId.charAt(dataSourceEnd) != DELIMITER) {
continue;
}
return segmentId.substring(0, dataSourceEnd);
}
}
public static Function<String, Interval> makeIntervalExtractor(final String dataSource)
{
return identifier -> {
SegmentId segmentIdentifierParts = tryParse(dataSource, identifier);
if (segmentIdentifierParts == null) {
throw new IAE("Invalid identifier [%s]", identifier);
}
return segmentIdentifierParts.getInterval();
};
}
/**
* Creates a dummy SegmentId with the given data source. This method is useful in benchmark and test code.
*/
public static SegmentId dummy(String dataSource)
{
return of(dataSource, Intervals.ETERNITY, "dummy_version", 0);
}
private final String dataSource;
/**
* {@code intervalStartMillis}, {@link #intervalEndMillis} and {@link #intervalChronology} are the three fields of
* an {@link Interval}. Storing them directly to flatten the structure and reduce the heap space consumption.
*/
private final long intervalStartMillis;
private final long intervalEndMillis;
@Nullable
private final Chronology intervalChronology;
private final String version;
private final int partitionNum;
/**
* Cache the hash code eagerly, because SegmentId is almost always expected to be used as a map key or
* for map lookup.
*/
private final int hashCode;
private SegmentId(String dataSource, Interval interval, String version, int partitionNum)
{
this.dataSource = STRING_INTERNER.intern(Objects.requireNonNull(dataSource));
this.intervalStartMillis = interval.getStartMillis();
this.intervalEndMillis = interval.getEndMillis();
this.intervalChronology = interval.getChronology();
// Versions are timestamp-based Strings, interning of them doesn't make sense. If this is not the case, interning
// could be conditionally allowed via a system property.
this.version = Objects.requireNonNull(version);
this.partitionNum = partitionNum;
this.hashCode = computeHashCode();
}
private int computeHashCode()
{
// Start with partitionNum and version hash codes, because they are often little sequential numbers. If they are
// added in the end of the chain, resulting hashCode of SegmentId could have worse distribution.
int hashCode = partitionNum;
// 1000003 is a constant used in Google AutoValue, provides a little better distribution than 31
hashCode = hashCode * 1000003 + version.hashCode();
hashCode = hashCode * 1000003 + dataSource.hashCode();
hashCode = hashCode * 1000003 + Long.hashCode(intervalStartMillis);
hashCode = hashCode * 1000003 + Long.hashCode(intervalEndMillis);
hashCode = hashCode * 1000003 + Objects.hashCode(intervalChronology);
return hashCode;
}
public String getDataSource()
{
return dataSource;
}
public DateTime getIntervalStart()
{
return new DateTime(intervalStartMillis, intervalChronology);
}
public DateTime getIntervalEnd()
{
return new DateTime(intervalEndMillis, intervalChronology);
}
public Interval getInterval()
{
return new Interval(intervalStartMillis, intervalEndMillis, intervalChronology);
}
@Nullable
public String getVersion()
{
return version;
}
public int getPartitionNum()
{
return partitionNum;
}
public SegmentId withInterval(Interval newInterval)
{
return of(dataSource, newInterval, version, partitionNum);
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (!(o instanceof SegmentId)) {
return false;
}
SegmentId that = (SegmentId) o;
// Compare hashCode instead of partitionNum: break the chain quicker if the objects are not equal. If the hashCodes
// are equal as well as all other fields used to compute them, the partitionNums are also guaranteed to be equal.
return hashCode == that.hashCode &&
dataSource.equals(that.dataSource) &&
intervalStartMillis == that.intervalStartMillis &&
intervalEndMillis == that.intervalEndMillis &&
Objects.equals(intervalChronology, that.intervalChronology) &&
version.equals(that.version);
}
@Override
public int hashCode()
{
return hashCode;
}
@Override
public int compareTo(SegmentId o)
{
int result = dataSource.compareTo(o.dataSource);
if (result != 0) {
return result;
}
result = Long.compare(intervalStartMillis, o.intervalStartMillis);
if (result != 0) {
return result;
}
result = Long.compare(intervalEndMillis, o.intervalEndMillis);
if (result != 0) {
return result;
}
result = version.compareTo(o.version);
if (result != 0) {
return result;
}
return Integer.compare(partitionNum, o.partitionNum);
}
@JsonValue
@Override
public String toString()
{
StringBuilder sb = new StringBuilder(safeUpperLimitOfStringSize());
sb.append(dataSource).append(DELIMITER)
.append(getIntervalStart()).append(DELIMITER)
.append(getIntervalEnd()).append(DELIMITER)
.append(version);
if (partitionNum != 0) {
sb.append(DELIMITER).append(partitionNum);
}
return sb.toString();
}
private int safeUpperLimitOfStringSize()
{
int delimiters = 4;
int partitionNumSizeUpperLimit = 3; // less than 1000 partitions
return dataSource.length() +
version.length() +
(DATE_TIME_SIZE_UPPER_LIMIT * 2) + // interval start and end
delimiters +
partitionNumSizeUpperLimit;
}
}

View File

@ -0,0 +1,40 @@
/*
* 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.java.util.common;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.junit.Assert;
import org.junit.Test;
public class DateTimesTest
{
@Test
public void testCommonDateTimePattern()
{
DateTime dt1 = DateTimes.nowUtc();
DateTime dt2 = new DateTime(System.currentTimeMillis(), DateTimes.inferTzFromString("IST"));
DateTime dt3 = new DateTime(System.currentTimeMillis(), DateTimeZone.forOffsetHoursMinutes(1, 30));
for (DateTime dt : new DateTime[] {dt1, dt2, dt3}) {
Assert.assertTrue(DateTimes.COMMON_DATE_TIME_PATTERN.matcher(dt.toString()).matches());
}
}
}

View File

@ -20,11 +20,11 @@
package org.apache.druid.java.util.common.guava.nary;
import com.google.common.collect.Lists;
import org.apache.druid.java.util.common.guava.Comparators;
import org.junit.Assert;
import org.junit.Test;
import java.util.Arrays;
import java.util.Comparator;
/**
*/
@ -36,7 +36,7 @@ public class SortedMergeIteratorTest
SortedMergeIterator<Integer, Integer> iter = SortedMergeIterator.create(
Arrays.asList(1, 4, 5, 7, 9).iterator(),
Arrays.asList(1, 2, 3, 6, 7, 8, 9, 10, 11).iterator(),
Comparators.comparable(),
Comparator.naturalOrder(),
new BinaryFn<Integer, Integer, Integer>()
{
@Override

View File

@ -148,7 +148,7 @@ public class DataSegmentTest
Assert.assertEquals(segment.getMetrics(), deserializedSegment.getMetrics());
Assert.assertEquals(segment.getShardSpec(), deserializedSegment.getShardSpec());
Assert.assertEquals(segment.getSize(), deserializedSegment.getSize());
Assert.assertEquals(segment.getIdentifier(), deserializedSegment.getIdentifier());
Assert.assertEquals(segment.getId(), deserializedSegment.getId());
deserializedSegment = mapper.readValue(mapper.writeValueAsString(segment), DataSegment.class);
Assert.assertEquals(0, segment.compareTo(deserializedSegment));
@ -172,7 +172,7 @@ public class DataSegmentTest
Assert.assertEquals(
"foo_2012-01-01T00:00:00.000Z_2012-01-02T00:00:00.000Z_2012-01-01T11:22:33.444Z",
segment.getIdentifier()
segment.getId().toString()
);
}
@ -188,7 +188,7 @@ public class DataSegmentTest
Assert.assertEquals(
"foo_2012-01-01T00:00:00.000Z_2012-01-02T00:00:00.000Z_2012-01-01T11:22:33.444Z",
segment.getIdentifier()
segment.getId().toString()
);
}
@ -204,7 +204,7 @@ public class DataSegmentTest
Assert.assertEquals(
"foo_2012-01-01T00:00:00.000Z_2012-01-02T00:00:00.000Z_2012-01-01T11:22:33.444Z_7",
segment.getIdentifier()
segment.getId().toString()
);
}

View File

@ -1,123 +0,0 @@
/*
* 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.timeline;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.timeline.DataSegmentUtils.SegmentIdentifierParts;
import org.junit.Assert;
import org.junit.Test;
/**
*/
public class DataSegmentUtilsTest
{
@Test
public void testBasic()
{
String datasource = "datasource";
SegmentIdentifierParts desc = new SegmentIdentifierParts(datasource, Intervals.of("2015-01-02/2015-01-03"), "ver", "0_0");
Assert.assertEquals("datasource_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_0", desc.toString());
Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
Assert.assertEquals("datasource_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver_0_0", desc.toString());
Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
desc = new SegmentIdentifierParts(datasource, Intervals.of("2015-01-02/2015-01-03"), "ver", null);
Assert.assertEquals("datasource_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver", desc.toString());
Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
Assert.assertEquals("datasource_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver", desc.toString());
Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
}
@Test
public void testDataSourceWithUnderscore1()
{
String datasource = "datasource_1";
SegmentIdentifierParts desc = new SegmentIdentifierParts(datasource, Intervals.of("2015-01-02/2015-01-03"), "ver", "0_0");
Assert.assertEquals("datasource_1_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_0", desc.toString());
Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
Assert.assertEquals("datasource_1_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver_0_0", desc.toString());
Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
desc = new SegmentIdentifierParts(datasource, Intervals.of("2015-01-02/2015-01-03"), "ver", null);
Assert.assertEquals("datasource_1_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver", desc.toString());
Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
Assert.assertEquals("datasource_1_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver", desc.toString());
Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
}
@Test
public void testDataSourceWithUnderscore2()
{
String dataSource = "datasource_2015-01-01T00:00:00.000Z";
SegmentIdentifierParts desc = new SegmentIdentifierParts(dataSource, Intervals.of("2015-01-02/2015-01-03"), "ver", "0_0");
Assert.assertEquals(
"datasource_2015-01-01T00:00:00.000Z_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_0",
desc.toString()
);
Assert.assertEquals(desc, DataSegmentUtils.valueOf(dataSource, desc.toString()));
desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
Assert.assertEquals(
"datasource_2015-01-01T00:00:00.000Z_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver_0_0",
desc.toString()
);
Assert.assertEquals(desc, DataSegmentUtils.valueOf(dataSource, desc.toString()));
desc = new SegmentIdentifierParts(dataSource, Intervals.of("2015-01-02/2015-01-03"), "ver", null);
Assert.assertEquals(
"datasource_2015-01-01T00:00:00.000Z_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver",
desc.toString()
);
Assert.assertEquals(desc, DataSegmentUtils.valueOf(dataSource, desc.toString()));
desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
Assert.assertEquals(
"datasource_2015-01-01T00:00:00.000Z_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver",
desc.toString()
);
Assert.assertEquals(desc, DataSegmentUtils.valueOf(dataSource, desc.toString()));
}
@Test
public void testInvalidFormat0()
{
Assert.assertNull(DataSegmentUtils.valueOf("ds", "datasource_2015-01-02T00:00:00.000Z_2014-10-20T00:00:00.000Z_version"));
}
@Test
public void testInvalidFormat1()
{
Assert.assertNull(DataSegmentUtils.valueOf("datasource", "datasource_invalid_interval_version"));
}
@Test
public void testInvalidFormat2()
{
Assert.assertNull(DataSegmentUtils.valueOf("datasource", "datasource_2015-01-02T00:00:00.000Z_version"));
}
}

View File

@ -0,0 +1,231 @@
/*
* 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.timeline;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.Intervals;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
import java.util.Arrays;
import java.util.List;
import java.util.Set;
public class SegmentIdTest
{
@Test
public void testBasic()
{
String datasource = "datasource";
SegmentId desc = SegmentId.of(datasource, Intervals.of("2015-01-02/2015-01-03"), "ver_0", 1);
Assert.assertEquals("datasource_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_1", desc.toString());
Assert.assertEquals(desc, SegmentId.tryParse(datasource, desc.toString()));
desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
Assert.assertEquals("datasource_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver_0_1", desc.toString());
Assert.assertEquals(desc, SegmentId.tryParse(datasource, desc.toString()));
desc = SegmentId.of(datasource, Intervals.of("2015-01-02/2015-01-03"), "ver", 0);
Assert.assertEquals("datasource_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver", desc.toString());
Assert.assertEquals(desc, SegmentId.tryParse(datasource, desc.toString()));
desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
Assert.assertEquals("datasource_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver", desc.toString());
Assert.assertEquals(desc, SegmentId.tryParse(datasource, desc.toString()));
}
@Test
public void testDataSourceWithUnderscore()
{
String datasource = "datasource_1";
SegmentId desc = SegmentId.of(datasource, Intervals.of("2015-01-02/2015-01-03"), "ver_0", 1);
Assert.assertEquals("datasource_1_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_1", desc.toString());
Assert.assertEquals(desc, SegmentId.tryParse(datasource, desc.toString()));
desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
Assert.assertEquals("datasource_1_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver_0_1", desc.toString());
Assert.assertEquals(desc, SegmentId.tryParse(datasource, desc.toString()));
desc = SegmentId.of(datasource, Intervals.of("2015-01-02/2015-01-03"), "ver", 0);
Assert.assertEquals("datasource_1_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver", desc.toString());
Assert.assertEquals(desc, SegmentId.tryParse(datasource, desc.toString()));
desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
Assert.assertEquals("datasource_1_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver", desc.toString());
Assert.assertEquals(desc, SegmentId.tryParse(datasource, desc.toString()));
}
/**
* Test the ambiguity of a datasource name ending with '_yyyy-mm-dd..' string that could be considered either as the
* end of the datasource name or the interval start in the segment id's string representation.
*/
@Test
public void testDataSourceWithUnderscoreAndTimeStringInDataSourceName()
{
String dataSource = "datasource_2015-01-01T00:00:00.000Z";
SegmentId desc = SegmentId.of(dataSource, Intervals.of("2015-01-02/2015-01-03"), "ver_0", 1);
Assert.assertEquals(
"datasource_2015-01-01T00:00:00.000Z_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_1",
desc.toString()
);
Assert.assertEquals(desc, SegmentId.tryParse(dataSource, desc.toString()));
desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
Assert.assertEquals(
"datasource_2015-01-01T00:00:00.000Z_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver_0_1",
desc.toString()
);
Assert.assertEquals(desc, SegmentId.tryParse(dataSource, desc.toString()));
desc = SegmentId.of(dataSource, Intervals.of("2015-01-02/2015-01-03"), "ver", 0);
Assert.assertEquals(
"datasource_2015-01-01T00:00:00.000Z_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver",
desc.toString()
);
Assert.assertEquals(desc, SegmentId.tryParse(dataSource, desc.toString()));
desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
Assert.assertEquals(
"datasource_2015-01-01T00:00:00.000Z_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver",
desc.toString()
);
Assert.assertEquals(desc, SegmentId.tryParse(dataSource, desc.toString()));
}
/**
* The interval start is later than the end
*/
@Test
public void testInvalidFormat0()
{
Assert.assertNull(
SegmentId.tryParse("datasource", "datasource_2015-01-02T00:00:00.000Z_2014-10-20T00:00:00.000Z_version")
);
}
/**
* No interval dates
*/
@Test
public void testInvalidFormat1()
{
Assert.assertNull(SegmentId.tryParse("datasource", "datasource_invalid_interval_version"));
}
/**
* Not enough interval dates
*/
@Test
public void testInvalidFormat2()
{
Assert.assertNull(SegmentId.tryParse("datasource", "datasource_2015-01-02T00:00:00.000Z_version"));
}
/**
* Tests that {@link SegmentId#tryExtractMostProbableDataSource} successfully extracts data sources from some
* reasonable segment ids.
*/
@Test
public void testTryParseHeuristically()
{
List<String> segmentIds = Arrays.asList(
"datasource_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_1",
"datasource_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver",
"datasource_1_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_1",
"datasource_1_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver",
"datasource_2015-01-01T00:00:00.000Z_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_1",
"datasource_2015-01-01T00:00:00.000Z_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver"
);
for (String segmentId : segmentIds) {
String dataSource = SegmentId.tryExtractMostProbableDataSource(segmentId);
Assert.assertTrue("datasource".equals(dataSource) || "datasource_1".equals(dataSource));
Assert.assertTrue(!SegmentId.iteratePossibleParsingsWithDataSource(dataSource, segmentId).isEmpty());
}
}
@Test
public void testTryParseVersionAmbiguity()
{
SegmentId segmentId =
SegmentId.tryParse("datasource", "datasource_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0");
Assert.assertNotNull(segmentId);
Assert.assertEquals("ver_0", segmentId.getVersion());
Assert.assertEquals(0, segmentId.getPartitionNum());
}
@Test
public void testIterateAllPossibleParsings()
{
String segmentId = "datasource_2015-01-01T00:00:00.000Z_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_1";
List<SegmentId> possibleParsings = ImmutableList.copyOf(SegmentId.iterateAllPossibleParsings(segmentId));
DateTime dt1 = DateTimes.of("2015-01-01T00:00:00.000Z");
DateTime dt2 = DateTimes.of("2015-01-02T00:00:00.000Z");
DateTime dt3 = DateTimes.of("2015-01-03T00:00:00.000Z");
Set<SegmentId> expected = ImmutableSet.of(
SegmentId.of("datasource", new Interval(dt1, dt2), "2015-01-03T00:00:00.000Z_ver_0", 1),
SegmentId.of("datasource", new Interval(dt1, dt2), "2015-01-03T00:00:00.000Z_ver_0_1", 0),
SegmentId.of("datasource_2015-01-01T00:00:00.000Z", new Interval(dt2, dt3), "ver_0", 1),
SegmentId.of("datasource_2015-01-01T00:00:00.000Z", new Interval(dt2, dt3), "ver_0_1", 0)
);
Assert.assertEquals(4, possibleParsings.size());
Assert.assertEquals(expected, ImmutableSet.copyOf(possibleParsings));
}
@Test
public void testIterateAllPossibleParsingsWithEmptyVersion()
{
String segmentId = "datasource_2015-01-01T00:00:00.000Z_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z__1";
List<SegmentId> possibleParsings = ImmutableList.copyOf(SegmentId.iterateAllPossibleParsings(segmentId));
DateTime dt1 = DateTimes.of("2015-01-01T00:00:00.000Z");
DateTime dt2 = DateTimes.of("2015-01-02T00:00:00.000Z");
DateTime dt3 = DateTimes.of("2015-01-03T00:00:00.000Z");
Set<SegmentId> expected = ImmutableSet.of(
SegmentId.of("datasource", new Interval(dt1, dt2), "2015-01-03T00:00:00.000Z_", 1),
SegmentId.of("datasource", new Interval(dt1, dt2), "2015-01-03T00:00:00.000Z__1", 0),
SegmentId.of("datasource_2015-01-01T00:00:00.000Z", new Interval(dt2, dt3), "", 1),
SegmentId.of("datasource_2015-01-01T00:00:00.000Z", new Interval(dt2, dt3), "_1", 0)
);
Assert.assertEquals(4, possibleParsings.size());
Assert.assertEquals(expected, ImmutableSet.copyOf(possibleParsings));
}
/**
* Three DateTime strings included, but not ascending, that makes a pair of parsings impossible, compared to {@link
* #testIterateAllPossibleParsings}.
*/
@Test
public void testIterateAllPossibleParsings2()
{
String segmentId = "datasource_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_2015-01-02T00:00:00.000Z_ver_1";
List<SegmentId> possibleParsings = ImmutableList.copyOf(SegmentId.iterateAllPossibleParsings(segmentId));
DateTime dt1 = DateTimes.of("2015-01-02T00:00:00.000Z");
DateTime dt2 = DateTimes.of("2015-01-03T00:00:00.000Z");
Set<SegmentId> expected = ImmutableSet.of(
SegmentId.of("datasource", new Interval(dt1, dt2), "2015-01-02T00:00:00.000Z_ver", 1),
SegmentId.of("datasource", new Interval(dt1, dt2), "2015-01-02T00:00:00.000Z_ver_1", 0)
);
Assert.assertEquals(2, possibleParsings.size());
Assert.assertEquals(expected, ImmutableSet.copyOf(possibleParsings));
}
}

View File

@ -59,10 +59,12 @@ public class AzureDataSegmentKiller implements DataSegmentKiller
azureStorage.emptyCloudBlobDirectory(containerName, dirPath);
}
catch (StorageException e) {
throw new SegmentLoadingException(e, "Couldn't kill segment[%s]: [%s]", segment.getIdentifier(), e.getExtendedErrorInformation() == null ? null : e.getExtendedErrorInformation().getErrorMessage());
Object extendedInfo =
e.getExtendedErrorInformation() == null ? null : e.getExtendedErrorInformation().getErrorMessage();
throw new SegmentLoadingException(e, "Couldn't kill segment[%s]: [%s]", segment.getId(), extendedInfo);
}
catch (URISyntaxException e) {
throw new SegmentLoadingException(e, "Couldn't kill segment[%s]: [%s]", segment.getIdentifier(), e.getReason());
throw new SegmentLoadingException(e, "Couldn't kill segment[%s]: [%s]", segment.getId(), e.getReason());
}
}

View File

@ -88,7 +88,7 @@ public class CloudFilesDataSegmentPusher implements DataSegmentPusher
final long indexSize = CompressionUtils.zip(indexFilesDir, zipOutFile);
log.info("Copying segment[%s] to CloudFiles at location[%s]", inSegment.getIdentifier(), segmentPath);
log.info("Copying segment[%s] to CloudFiles at location[%s]", inSegment.getId(), segmentPath);
return CloudFilesUtils.retryCloudFilesOperation(
() -> {
CloudFilesObject segmentData = new CloudFilesObject(

View File

@ -119,7 +119,7 @@ public class DistinctCountGroupByQueryTest
client_type,
client_type
))
.setInterval(QueryRunnerTestHelper.fullOnInterval)
.setInterval(QueryRunnerTestHelper.fullOnIntervalSpec)
.setLimitSpec(
new DefaultLimitSpec(
Collections.singletonList(new OrderByColumnSpec(client_type, OrderByColumnSpec.Direction.DESCENDING)),

View File

@ -88,7 +88,7 @@ public class DistinctCountTimeseriesQueryTest
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
.intervals(QueryRunnerTestHelper.fullOnInterval)
.intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
QueryRunnerTestHelper.rowsCount,

View File

@ -117,7 +117,7 @@ public class DistinctCountTopNQueryTest
TopNQuery query = new TopNQueryBuilder().dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
.intervals(QueryRunnerTestHelper.fullOnInterval)
.intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.dimension(client_type)
.metric("UV")
.threshold(10)

View File

@ -82,7 +82,7 @@ public class GoogleDataSegmentFinder implements DataSegmentFinder
InputStream is = storage.get(item.getBucket(), item.getName());
final DataSegment dataSegment = jsonMapper.readValue(is, DataSegment.class);
LOG.info("Found segment [%s] located at [%s]", dataSegment.getIdentifier(), indexZip);
LOG.info("Found segment [%s] located at [%s]", dataSegment.getId(), indexZip);
Map<String, Object> loadSpec = dataSegment.getLoadSpec();

View File

@ -59,7 +59,7 @@ public class GoogleDataSegmentKiller implements DataSegmentKiller
deleteIfPresent(bucket, descriptorPath);
}
catch (IOException e) {
throw new SegmentLoadingException(e, "Couldn't kill segment[%s]: [%s]", segment.getIdentifier(), e.getMessage());
throw new SegmentLoadingException(e, "Couldn't kill segment[%s]: [%s]", segment.getId(), e.getMessage());
}
}

View File

@ -354,7 +354,7 @@ public class MaterializedViewSupervisor implements Supervisor
// drop derivative segments which interval equals the interval in toDeleteBaseSegments
for (Interval interval : toDropInterval.keySet()) {
for (DataSegment segment : derivativeSegments.get(interval)) {
segmentManager.removeSegment(dataSource, segment.getIdentifier());
segmentManager.removeSegment(segment.getId());
}
}
// data of the latest interval will be built firstly.
@ -462,7 +462,7 @@ public class MaterializedViewSupervisor implements Supervisor
{
log.info("Clear all metadata of dataSource %s", dataSource);
metadataStorageCoordinator.deletePendingSegments(dataSource, ALL_INTERVAL);
segmentManager.removeDatasource(dataSource);
segmentManager.removeDataSource(dataSource);
metadataStorageCoordinator.deleteDataSourceMetadata(dataSource);
}

View File

@ -46,7 +46,7 @@ public class MaterializedViewQueryQueryToolChestTest
TimeseriesQuery realQuery = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.dayGran)
.intervals(QueryRunnerTestHelper.fullOnInterval)
.intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(QueryRunnerTestHelper.rowsCount)
.descending(true)
.build();
@ -77,9 +77,10 @@ public class MaterializedViewQueryQueryToolChestTest
Result<TimeseriesResultValue> result = new Result<>(DateTimes.nowUtc(), new TimeseriesResultValue(ImmutableMap.<String, Object>builder()
.put("dim1", "dimvalue1")
.build()));
Result<TimeseriesResultValue> result = new Result<>(
DateTimes.nowUtc(),
new TimeseriesResultValue(ImmutableMap.of("dim1", "dimvalue1"))
);
Result<TimeseriesResultValue> postResult = (Result<TimeseriesResultValue>) postFn.apply(result);
Map<String, Object> postResultMap = postResult.getValue().getBaseObject();

View File

@ -45,7 +45,7 @@ import static org.apache.druid.query.QueryRunnerTestHelper.addRowsIndexConstant;
import static org.apache.druid.query.QueryRunnerTestHelper.allGran;
import static org.apache.druid.query.QueryRunnerTestHelper.commonDoubleAggregators;
import static org.apache.druid.query.QueryRunnerTestHelper.dataSource;
import static org.apache.druid.query.QueryRunnerTestHelper.fullOnInterval;
import static org.apache.druid.query.QueryRunnerTestHelper.fullOnIntervalSpec;
import static org.apache.druid.query.QueryRunnerTestHelper.indexMetric;
import static org.apache.druid.query.QueryRunnerTestHelper.marketDimension;
@ -75,7 +75,7 @@ public class MaterializedViewQueryTest
.dimension(marketDimension)
.metric(indexMetric)
.threshold(4)
.intervals(fullOnInterval)
.intervals(fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@ -95,6 +95,6 @@ public class MaterializedViewQueryTest
Assert.assertEquals(query, serdeQuery);
Assert.assertEquals(new TableDataSource(dataSource), query.getDataSource());
Assert.assertEquals(allGran, query.getGranularity());
Assert.assertEquals(fullOnInterval.getIntervals(), query.getIntervals());
Assert.assertEquals(fullOnIntervalSpec.getIntervals(), query.getIntervals());
}
}

View File

@ -43,6 +43,7 @@ import org.apache.druid.query.groupby.strategy.GroupByStrategySelector;
import org.apache.druid.query.groupby.strategy.GroupByStrategyV2;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.timeline.SegmentId;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
@ -114,8 +115,8 @@ public class MapVirtualColumnGroupByTest
runner = QueryRunnerTestHelper.makeQueryRunner(
factory,
"index",
new IncrementalIndexSegment(incrementalIndex, "index"),
SegmentId.dummy("index"),
new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("index")),
"incremental"
);
}

View File

@ -45,6 +45,7 @@ import org.apache.druid.query.select.SelectQueryRunnerFactory;
import org.apache.druid.query.select.SelectResultValue;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.timeline.SegmentId;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
@ -110,14 +111,14 @@ public class MapVirtualColumnSelectTest
Arrays.asList(
QueryRunnerTestHelper.makeQueryRunner(
factory,
"index1",
new IncrementalIndexSegment(index1, "index1"),
SegmentId.dummy("index1"),
new IncrementalIndexSegment(index1, SegmentId.dummy("index1")),
"incremental"
),
QueryRunnerTestHelper.makeQueryRunner(
factory,
"index2",
new QueryableIndexSegment("index2", index2),
SegmentId.dummy("index2"),
new QueryableIndexSegment(index2, SegmentId.dummy("index2")),
"queryable"
)
)
@ -136,7 +137,7 @@ public class MapVirtualColumnSelectTest
return Druids.newSelectQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
.intervals(QueryRunnerTestHelper.fullOnInterval)
.intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.pagingSpec(new PagingSpec(null, 3));
}

View File

@ -40,6 +40,7 @@ import org.apache.druid.query.topn.TopNQueryQueryToolChest;
import org.apache.druid.query.topn.TopNQueryRunnerFactory;
import org.apache.druid.query.topn.TopNResultValue;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.timeline.SegmentId;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
@ -75,8 +76,8 @@ public class MapVirtualColumnTopNTest
runner = QueryRunnerTestHelper.makeQueryRunner(
factory,
"index1",
new IncrementalIndexSegment(incrementalIndex, "index1"),
SegmentId.dummy("index1"),
new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("index1")),
"incremental"
);
}

View File

@ -28,6 +28,7 @@ import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.segment.loading.DataSegmentFinder;
import org.apache.druid.segment.loading.SegmentLoadingException;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocatedFileStatus;
@ -60,7 +61,7 @@ public class HdfsDataSegmentFinder implements DataSegmentFinder
public Set<DataSegment> findSegments(String workingDirPathStr, boolean updateDescriptor)
throws SegmentLoadingException
{
final Map<String, Pair<DataSegment, Long>> timestampedSegments = new HashMap<>();
final Map<SegmentId, Pair<DataSegment, Long>> timestampedSegments = new HashMap<>();
final Path workingDirPath = new Path(workingDirPathStr);
FileSystem fs;
try {
@ -109,7 +110,7 @@ public class HdfsDataSegmentFinder implements DataSegmentFinder
if (fs.exists(indexZip)) {
final DataSegment dataSegment = mapper.readValue(fs.open(path), DataSegment.class);
log.info("Found segment [%s] located at [%s]", dataSegment.getIdentifier(), indexZip);
log.info("Found segment [%s] located at [%s]", dataSegment.getId(), indexZip);
final Map<String, Object> loadSpec = dataSegment.getLoadSpec();
final String pathWithoutScheme = indexZip.toUri().getPath();

View File

@ -58,7 +58,7 @@ public class HdfsDataSegmentKiller implements DataSegmentKiller
public void kill(DataSegment segment) throws SegmentLoadingException
{
final Path segmentPath = getPath(segment);
log.info("Killing segment[%s] mapped to path[%s]", segment.getIdentifier(), segmentPath);
log.info("Killing segment[%s] mapped to path[%s]", segment.getId(), segmentPath);
try {
String filename = segmentPath.getName();

View File

@ -106,7 +106,7 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher
log.info(
"Copying segment[%s] to HDFS at location[%s/%s]",
segment.getIdentifier(),
segment.getId(),
fullyQualifiedStorageDirectory.get(),
storageDir
);

View File

@ -216,17 +216,17 @@ public class HdfsDataSegmentFinderTest
DataSegment updatedSegment5 = null;
for (DataSegment dataSegment : segments) {
if (dataSegment.getIdentifier().equals(SEGMENT_1.getIdentifier())) {
if (dataSegment.getId().equals(SEGMENT_1.getId())) {
updatedSegment1 = dataSegment;
} else if (dataSegment.getIdentifier().equals(SEGMENT_2.getIdentifier())) {
} else if (dataSegment.getId().equals(SEGMENT_2.getId())) {
updatedSegment2 = dataSegment;
} else if (dataSegment.getIdentifier().equals(SEGMENT_3.getIdentifier())) {
} else if (dataSegment.getId().equals(SEGMENT_3.getId())) {
updatedSegment3 = dataSegment;
} else if (dataSegment.getIdentifier().equals(SEGMENT_4_0.getIdentifier())) {
} else if (dataSegment.getId().equals(SEGMENT_4_0.getId())) {
updatedSegment4_0 = dataSegment;
} else if (dataSegment.getIdentifier().equals(SEGMENT_4_1.getIdentifier())) {
} else if (dataSegment.getId().equals(SEGMENT_4_1.getId())) {
updatedSegment4_1 = dataSegment;
} else if (dataSegment.getIdentifier().equals(SEGMENT_5.getIdentifier())) {
} else if (dataSegment.getId().equals(SEGMENT_5.getId())) {
updatedSegment5 = dataSegment;
} else {
Assert.fail("Unexpected segment");

View File

@ -174,7 +174,7 @@ public class ApproximateHistogramGroupByQueryTest
QueryRunnerTestHelper.marketDimension,
"marketalias"
))
.setInterval(QueryRunnerTestHelper.fullOnInterval)
.setInterval(QueryRunnerTestHelper.fullOnIntervalSpec)
.setLimitSpec(
new DefaultLimitSpec(
Collections.singletonList(new OrderByColumnSpec("marketalias", OrderByColumnSpec.Direction.DESCENDING)),
@ -233,7 +233,7 @@ public class ApproximateHistogramGroupByQueryTest
QueryRunnerTestHelper.marketDimension,
"marketalias"
))
.setInterval(QueryRunnerTestHelper.fullOnInterval)
.setInterval(QueryRunnerTestHelper.fullOnIntervalSpec)
.setLimitSpec(
new DefaultLimitSpec(
Collections.singletonList(new OrderByColumnSpec("marketalias", OrderByColumnSpec.Direction.DESCENDING)),

View File

@ -127,7 +127,7 @@ public class ApproximateHistogramTopNQueryTest
.dimension(QueryRunnerTestHelper.marketDimension)
.metric(QueryRunnerTestHelper.dependentPostAggMetric)
.threshold(4)
.intervals(QueryRunnerTestHelper.fullOnInterval)
.intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(

View File

@ -26,6 +26,7 @@ import com.google.common.base.Preconditions;
import com.google.common.base.Supplier;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import org.apache.druid.data.input.Committer;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.impl.InputRowParser;
@ -64,7 +65,7 @@ import org.apache.druid.segment.realtime.FireDepartment;
import org.apache.druid.segment.realtime.FireDepartmentMetrics;
import org.apache.druid.segment.realtime.appenderator.Appenderator;
import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult;
import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata;
import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver;
import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
@ -111,7 +112,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.stream.Collectors;
/**
* Kafka index task runner which doesn't support incremental segment publishing. We keep this to support rolling update.
@ -436,7 +436,7 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
? Utils.nullableListOf((InputRow) null)
: parser.parseBatch(ByteBuffer.wrap(valueBytes));
boolean isPersistRequired = false;
final Map<String, Set<SegmentIdentifier>> segmentsToMoveOut = new HashMap<>();
final Map<String, Set<SegmentIdWithShardSpec>> segmentsToMoveOut = new HashMap<>();
for (InputRow row : rows) {
if (row != null && task.withinMinMaxRecordTime(row)) {
@ -477,10 +477,9 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
if (isPersistRequired) {
driver.persist(committerSupplier.get());
}
segmentsToMoveOut.forEach((String sequence, Set<SegmentIdentifier> segments) -> driver.moveSegmentOut(
sequence,
new ArrayList<SegmentIdentifier>(segments)
));
segmentsToMoveOut.forEach((String sequence, Set<SegmentIdWithShardSpec> segments) -> {
driver.moveSegmentOut(sequence, new ArrayList<>(segments));
});
}
catch (ParseException e) {
handleParseException(e, record);
@ -558,14 +557,10 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
sequenceNames.values()
).get();
final List<String> publishedSegments = published.getSegments()
.stream()
.map(DataSegment::getIdentifier)
.collect(Collectors.toList());
List<?> publishedSegmentIds = Lists.transform(published.getSegments(), DataSegment::getId);
log.info(
"Published segments[%s] with metadata[%s].",
publishedSegments,
"Published segments %s with metadata[%s].",
publishedSegmentIds,
Preconditions.checkNotNull(published.getCommitMetadata(), "commitMetadata")
);
@ -585,11 +580,11 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
}
if (handedOff == null) {
log.warn("Failed to handoff segments[%s]", publishedSegments);
log.warn("Failed to handoff segments %s", publishedSegmentIds);
} else {
log.info(
"Handoff completed for segments[%s] with metadata[%s]",
handedOff.getSegments().stream().map(DataSegment::getIdentifier).collect(Collectors.toList()),
"Handoff completed for segments %s with metadata[%s]",
Lists.transform(handedOff.getSegments(), DataSegment::getId),
Preconditions.checkNotNull(handedOff.getCommitMetadata(), "commitMetadata")
);
}

View File

@ -33,6 +33,7 @@ import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.segment.loading.DataSegmentFinder;
import org.apache.druid.segment.loading.SegmentLoadingException;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import java.io.ByteArrayInputStream;
import java.io.IOException;
@ -65,7 +66,7 @@ public class S3DataSegmentFinder implements DataSegmentFinder
@Override
public Set<DataSegment> findSegments(String workingDirPath, boolean updateDescriptor) throws SegmentLoadingException
{
final Map<String, Pair<DataSegment, Long>> timestampedSegments = new HashMap<>();
final Map<SegmentId, Pair<DataSegment, Long>> timestampedSegments = new HashMap<>();
try {
final Iterator<S3ObjectSummary> objectSummaryIterator = S3Utils.objectSummaryIterator(
@ -87,7 +88,7 @@ public class S3DataSegmentFinder implements DataSegmentFinder
S3ObjectInputStream is = indexObject.getObjectContent()) {
final ObjectMetadata objectMetadata = indexObject.getObjectMetadata();
final DataSegment dataSegment = jsonMapper.readValue(is, DataSegment.class);
log.info("Found segment [%s] located at [%s]", dataSegment.getIdentifier(), indexZip);
log.info("Found segment [%s] located at [%s]", dataSegment.getId(), indexZip);
final Map<String, Object> loadSpec = dataSegment.getLoadSpec();
if (!S3StorageDruidModule.SCHEME.equals(loadSpec.get("type")) ||

View File

@ -62,7 +62,7 @@ public class S3DataSegmentKiller implements DataSegmentKiller
}
}
catch (AmazonServiceException e) {
throw new SegmentLoadingException(e, "Couldn't kill segment[%s]: [%s]", segment.getIdentifier(), e);
throw new SegmentLoadingException(e, "Couldn't kill segment[%s]: [%s]", segment.getId(), e);
}
}

View File

@ -110,7 +110,7 @@ public class S3DataSegmentMover implements DataSegmentMover
);
}
catch (AmazonServiceException e) {
throw new SegmentLoadingException(e, "Unable to move segment[%s]: [%s]", segment.getIdentifier(), e);
throw new SegmentLoadingException(e, "Unable to move segment[%s]: [%s]", segment.getId(), e);
}
}

View File

@ -90,7 +90,7 @@ public class S3DataSegmentPusher implements DataSegmentPusher
{
final String s3Path = S3Utils.constructSegmentPath(config.getBaseKey(), getStorageDir(inSegment, useUniquePath));
log.info("Copying segment[%s] to S3 at location[%s]", inSegment.getIdentifier(), s3Path);
log.info("Copying segment[%s] to S3 at location[%s]", inSegment.getId(), s3Path);
final File zipOutFile = File.createTempFile("druid", "index.zip");
final long indexSize = CompressionUtils.zip(indexFilesDir, zipOutFile);

View File

@ -189,18 +189,18 @@ public class S3DataSegmentFinderTest
DataSegment updatedSegment4_1 = null;
for (DataSegment dataSegment : segments) {
if (dataSegment.getIdentifier().equals(SEGMENT_1.getIdentifier())) {
if (dataSegment.getId().equals(SEGMENT_1.getId())) {
updatedSegment1 = dataSegment;
} else if (dataSegment.getIdentifier().equals(SEGMENT_2.getIdentifier())) {
} else if (dataSegment.getId().equals(SEGMENT_2.getId())) {
updatedSegment2 = dataSegment;
} else if (dataSegment.getIdentifier().equals(SEGMENT_3.getIdentifier())) {
} else if (dataSegment.getId().equals(SEGMENT_3.getId())) {
updatedSegment3 = dataSegment;
} else if (dataSegment.getIdentifier().equals(SEGMENT_4_0.getIdentifier())) {
} else if (dataSegment.getId().equals(SEGMENT_4_0.getId())) {
updatedSegment4_0 = dataSegment;
} else if (dataSegment.getIdentifier().equals(SEGMENT_4_1.getIdentifier())) {
} else if (dataSegment.getId().equals(SEGMENT_4_1.getId())) {
updatedSegment4_1 = dataSegment;
} else {
Assert.fail("Unexpected segment identifier : " + dataSegment.getIdentifier());
Assert.fail("Unexpected segment identifier : " + dataSegment.getId());
}
}

View File

@ -73,7 +73,7 @@ public class VarianceTopNQueryTest
.dimension(QueryRunnerTestHelper.marketDimension)
.metric(QueryRunnerTestHelper.uniqueMetric)
.threshold(3)
.intervals(QueryRunnerTestHelper.fullOnInterval)
.intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(

View File

@ -547,10 +547,7 @@ public class HadoopDruidIndexerConfig
public Path makeDescriptorInfoPath(DataSegment segment)
{
return new Path(
makeDescriptorInfoDir(),
StringUtils.removeChar(StringUtils.format("%s.json", segment.getIdentifier()), ':')
);
return new Path(makeDescriptorInfoDir(), StringUtils.removeChar(segment.getId() + ".json", ':'));
}
public void addJobProperties(Job job)

View File

@ -120,7 +120,7 @@ public class IndexGeneratorJob implements Jobby
for (FileStatus status : fs.listStatus(descriptorInfoDir)) {
final DataSegment segment = jsonMapper.readValue(fs.open(status.getPath()), DataSegment.class);
publishedSegmentsBuilder.add(segment);
log.info("Adding segment %s to the list of published segments", segment.getIdentifier());
log.info("Adding segment %s to the list of published segments", segment.getId());
}
}
catch (FileNotFoundException e) {

View File

@ -83,7 +83,7 @@ public class DatasourceInputFormat extends InputFormat<NullWritable, InputRow>
i,
segments.size(),
dataSource,
segment.getSegment().getIdentifier(),
segment.getSegment().getId(),
segment.getInterval()
);
}

View File

@ -82,7 +82,7 @@ public class DatasourceRecordReader extends RecordReader<NullWritable, InputRow>
public WindowedStorageAdapter apply(WindowedDataSegment segment)
{
try {
logger.info("Getting storage path for segment [%s]", segment.getSegment().getIdentifier());
logger.info("Getting storage path for segment [%s]", segment.getSegment().getId());
Path path = new Path(JobHelper.getURIFromSegment(segment.getSegment()));
logger.info("Fetch segment files from [%s]", path);

View File

@ -114,7 +114,7 @@ public class DetermineHashedPartitionsJobTest
0,
1,
first,
new PeriodGranularity(new Period("P1D"), null, DateTimes.inferTzfromString("America/Los_Angeles"))
new PeriodGranularity(new Period("P1D"), null, DateTimes.inferTzFromString("America/Los_Angeles"))
}
}
);

View File

@ -118,7 +118,7 @@ public class HadoopIngestionSpecTest
Assert.assertEquals(
"getSegmentGranularity",
new PeriodGranularity(new Period("PT1H"), null, DateTimes.inferTzfromString("America/Los_Angeles")),
new PeriodGranularity(new Period("PT1H"), null, DateTimes.inferTzFromString("America/Los_Angeles")),
granularitySpec.getSegmentGranularity()
);
}

View File

@ -24,7 +24,6 @@ import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.segment.IndexSpec;
import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Test;
@ -69,7 +68,7 @@ public class HadoopTuningConfigTest
Assert.assertEquals("/tmp/workingpath", actual.getWorkingPath());
Assert.assertEquals("version", actual.getVersion());
Assert.assertNotNull(actual.getPartitionsSpec());
Assert.assertEquals(ImmutableMap.<DateTime, List<HadoopyShardSpec>>of(), actual.getShardSpecs());
Assert.assertEquals(ImmutableMap.<Long, List<HadoopyShardSpec>>of(), actual.getShardSpecs());
Assert.assertEquals(new IndexSpec(), actual.getIndexSpec());
Assert.assertEquals(100, actual.getRowFlushBoundary());
Assert.assertEquals(true, actual.isLeaveIntermediate());

View File

@ -266,7 +266,7 @@ public class HadoopConverterJobTest
Thread.sleep(10);
}
manager.poll();
final ImmutableDruidDataSource druidDataSource = manager.getInventoryValue(DATASOURCE);
final ImmutableDruidDataSource druidDataSource = manager.getDataSource(DATASOURCE);
manager.stop();
return Lists.newArrayList(druidDataSource.getSegments());
}
@ -333,7 +333,7 @@ public class HadoopConverterJobTest
@Override
public int compare(DataSegment o1, DataSegment o2)
{
return o1.getIdentifier().compareTo(o2.getIdentifier());
return o1.getId().compareTo(o2.getId());
}
};
Collections.sort(

View File

@ -23,7 +23,7 @@ import org.apache.druid.data.input.InputRow;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import java.io.IOException;
@ -45,7 +45,7 @@ public class ActionBasedSegmentAllocator implements SegmentAllocator
}
@Override
public SegmentIdentifier allocate(
public SegmentIdWithShardSpec allocate(
final InputRow row,
final String sequenceName,
final String previousSegmentId,

View File

@ -19,12 +19,11 @@
package org.apache.druid.indexing.appenderator;
import com.google.common.base.Function;
import com.google.common.collect.Iterables;
import org.apache.druid.indexing.common.actions.SegmentListUsedAction;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.java.util.common.JodaUtils;
import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.realtime.appenderator.UsedSegmentChecker;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.Interval;
@ -46,11 +45,11 @@ public class ActionBasedUsedSegmentChecker implements UsedSegmentChecker
}
@Override
public Set<DataSegment> findUsedSegments(Set<SegmentIdentifier> identifiers) throws IOException
public Set<DataSegment> findUsedSegments(Set<SegmentIdWithShardSpec> identifiers) throws IOException
{
// Group by dataSource
final Map<String, Set<SegmentIdentifier>> identifiersByDataSource = new TreeMap<>();
for (SegmentIdentifier identifier : identifiers) {
final Map<String, Set<SegmentIdWithShardSpec>> identifiersByDataSource = new TreeMap<>();
for (SegmentIdWithShardSpec identifier : identifiers) {
if (!identifiersByDataSource.containsKey(identifier.getDataSource())) {
identifiersByDataSource.put(identifier.getDataSource(), new HashSet<>());
}
@ -59,16 +58,9 @@ public class ActionBasedUsedSegmentChecker implements UsedSegmentChecker
final Set<DataSegment> retVal = new HashSet<>();
for (Map.Entry<String, Set<SegmentIdentifier>> entry : identifiersByDataSource.entrySet()) {
for (Map.Entry<String, Set<SegmentIdWithShardSpec>> entry : identifiersByDataSource.entrySet()) {
final List<Interval> intervals = JodaUtils.condenseIntervals(
Iterables.transform(entry.getValue(), new Function<SegmentIdentifier, Interval>()
{
@Override
public Interval apply(SegmentIdentifier input)
{
return input.getInterval();
}
})
Iterables.transform(entry.getValue(), input -> input.getInterval())
);
final List<DataSegment> usedSegmentsForIntervals = taskActionClient.submit(
@ -76,7 +68,7 @@ public class ActionBasedUsedSegmentChecker implements UsedSegmentChecker
);
for (DataSegment segment : usedSegmentsForIntervals) {
if (identifiers.contains(SegmentIdentifier.fromDataSegment(segment))) {
if (identifiers.contains(SegmentIdWithShardSpec.fromDataSegment(segment))) {
retVal.add(segment);
}
}

View File

@ -22,19 +22,19 @@ package org.apache.druid.indexing.appenderator;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.indexing.common.actions.TaskAction;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
/**
* This class is used in {@link ActionBasedSegmentAllocator} and expected to generate a
* {@link TaskAction<SegmentIdentifier>} which is submitted to overlords to allocate a new segment.
* The {@link #generate} method can return any implementation of {@link TaskAction<SegmentIdentifier>}.
* {@link TaskAction< SegmentIdWithShardSpec >} which is submitted to overlords to allocate a new segment.
* The {@link #generate} method can return any implementation of {@link TaskAction< SegmentIdWithShardSpec >}.
*
* @see org.apache.druid.indexing.common.actions.SegmentAllocateAction
* @see org.apache.druid.indexing.common.actions.SurrogateAction
*/
public interface SegmentAllocateActionGenerator
{
TaskAction<SegmentIdentifier> generate(
TaskAction<SegmentIdWithShardSpec> generate(
DataSchema dataSchema,
InputRow row,
String sequenceName,

View File

@ -35,7 +35,7 @@ import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@ -54,7 +54,7 @@ import java.util.stream.Collectors;
* <p/>
* If this action cannot acquire an appropriate lock, or if it cannot expand an existing segment set, it returns null.
*/
public class SegmentAllocateAction implements TaskAction<SegmentIdentifier>
public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
{
private static final Logger log = new Logger(SegmentAllocateAction.class);
@ -134,15 +134,15 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdentifier>
}
@Override
public TypeReference<SegmentIdentifier> getReturnTypeReference()
public TypeReference<SegmentIdWithShardSpec> getReturnTypeReference()
{
return new TypeReference<SegmentIdentifier>()
return new TypeReference<SegmentIdWithShardSpec>()
{
};
}
@Override
public SegmentIdentifier perform(
public SegmentIdWithShardSpec perform(
final Task task,
final TaskActionToolbox toolbox
)
@ -166,9 +166,9 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdentifier>
msc.getUsedSegmentsForInterval(dataSource, rowInterval)
);
final SegmentIdentifier identifier = usedSegmentsForRow.isEmpty() ?
tryAllocateFirstSegment(toolbox, task, rowInterval) :
tryAllocateSubsequentSegment(
final SegmentIdWithShardSpec identifier = usedSegmentsForRow.isEmpty() ?
tryAllocateFirstSegment(toolbox, task, rowInterval) :
tryAllocateSubsequentSegment(
toolbox,
task,
rowInterval,
@ -212,7 +212,7 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdentifier>
}
}
private SegmentIdentifier tryAllocateFirstSegment(TaskActionToolbox toolbox, Task task, Interval rowInterval)
private SegmentIdWithShardSpec tryAllocateFirstSegment(TaskActionToolbox toolbox, Task task, Interval rowInterval)
{
// No existing segments for this row, but there might still be nearby ones that conflict with our preferred
// segment granularity. Try that first, and then progressively smaller ones if it fails.
@ -222,7 +222,7 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdentifier>
.collect(Collectors.toList());
for (Interval tryInterval : tryIntervals) {
if (tryInterval.contains(rowInterval)) {
final SegmentIdentifier identifier = tryAllocate(toolbox, task, tryInterval, rowInterval, false);
final SegmentIdWithShardSpec identifier = tryAllocate(toolbox, task, tryInterval, rowInterval, false);
if (identifier != null) {
return identifier;
}
@ -231,7 +231,7 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdentifier>
return null;
}
private SegmentIdentifier tryAllocateSubsequentSegment(
private SegmentIdWithShardSpec tryAllocateSubsequentSegment(
TaskActionToolbox toolbox,
Task task,
Interval rowInterval,
@ -249,7 +249,7 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdentifier>
}
}
private SegmentIdentifier tryAllocate(
private SegmentIdWithShardSpec tryAllocate(
TaskActionToolbox toolbox,
Task task,
Interval tryInterval,
@ -269,12 +269,13 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdentifier>
}
if (lockResult.isOk()) {
final SegmentIdentifier identifier;
final SegmentIdWithShardSpec identifier;
try {
identifier = toolbox.getTaskLockbox().doInCriticalSection(
task,
ImmutableList.of(tryInterval),
CriticalAction.<SegmentIdentifier>builder()
CriticalAction
.<SegmentIdWithShardSpec>builder()
.onValidLocks(
() -> toolbox.getIndexerMetadataStorageCoordinator().allocatePendingSegment(
dataSource,
@ -284,9 +285,8 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdentifier>
lockResult.getTaskLock().getVersion(),
skipSegmentLineageCheck
)
).onInvalidLocks(
() -> null
)
)
.onInvalidLocks(() -> null)
.build()
);
}

View File

@ -23,9 +23,9 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.DataSegmentUtils;
import java.util.Set;
@ -84,7 +84,7 @@ public class SegmentInsertAction implements TaskAction<Set<DataSegment>>
public String toString()
{
return "SegmentInsertAction{" +
"segments=" + DataSegmentUtils.getIdentifiersString(segments) +
"segments=" + Iterables.transform(segments, DataSegment::getId) +
'}';
}
}

View File

@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import org.apache.druid.indexing.common.task.IndexTaskUtils;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.CriticalAction;
@ -31,7 +32,6 @@ import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.query.DruidMetrics;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.DataSegmentUtils;
import org.joda.time.Interval;
import java.util.List;
@ -115,7 +115,7 @@ public class SegmentMetadataUpdateAction implements TaskAction<Void>
public String toString()
{
return "SegmentMetadataUpdateAction{" +
"segments=" + DataSegmentUtils.getIdentifiersString(segments) +
"segments=" + Iterables.transform(segments, DataSegment::getId) +
'}';
}
}

View File

@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import org.apache.druid.indexing.common.task.IndexTaskUtils;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.CriticalAction;
@ -31,7 +32,6 @@ import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.query.DruidMetrics;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.DataSegmentUtils;
import org.joda.time.Interval;
import java.util.List;
@ -117,7 +117,7 @@ public class SegmentNukeAction implements TaskAction<Void>
public String toString()
{
return "SegmentNukeAction{" +
"segments=" + DataSegmentUtils.getIdentifiersString(segments) +
"segments=" + Iterables.transform(segments, DataSegment::getId) +
'}';
}
}

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import org.apache.druid.indexing.common.task.IndexTaskUtils;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.CriticalAction;
@ -31,7 +32,6 @@ import org.apache.druid.indexing.overlord.SegmentPublishResult;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.query.DruidMetrics;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.DataSegmentUtils;
import java.util.Set;
import java.util.stream.Collectors;
@ -155,7 +155,7 @@ public class SegmentTransactionalInsertAction implements TaskAction<SegmentPubli
public String toString()
{
return "SegmentInsertAction{" +
"segments=" + DataSegmentUtils.getIdentifiersString(segments) +
"segments=" + Iterables.transform(segments, DataSegment::getId) +
", startMetadata=" + startMetadata +
", endMetadata=" + endMetadata +
'}';

View File

@ -113,7 +113,7 @@ public class YeOldePlumberSchool implements PlumberSchool
);
// Temporary directory to hold spilled segments.
final File persistDir = new File(tmpSegmentDir, theSink.getSegment().getIdentifier());
final File persistDir = new File(tmpSegmentDir, theSink.getSegment().getId().toString());
// Set of spilled segments. Will be merged at the end.
final Set<File> spilled = new HashSet<>();
@ -205,11 +205,7 @@ public class YeOldePlumberSchool implements PlumberSchool
dataSegmentPusher.push(fileToUpload, segmentToUpload, false);
log.info(
"Uploaded segment[%s]",
segmentToUpload.getIdentifier()
);
log.info("Uploaded segment[%s]", segmentToUpload.getId());
}
catch (Exception e) {
log.warn(e, "Failed to merge and upload");

View File

@ -98,7 +98,7 @@ public class AppendTask extends MergeTaskBase
input.getInterval(),
Preconditions.checkNotNull(
segments.get(segment),
"File for segment %s", segment.getIdentifier()
"File for segment %s", segment.getId()
)
);
}

View File

@ -84,13 +84,13 @@ public class ArchiveTask extends AbstractFixedIntervalTask
if (unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) {
throw new ISE(
"WTF?! Unused segment[%s] has version[%s] > task version[%s]",
unusedSegment.getIdentifier(),
unusedSegment.getId(),
unusedSegment.getVersion(),
myLock.getVersion()
);
}
log.info("OK to archive segment: %s", unusedSegment.getIdentifier());
log.info("OK to archive segment: %s", unusedSegment.getId());
}
// Move segments

View File

@ -489,7 +489,7 @@ public class CompactionTask extends AbstractTask
for (Pair<QueryableIndex, DataSegment> pair : queryableIndexAndSegments) {
final QueryableIndex index = pair.lhs;
if (index.getMetadata() == null) {
throw new RE("Index metadata doesn't exist for segment[%s]", pair.rhs.getIdentifier());
throw new RE("Index metadata doesn't exist for segment[%s]", pair.rhs.getId());
}
}
@ -569,9 +569,9 @@ public class CompactionTask extends AbstractTask
intervalComparator.compare(shouldBeSmaller, shouldBeLarger) <= 0,
"QueryableIndexes are not sorted! Interval[%s] of segment[%s] is laster than interval[%s] of segment[%s]",
shouldBeSmaller,
queryableIndices.get(i).rhs.getIdentifier(),
queryableIndices.get(i).rhs.getId(),
shouldBeLarger,
queryableIndices.get(i + 1).rhs.getIdentifier()
queryableIndices.get(i + 1).rhs.getId()
);
}
@ -636,7 +636,7 @@ public class CompactionTask extends AbstractTask
for (PartitionChunk<DataSegment> chunk : partitionHolder) {
final DataSegment segment = chunk.getObject();
final QueryableIndex queryableIndex = indexIO.loadIndex(
Preconditions.checkNotNull(segmentFileMap.get(segment), "File for segment %s", segment.getIdentifier())
Preconditions.checkNotNull(segmentFileMap.get(segment), "File for segment %s", segment.getId())
);
segments.add(Pair.of(queryableIndex, segment));
}

View File

@ -25,12 +25,11 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Joiner;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.hash.HashFunction;
import com.google.common.hash.Hashing;
import com.google.common.util.concurrent.ListenableFuture;
@ -80,7 +79,7 @@ import org.apache.druid.segment.realtime.appenderator.Appenderators;
import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver;
import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver;
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata;
import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
import org.apache.druid.segment.realtime.firehose.ChatHandler;
@ -910,7 +909,7 @@ public class IndexTask extends AbstractTask implements ChatHandler
final SegmentAllocator segmentAllocator;
if (isGuaranteedRollup) {
// Overwrite mode, guaranteed rollup: segments are all known in advance and there is one per sequenceName.
final Map<String, SegmentIdentifier> lookup = new HashMap<>();
final Map<String, SegmentIdWithShardSpec> lookup = new HashMap<>();
for (Map.Entry<Interval, List<ShardSpec>> entry : shardSpecs.getMap().entrySet()) {
for (ShardSpec shardSpec : entry.getValue()) {
@ -928,7 +927,7 @@ public class IndexTask extends AbstractTask implements ChatHandler
final String version = findVersion(versions, entry.getKey());
lookup.put(
Appenderators.getSequenceName(entry.getKey(), version, shardSpec),
new SegmentIdentifier(getDataSource(), entry.getKey(), version, shardSpecForPublishing)
new SegmentIdWithShardSpec(getDataSource(), entry.getKey(), version, shardSpecForPublishing)
);
}
}
@ -966,7 +965,7 @@ public class IndexTask extends AbstractTask implements ChatHandler
}
final int partitionNum = counters.computeIfAbsent(interval, x -> new AtomicInteger()).getAndIncrement();
return new SegmentIdentifier(
return new SegmentIdWithShardSpec(
getDataSource(),
interval,
findVersion(versions, interval),
@ -1077,14 +1076,7 @@ public class IndexTask extends AbstractTask implements ChatHandler
buildSegmentsMeters.getUnparseable(),
buildSegmentsMeters.getThrownAway()
);
log.info(
"Published segments[%s]", Joiner.on(", ").join(
Iterables.transform(
published.getSegments(),
DataSegment::getIdentifier
)
)
);
log.info("Published segments: %s", Lists.transform(published.getSegments(), DataSegment::getId));
toolbox.getTaskReportFileWriter().write(getTaskCompletionReports());
return TaskStatus.success(getId());

View File

@ -88,13 +88,13 @@ public class KillTask extends AbstractFixedIntervalTask
if (unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) {
throw new ISE(
"WTF?! Unused segment[%s] has version[%s] > task version[%s]",
unusedSegment.getIdentifier(),
unusedSegment.getId(),
unusedSegment.getVersion(),
myLock.getVersion()
);
}
log.info("OK to kill segment: %s", unusedSegment.getIdentifier());
log.info("OK to kill segment: %s", unusedSegment.getId());
}
// Kill segments

View File

@ -27,7 +27,6 @@ import com.google.common.base.Objects;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Ordering;
@ -47,6 +46,7 @@ import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.partition.NoneShardSpec;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@ -58,6 +58,7 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
import java.util.stream.Collectors;
/**
*/
@ -151,21 +152,7 @@ public abstract class MergeTaskBase extends AbstractFixedIntervalTask
try {
final long startTime = System.currentTimeMillis();
log.info(
"Starting merge of id[%s], segments: %s",
getId(),
Lists.transform(
segments,
new Function<DataSegment, String>()
{
@Override
public String apply(DataSegment input)
{
return input.getIdentifier();
}
}
)
);
log.info("Starting merge of id[%s], segments: %s", getId(), Lists.transform(segments, DataSegment::getId));
// download segments to merge
final Map<DataSegment, File> gettedSegments = toolbox.fetchSegments(segments);
@ -216,24 +203,16 @@ public abstract class MergeTaskBase extends AbstractFixedIntervalTask
if (!super.isReady(taskActionClient)) {
return false;
} else {
final Function<DataSegment, String> toIdentifier = new Function<DataSegment, String>()
{
@Override
public String apply(DataSegment dataSegment)
{
return dataSegment.getIdentifier();
}
};
final Set<String> current = ImmutableSet.copyOf(
Iterables.transform(
taskActionClient.submit(new SegmentListUsedAction(getDataSource(), getInterval(), null)),
toIdentifier
)
);
final Set<String> requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier));
final Set<SegmentId> current = taskActionClient
.submit(new SegmentListUsedAction(getDataSource(), getInterval(), null))
.stream()
.map(DataSegment::getId)
.collect(Collectors.toSet());
final Set<String> missingFromRequested = Sets.difference(current, requested);
final Set<SegmentId> requested = segments.stream().map(DataSegment::getId).collect(Collectors.toSet());
final Set<SegmentId> missingFromRequested = Sets.difference(current, requested);
if (!missingFromRequested.isEmpty()) {
throw new ISE(
"Merge is invalid: current segment(s) are not in the requested set: %s",
@ -241,7 +220,7 @@ public abstract class MergeTaskBase extends AbstractFixedIntervalTask
);
}
final Set<String> missingFromCurrent = Sets.difference(requested, current);
final Set<SegmentId> missingFromCurrent = Sets.difference(requested, current);
if (!missingFromCurrent.isEmpty()) {
throw new ISE(
"Merge is invalid: requested segment(s) are not in the current set: %s",

View File

@ -92,13 +92,13 @@ public class MoveTask extends AbstractFixedIntervalTask
if (unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) {
throw new ISE(
"WTF?! Unused segment[%s] has version[%s] > task version[%s]",
unusedSegment.getIdentifier(),
unusedSegment.getId(),
unusedSegment.getVersion(),
myLock.getVersion()
);
}
log.info("OK to move segment: %s", unusedSegment.getIdentifier());
log.info("OK to move segment: %s", unusedSegment.getId());
}
// Move segments

View File

@ -85,13 +85,13 @@ public class RestoreTask extends AbstractFixedIntervalTask
if (unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) {
throw new ISE(
"WTF?! Unused segment[%s] has version[%s] > task version[%s]",
unusedSegment.getIdentifier(),
unusedSegment.getId(),
unusedSegment.getVersion(),
myLock.getVersion()
);
}
log.info("OK to restore segment: %s", unusedSegment.getIdentifier());
log.info("OK to restore segment: %s", unusedSegment.getId());
}
final List<DataSegment> restoredSegments = new ArrayList<>();

View File

@ -49,7 +49,7 @@ import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.realtime.firehose.ChatHandler;
import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
import org.apache.druid.segment.realtime.firehose.ChatHandlers;
@ -316,7 +316,7 @@ public class ParallelIndexSupervisorTask extends AbstractTask implements ChatHan
// Internal APIs
/**
* Allocate a new {@link SegmentIdentifier} for a request from {@link ParallelIndexSubTask}.
* Allocate a new {@link SegmentIdWithShardSpec} for a request from {@link ParallelIndexSubTask}.
* The returned segmentIdentifiers have different {@code partitionNum} (thereby different {@link NumberedShardSpec})
* per bucket interval.
*/
@ -340,7 +340,7 @@ public class ParallelIndexSupervisorTask extends AbstractTask implements ChatHan
}
try {
final SegmentIdentifier segmentIdentifier = allocateNewSegment(timestamp);
final SegmentIdWithShardSpec segmentIdentifier = allocateNewSegment(timestamp);
return Response.ok(toolbox.getObjectMapper().writeValueAsBytes(segmentIdentifier)).build();
}
catch (IOException | IllegalStateException e) {
@ -352,7 +352,7 @@ public class ParallelIndexSupervisorTask extends AbstractTask implements ChatHan
}
@VisibleForTesting
SegmentIdentifier allocateNewSegment(DateTime timestamp) throws IOException
SegmentIdWithShardSpec allocateNewSegment(DateTime timestamp) throws IOException
{
final String dataSource = getDataSource();
final GranularitySpec granularitySpec = getIngestionSchema().getDataSchema().getGranularitySpec();
@ -378,7 +378,7 @@ public class ParallelIndexSupervisorTask extends AbstractTask implements ChatHan
}
final int partitionNum = counters.increment(interval.toString(), 1);
return new SegmentIdentifier(
return new SegmentIdWithShardSpec(
dataSource,
interval,
findVersion(versions, interval),

View File

@ -26,7 +26,7 @@ import org.apache.druid.indexing.common.TaskInfoProvider;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.http.client.HttpClient;
import org.apache.druid.java.util.http.client.response.FullResponseHolder;
import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.DataSegment;
import org.jboss.netty.handler.codec.http.HttpMethod;
import org.joda.time.DateTime;
@ -57,7 +57,7 @@ public class ParallelIndexTaskClient extends IndexTaskClient
return subtaskId;
}
public SegmentIdentifier allocateSegment(String supervisorTaskId, DateTime timestamp) throws IOException
public SegmentIdWithShardSpec allocateSegment(String supervisorTaskId, DateTime timestamp) throws IOException
{
final FullResponseHolder response = submitSmileRequest(
supervisorTaskId,
@ -77,7 +77,7 @@ public class ParallelIndexTaskClient extends IndexTaskClient
} else {
return deserialize(
response.getContent(),
new TypeReference<SegmentIdentifier>()
new TypeReference<SegmentIdWithShardSpec>()
{
}
);

View File

@ -37,7 +37,7 @@ import org.apache.druid.indexing.common.task.batch.parallel.TaskMonitor.MonitorE
import org.apache.druid.indexing.common.task.batch.parallel.TaskMonitor.SubTaskCompleteEvent;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
import org.apache.druid.segment.realtime.appenderator.UsedSegmentChecker;
import org.apache.druid.timeline.DataSegment;
@ -397,11 +397,11 @@ public class SinglePhaseParallelIndexTaskRunner implements ParallelIndexTaskRunn
log.info("Published [%d] segments", segmentsToPublish.size());
} else {
log.info("Transaction failure while publishing segments, checking if someone else beat us to it.");
final Set<SegmentIdentifier> segmentsIdentifiers = segmentsMap
final Set<SegmentIdWithShardSpec> segmentsIdentifiers = segmentsMap
.values()
.stream()
.flatMap(report -> report.getSegments().stream())
.map(SegmentIdentifier::fromDataSegment)
.map(SegmentIdWithShardSpec::fromDataSegment)
.collect(Collectors.toSet());
if (usedSegmentChecker.findUsedSegments(segmentsIdentifiers)
.equals(segmentsToPublish)) {

View File

@ -179,7 +179,7 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
indexIO.loadIndex(
Preconditions.checkNotNull(
segmentFileMap.get(segment),
"File for segment %s", segment.getIdentifier()
"File for segment %s", segment.getId()
)
)
),

View File

@ -619,11 +619,7 @@ public class TaskLockbox
* @param intervals intervals
* @param action action to be performed inside of the critical section
*/
public <T> T doInCriticalSection(
Task task,
List<Interval> intervals,
CriticalAction<T> action
) throws Exception
public <T> T doInCriticalSection(Task task, List<Interval> intervals, CriticalAction<T> action) throws Exception
{
giant.lock();

View File

@ -55,6 +55,7 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
@ -320,7 +321,7 @@ public class WorkerHolder
public void waitForInitialization() throws InterruptedException
{
if (!syncer.awaitInitialization(3 * syncer.getServerHttpTimeout())) {
if (!syncer.awaitInitialization(3 * syncer.getServerHttpTimeout(), TimeUnit.MILLISECONDS)) {
throw new RE("Failed to sync with worker[%s].", worker.getHost());
}
}

View File

@ -27,12 +27,12 @@ import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Function;
import com.google.common.base.Joiner;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.base.Supplier;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
@ -744,10 +744,8 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
for (SegmentsAndMetadata handedOff : handedOffList) {
log.info(
"Handoff completed for segments[%s] with metadata[%s].",
Joiner.on(", ").join(
handedOff.getSegments().stream().map(DataSegment::getIdentifier).collect(Collectors.toList())
),
"Handoff completed for segments %s with metadata[%s].",
Lists.transform(handedOff.getSegments(), DataSegment::getId),
Preconditions.checkNotNull(handedOff.getCommitMetadata(), "commitMetadata")
);
}
@ -893,11 +891,8 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
public void onSuccess(SegmentsAndMetadata publishedSegmentsAndMetadata)
{
log.info(
"Published segments[%s] with metadata[%s].",
publishedSegmentsAndMetadata.getSegments()
.stream()
.map(DataSegment::getIdentifier)
.collect(Collectors.toList()),
"Published segments %s with metadata[%s].",
Lists.transform(publishedSegmentsAndMetadata.getSegments(), DataSegment::getId),
Preconditions.checkNotNull(publishedSegmentsAndMetadata.getCommitMetadata(), "commitMetadata")
);
@ -922,11 +917,8 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
{
if (handoffSegmentsAndMetadata == null) {
log.warn(
"Failed to handoff segments[%s]",
publishedSegmentsAndMetadata.getSegments()
.stream()
.map(DataSegment::getIdentifier)
.collect(Collectors.toList())
"Failed to handoff segments %s",
Lists.transform(publishedSegmentsAndMetadata.getSegments(), DataSegment::getId)
);
}
handoffFuture.set(handoffSegmentsAndMetadata);

View File

@ -24,7 +24,7 @@ import com.google.common.collect.ImmutableSet;
import org.apache.druid.indexing.common.actions.SegmentListUsedAction;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.realtime.appenderator.UsedSegmentChecker;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.LinearShardSpec;
@ -98,9 +98,9 @@ public class ActionBasedUsedSegmentCheckerTest
final UsedSegmentChecker checker = new ActionBasedUsedSegmentChecker(taskActionClient);
final Set<DataSegment> segments = checker.findUsedSegments(
ImmutableSet.of(
new SegmentIdentifier("foo", Intervals.of("2000/P1D"), "a", new LinearShardSpec(1)),
new SegmentIdentifier("foo", Intervals.of("2001/P1D"), "b", new LinearShardSpec(0)),
new SegmentIdentifier("bar", Intervals.of("2002/P1D"), "b", new LinearShardSpec(0))
new SegmentIdWithShardSpec("foo", Intervals.of("2000/P1D"), "a", new LinearShardSpec(1)),
new SegmentIdWithShardSpec("foo", Intervals.of("2001/P1D"), "b", new LinearShardSpec(0)),
new SegmentIdWithShardSpec("bar", Intervals.of("2002/P1D"), "b", new LinearShardSpec(0))
)
);

View File

@ -35,7 +35,7 @@ import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.LinearShardSpec;
import org.apache.druid.timeline.partition.NumberedShardSpec;
@ -111,7 +111,7 @@ public class SegmentAllocateActionTest
taskActionTestKit.getTaskLockbox().add(task);
final SegmentIdentifier id1 = allocate(
final SegmentIdWithShardSpec id1 = allocate(
task,
PARTY_TIME,
Granularities.NONE,
@ -119,21 +119,21 @@ public class SegmentAllocateActionTest
"s1",
null
);
final SegmentIdentifier id2 = allocate(
final SegmentIdWithShardSpec id2 = allocate(
task,
PARTY_TIME,
Granularities.NONE,
Granularities.HOUR,
"s1",
id1.getIdentifierAsString()
id1.toString()
);
final SegmentIdentifier id3 = allocate(
final SegmentIdWithShardSpec id3 = allocate(
task,
PARTY_TIME,
Granularities.NONE,
Granularities.HOUR,
"s1",
id2.getIdentifierAsString()
id2.toString()
);
final TaskLock partyLock = Iterables.getOnlyElement(
@ -143,7 +143,7 @@ public class SegmentAllocateActionTest
assertSameIdentifier(
id1,
new SegmentIdentifier(
new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
partyLock.getVersion(),
@ -152,7 +152,7 @@ public class SegmentAllocateActionTest
);
assertSameIdentifier(
id2,
new SegmentIdentifier(
new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
partyLock.getVersion(),
@ -161,7 +161,7 @@ public class SegmentAllocateActionTest
);
assertSameIdentifier(
id3,
new SegmentIdentifier(
new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
partyLock.getVersion(),
@ -177,7 +177,7 @@ public class SegmentAllocateActionTest
taskActionTestKit.getTaskLockbox().add(task);
final SegmentIdentifier id1 = allocate(
final SegmentIdWithShardSpec id1 = allocate(
task,
PARTY_TIME,
Granularities.NONE,
@ -185,53 +185,53 @@ public class SegmentAllocateActionTest
"s1",
null
);
final SegmentIdentifier id2 = allocate(
final SegmentIdWithShardSpec id2 = allocate(
task,
THE_DISTANT_FUTURE,
Granularities.NONE,
Granularities.HOUR,
"s1",
id1.getIdentifierAsString()
id1.toString()
);
final SegmentIdentifier id3 = allocate(
final SegmentIdWithShardSpec id3 = allocate(
task,
PARTY_TIME,
Granularities.NONE,
Granularities.HOUR,
"s1",
id2.getIdentifierAsString()
id2.toString()
);
final SegmentIdentifier id4 = allocate(
final SegmentIdWithShardSpec id4 = allocate(
task,
PARTY_TIME,
Granularities.NONE,
Granularities.HOUR,
"s1",
id1.getIdentifierAsString()
id1.toString()
);
final SegmentIdentifier id5 = allocate(
final SegmentIdWithShardSpec id5 = allocate(
task,
THE_DISTANT_FUTURE,
Granularities.NONE,
Granularities.HOUR,
"s1",
id1.getIdentifierAsString()
id1.toString()
);
final SegmentIdentifier id6 = allocate(
final SegmentIdWithShardSpec id6 = allocate(
task,
THE_DISTANT_FUTURE,
Granularities.NONE,
Granularities.MINUTE,
"s1",
id1.getIdentifierAsString()
id1.toString()
);
final SegmentIdentifier id7 = allocate(
final SegmentIdWithShardSpec id7 = allocate(
task,
THE_DISTANT_FUTURE,
Granularities.NONE,
Granularities.DAY,
"s1",
id1.getIdentifierAsString()
id1.toString()
);
final TaskLock partyLock = Iterables.getOnlyElement(
@ -263,7 +263,7 @@ public class SegmentAllocateActionTest
assertSameIdentifier(
id1,
new SegmentIdentifier(
new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
partyLock.getVersion(),
@ -272,7 +272,7 @@ public class SegmentAllocateActionTest
);
assertSameIdentifier(
id2,
new SegmentIdentifier(
new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(THE_DISTANT_FUTURE),
futureLock.getVersion(),
@ -281,7 +281,7 @@ public class SegmentAllocateActionTest
);
assertSameIdentifier(
id3,
new SegmentIdentifier(
new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
partyLock.getVersion(),
@ -301,33 +301,33 @@ public class SegmentAllocateActionTest
taskActionTestKit.getTaskLockbox().add(task);
final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.HOUR, "s1", null);
final SegmentIdentifier id2 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.HOUR, "s2", null);
final SegmentIdentifier id3 = allocate(
final SegmentIdWithShardSpec id1 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.HOUR, "s1", null);
final SegmentIdWithShardSpec id2 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.HOUR, "s2", null);
final SegmentIdWithShardSpec id3 = allocate(
task,
PARTY_TIME,
Granularities.NONE,
Granularities.HOUR,
"s1",
id1.getIdentifierAsString()
id1.toString()
);
final SegmentIdentifier id4 = allocate(
final SegmentIdWithShardSpec id4 = allocate(
task,
THE_DISTANT_FUTURE,
Granularities.NONE,
Granularities.HOUR,
"s1",
id3.getIdentifierAsString()
id3.toString()
);
final SegmentIdentifier id5 = allocate(
final SegmentIdWithShardSpec id5 = allocate(
task,
THE_DISTANT_FUTURE,
Granularities.NONE,
Granularities.HOUR,
"s2",
id2.getIdentifierAsString()
id2.toString()
);
final SegmentIdentifier id6 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.HOUR, "s1", null);
final SegmentIdWithShardSpec id6 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.HOUR, "s1", null);
final TaskLock partyLock = Iterables.getOnlyElement(
FluentIterable.from(taskActionTestKit.getTaskLockbox().findLocksForTask(task))
@ -358,7 +358,7 @@ public class SegmentAllocateActionTest
assertSameIdentifier(
id1,
new SegmentIdentifier(
new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
partyLock.getVersion(),
@ -367,7 +367,7 @@ public class SegmentAllocateActionTest
);
assertSameIdentifier(
id2,
new SegmentIdentifier(
new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
partyLock.getVersion(),
@ -376,7 +376,7 @@ public class SegmentAllocateActionTest
);
assertSameIdentifier(
id3,
new SegmentIdentifier(
new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
partyLock.getVersion(),
@ -385,7 +385,7 @@ public class SegmentAllocateActionTest
);
assertSameIdentifier(
id4,
new SegmentIdentifier(
new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(THE_DISTANT_FUTURE),
futureLock.getVersion(),
@ -394,7 +394,7 @@ public class SegmentAllocateActionTest
);
assertSameIdentifier(
id5,
new SegmentIdentifier(
new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(THE_DISTANT_FUTURE),
futureLock.getVersion(),
@ -431,7 +431,7 @@ public class SegmentAllocateActionTest
taskActionTestKit.getTaskLockbox().add(task);
final SegmentIdentifier id1 = allocate(
final SegmentIdWithShardSpec id1 = allocate(
task,
PARTY_TIME,
Granularities.NONE,
@ -439,18 +439,18 @@ public class SegmentAllocateActionTest
"s1",
null
);
final SegmentIdentifier id2 = allocate(
final SegmentIdWithShardSpec id2 = allocate(
task,
PARTY_TIME,
Granularities.NONE,
Granularities.HOUR,
"s1",
id1.getIdentifierAsString()
id1.toString()
);
assertSameIdentifier(
id1,
new SegmentIdentifier(
new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
PARTY_TIME.toString(),
@ -459,7 +459,7 @@ public class SegmentAllocateActionTest
);
assertSameIdentifier(
id2,
new SegmentIdentifier(
new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
PARTY_TIME.toString(),
@ -492,7 +492,7 @@ public class SegmentAllocateActionTest
taskActionTestKit.getTaskLockbox().add(task);
final SegmentIdentifier id1 = allocate(
final SegmentIdWithShardSpec id1 = allocate(
task,
PARTY_TIME,
Granularities.NONE,
@ -500,18 +500,18 @@ public class SegmentAllocateActionTest
"s1",
null
);
final SegmentIdentifier id2 = allocate(
final SegmentIdWithShardSpec id2 = allocate(
task,
PARTY_TIME,
Granularities.NONE,
Granularities.HOUR,
"s1",
id1.getIdentifierAsString()
id1.toString()
);
assertSameIdentifier(
id1,
new SegmentIdentifier(
new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
PARTY_TIME.toString(),
@ -520,7 +520,7 @@ public class SegmentAllocateActionTest
);
assertSameIdentifier(
id2,
new SegmentIdentifier(
new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
PARTY_TIME.toString(),
@ -553,11 +553,11 @@ public class SegmentAllocateActionTest
taskActionTestKit.getTaskLockbox().add(task);
final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.DAY, "s1", null);
final SegmentIdWithShardSpec id1 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.DAY, "s1", null);
assertSameIdentifier(
id1,
new SegmentIdentifier(
new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
PARTY_TIME.toString(),
@ -590,11 +590,11 @@ public class SegmentAllocateActionTest
taskActionTestKit.getTaskLockbox().add(task);
final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.MINUTE, "s1", null);
final SegmentIdWithShardSpec id1 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.MINUTE, "s1", null);
assertSameIdentifier(
id1,
new SegmentIdentifier(
new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
PARTY_TIME.toString(),
@ -627,7 +627,7 @@ public class SegmentAllocateActionTest
taskActionTestKit.getTaskLockbox().add(task);
final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularities.DAY, Granularities.DAY, "s1", null);
final SegmentIdWithShardSpec id1 = allocate(task, PARTY_TIME, Granularities.DAY, Granularities.DAY, "s1", null);
Assert.assertNull(id1);
}
@ -638,7 +638,7 @@ public class SegmentAllocateActionTest
final Task task = new NoopTask(null, null, 0, 0, null, null, null);
taskActionTestKit.getTaskLockbox().add(task);
final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularities.DAY, Granularities.HOUR, "s1", null);
final SegmentIdWithShardSpec id1 = allocate(task, PARTY_TIME, Granularities.DAY, Granularities.HOUR, "s1", null);
Assert.assertNull(id1);
}
@ -667,7 +667,7 @@ public class SegmentAllocateActionTest
taskActionTestKit.getTaskLockbox().add(task);
final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.HOUR, "s1", null);
final SegmentIdWithShardSpec id1 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.HOUR, "s1", null);
Assert.assertNull(id1);
}
@ -699,7 +699,7 @@ public class SegmentAllocateActionTest
Assert.assertEquals("prev", action2.getPreviousSegmentId());
}
private SegmentIdentifier allocate(
private SegmentIdWithShardSpec allocate(
final Task task,
final DateTime timestamp,
final Granularity queryGranularity,
@ -720,7 +720,7 @@ public class SegmentAllocateActionTest
return action.perform(task, taskActionTestKit.getTaskActionToolbox());
}
private void assertSameIdentifier(final SegmentIdentifier one, final SegmentIdentifier other)
private void assertSameIdentifier(final SegmentIdWithShardSpec one, final SegmentIdWithShardSpec other)
{
Assert.assertEquals(one, other);
Assert.assertEquals(one.getShardSpec().getPartitionNum(), other.getShardSpec().getPartitionNum());

View File

@ -73,9 +73,7 @@ public class SegmentListActionsTest
expectedUsedSegments.forEach(s -> actionTestKit.getTaskLockbox().unlock(task, s.getInterval()));
expectedUnusedSegments.forEach(
s -> actionTestKit.getMetadataSegmentManager().removeSegment(task.getDataSource(), s.getIdentifier())
);
expectedUnusedSegments.forEach(s -> actionTestKit.getMetadataSegmentManager().removeSegment(s.getId()));
}
private DataSegment createSegment(Interval interval, String version)

View File

@ -87,7 +87,7 @@ import org.apache.druid.segment.loading.SegmentLoader;
import org.apache.druid.segment.loading.SegmentLoaderConfig;
import org.apache.druid.segment.loading.SegmentLoaderLocalCacheManager;
import org.apache.druid.segment.loading.StorageLocationConfig;
import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory;
import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter;
import org.apache.druid.segment.transform.ExpressionTransform;
@ -1584,7 +1584,7 @@ public class IndexTaskTest
SegmentAllocateAction action = (SegmentAllocateAction) taskAction;
Interval interval = action.getPreferredSegmentGranularity().bucket(action.getTimestamp());
ShardSpec shardSpec = new NumberedShardSpec(segmentAllocatePartitionCounter++, 0);
return (RetType) new SegmentIdentifier(action.getDataSource(), interval, "latestVersion", shardSpec);
return (RetType) new SegmentIdWithShardSpec(action.getDataSource(), interval, "latestVersion", shardSpec);
}
return null;

View File

@ -52,7 +52,7 @@ import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.segment.loading.DataSegmentKiller;
import org.apache.druid.segment.loading.LocalDataSegmentPusher;
import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig;
import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider;
import org.apache.druid.server.security.AllowAllAuthorizer;
import org.apache.druid.server.security.Authorizer;
@ -386,7 +386,7 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase
}
@Override
public SegmentIdentifier allocateSegment(String supervisorTaskId, DateTime timestamp) throws IOException
public SegmentIdWithShardSpec allocateSegment(String supervisorTaskId, DateTime timestamp) throws IOException
{
return supervisorTask.allocateNewSegment(timestamp);
}

View File

@ -66,6 +66,7 @@ import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory;
import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.server.metrics.NoopServiceEmitter;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.partition.LinearShardSpec;
import org.easymock.EasyMock;
import org.joda.time.Interval;
@ -398,15 +399,15 @@ public class IngestSegmentFirehoseFactoryTimelineTest
@Override
public String toString()
{
final List<String> segmentIdentifiers = new ArrayList<>();
final List<SegmentId> segmentIds = new ArrayList<>();
for (DataSegment segment : segments) {
segmentIdentifiers.add(segment.getIdentifier());
segmentIds.add(segment.getId());
}
return "TestCase{" +
"interval=" + interval +
", expectedCount=" + expectedCount +
", expectedSum=" + expectedSum +
", segments=" + segmentIdentifiers +
", segments=" + segmentIds +
'}';
}
}

View File

@ -26,7 +26,7 @@ import org.apache.druid.indexing.overlord.DataSourceMetadata;
import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
import org.apache.druid.indexing.overlord.SegmentPublishResult;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.Interval;
@ -120,7 +120,7 @@ public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataSto
}
@Override
public SegmentIdentifier allocatePendingSegment(
public SegmentIdWithShardSpec allocatePendingSegment(
String dataSource,
String sequenceName,
String previousSegmentId,

View File

@ -181,7 +181,7 @@ public class ITKafkaIndexingServiceTest extends AbstractIndexerTest
new StringSerializer()
);
DateTimeZone zone = DateTimes.inferTzfromString("UTC");
DateTimeZone zone = DateTimes.inferTzFromString("UTC");
// format for putting into events
DateTimeFormatter event_fmt = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss'Z'");

View File

@ -159,7 +159,7 @@ public class ITKafkaTest extends AbstractIndexerTest
new StringSerializer()
);
DateTimeZone zone = DateTimes.inferTzfromString("UTC");
DateTimeZone zone = DateTimes.inferTzFromString("UTC");
// format for putting into events
DateTimeFormatter event_fmt = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss'Z'");

View File

@ -56,7 +56,7 @@ public class DruidDefaultSerializersModule extends SimpleModule
throws IOException
{
String tzId = jp.getText();
return DateTimes.inferTzfromString(tzId);
return DateTimes.inferTzFromString(tzId);
}
}
);

View File

@ -21,6 +21,7 @@ package org.apache.druid.query;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.timeline.SegmentId;
import org.joda.time.DateTime;
import java.util.Collections;
@ -31,17 +32,13 @@ import java.util.Map;
*/
public class BySegmentQueryRunner<T> implements QueryRunner<T>
{
private final String segmentIdentifier;
private final SegmentId segmentId;
private final DateTime timestamp;
private final QueryRunner<T> base;
public BySegmentQueryRunner(
String segmentIdentifier,
DateTime timestamp,
QueryRunner<T> base
)
public BySegmentQueryRunner(SegmentId segmentId, DateTime timestamp, QueryRunner<T> base)
{
this.segmentIdentifier = segmentIdentifier;
this.segmentId = segmentId;
this.timestamp = timestamp;
this.base = base;
}
@ -55,11 +52,11 @@ public class BySegmentQueryRunner<T> implements QueryRunner<T>
final List<T> results = baseSequence.toList();
return Sequences.simple(
Collections.singletonList(
(T) new Result<BySegmentResultValueClass<T>>(
(T) new Result<>(
timestamp,
new BySegmentResultValueClass<T>(
new BySegmentResultValueClass<>(
results,
segmentIdentifier,
segmentId.toString(),
queryPlus.getQuery().getIntervals().get(0)
)
)

View File

@ -31,6 +31,15 @@ import java.util.stream.Collectors;
public class BySegmentResultValueClass<T> implements BySegmentResultValue<T>
{
private final List<T> results;
/**
* Segment id is stored as a String rather than {@link org.apache.druid.timeline.SegmentId}, because when a
* BySegmentResultValueClass object is sent across Druid nodes, on the reciever (deserialization) side it's impossible
* to unambiguously convert a segment id string (as transmitted in the JSON format) back into a {@code SegmentId}
* object ({@link org.apache.druid.timeline.SegmentId#tryParse} javadoc explains that ambiguities in details). It
* would be fine to have the type of this field of Object, setting it to {@code SegmentId} on the sender side and
* remaining as a String on the reciever side, but it's even less type-safe than always storing the segment id as
* a String.
*/
private final String segmentId;
private final Interval interval;

View File

@ -85,7 +85,7 @@ public class FinalizeResultsQueryRunner<T> implements QueryRunner<T>
BySegmentResultValue<T> resultsClass = result.getValue();
return (T) new Result<BySegmentResultValueClass>(
return (T) new Result<>(
result.getTimestamp(),
new BySegmentResultValueClass(
Lists.transform(resultsClass.getResults(), baseFinalizer),

View File

@ -37,10 +37,7 @@ public class Result<T> implements Comparable<Result<T>>
private final T value;
@JsonCreator
public Result(
@JsonProperty("timestamp") DateTime timestamp,
@JsonProperty("result") T value
)
public Result(@JsonProperty("timestamp") DateTime timestamp, @JsonProperty("result") T value)
{
this.timestamp = timestamp;
this.value = value;

Some files were not shown because too many files have changed in this diff Show More