Merge branch 'master' into 6088-Time-Ordering-On-Scans-N-Way-Merge

This commit is contained in:
Justin Borromeo 2019-02-26 16:39:16 -08:00
commit 5bd0e1a32c
89 changed files with 3405 additions and 2858 deletions

View File

@ -291,5 +291,10 @@ codestyle/checkstyle.xml.
"/>
<property name="illegalPattern" value="true"/> <property name="illegalPattern" value="true"/>
<property name="message" value="Duplicate line"/> <property name="message" value="Duplicate line"/>
</module> </module>
<!-- Added as per the issue #6936 - Prohibit method names starting with capital letters -->
<module name="MethodName">
<property name = "format" value = "^[a-z_]*[a-z0-9][a-zA-Z0-9_]*$"/>
</module>
</module> </module>
</module> </module>

View File

@ -107,7 +107,17 @@ public final class DateTimes
public static DateTime of(String instant) public static DateTime of(String instant)
{ {
return new DateTime(instant, ISOChronology.getInstanceUTC()); try {
return new DateTime(instant, ISOChronology.getInstanceUTC());
}
catch (IllegalArgumentException ex) {
try {
return new DateTime(Long.valueOf(instant), ISOChronology.getInstanceUTC());
}
catch (IllegalArgumentException ex2) {
throw ex;
}
}
} }
public static DateTime of( public static DateTime of(

View File

@ -0,0 +1,43 @@
/*
* 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 com.google.common.base.Preconditions;
import java.net.URI;
public final class URIs
{
public static URI parse(String strUri, String defaultScheme)
{
Preconditions.checkNotNull(strUri, "strUri");
Preconditions.checkNotNull(defaultScheme, "defaultScheme");
final String[] tokens = strUri.split("://");
if (tokens.length == 1) {
return URI.create(StringUtils.format("%s://%s", defaultScheme, strUri));
} else {
return URI.create(strUri);
}
}
private URIs()
{
}
}

View File

@ -37,4 +37,23 @@ public class DateTimesTest
Assert.assertTrue(DateTimes.COMMON_DATE_TIME_PATTERN.matcher(dt.toString()).matches()); Assert.assertTrue(DateTimes.COMMON_DATE_TIME_PATTERN.matcher(dt.toString()).matches());
} }
} }
@Test
public void testStringToDateTimeConversion()
{
String seconds = "2018-01-30T06:00:00";
DateTime dt2 = DateTimes.of(seconds);
Assert.assertEquals("2018-01-30T06:00:00.000Z", dt2.toString());
String milis = "1517292000000";
DateTime dt1 = DateTimes.of(milis);
Assert.assertEquals("2018-01-30T06:00:00.000Z", dt1.toString());
}
@Test(expected = IllegalArgumentException.class)
public void testStringToDateTimeConverstion_RethrowInitialException()
{
String invalid = "51729200AZ";
DateTimes.of(invalid);
}
} }

View File

@ -0,0 +1,70 @@
/*
* 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.junit.Assert;
import org.junit.Test;
import java.net.URI;
public class URIsTest
{
@Test
public void testFullUri()
{
final String strUri = "https://test-user@127.0.0.1:8000/test/path?test-query#test-fragment";
final URI uri = URIs.parse(strUri, "http");
Assert.assertEquals("https", uri.getScheme());
Assert.assertEquals("test-user", uri.getUserInfo());
Assert.assertEquals("127.0.0.1", uri.getHost());
Assert.assertEquals(8000, uri.getPort());
Assert.assertEquals("/test/path", uri.getPath());
Assert.assertEquals("test-query", uri.getQuery());
Assert.assertEquals("test-fragment", uri.getFragment());
}
@Test
public void testWithoutScheme()
{
final String strUri = "test-user@127.0.0.1:8000/test/path?test-query#test-fragment";
final URI uri = URIs.parse(strUri, "http");
Assert.assertEquals("http", uri.getScheme());
Assert.assertEquals("test-user", uri.getUserInfo());
Assert.assertEquals("127.0.0.1", uri.getHost());
Assert.assertEquals(8000, uri.getPort());
Assert.assertEquals("/test/path", uri.getPath());
Assert.assertEquals("test-query", uri.getQuery());
Assert.assertEquals("test-fragment", uri.getFragment());
}
@Test
public void testSimpleUri()
{
final String strUri = "127.0.0.1:8000";
final URI uri = URIs.parse(strUri, "https");
Assert.assertEquals("https", uri.getScheme());
Assert.assertNull(uri.getUserInfo());
Assert.assertEquals("127.0.0.1", uri.getHost());
Assert.assertEquals(8000, uri.getPort());
}
}

View File

@ -274,7 +274,7 @@ public class HllSketchSqlAggregatorTest extends CalciteTestBase
null, null,
null null
), ),
BaseCalciteQueryTest.NOT(BaseCalciteQueryTest.SELECTOR("dim2", "", null)) BaseCalciteQueryTest.not(BaseCalciteQueryTest.selector("dim2", "", null))
), ),
new HllSketchBuildAggregatorFactory( new HllSketchBuildAggregatorFactory(
"a3", "a3",

View File

@ -342,7 +342,7 @@ public class DoublesSketchAggregatorTest
} }
@Test @Test
public void QueryingDataWithFieldNameValueAsFloatInsteadOfSketch() throws Exception public void queryingDataWithFieldNameValueAsFloatInsteadOfSketch() throws Exception
{ {
Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment( Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
new File(this.getClass().getClassLoader().getResource("quantiles/doubles_build_data.tsv").getFile()), new File(this.getClass().getClassLoader().getResource("quantiles/doubles_build_data.tsv").getFile()),
@ -418,7 +418,7 @@ public class DoublesSketchAggregatorTest
} }
@Test @Test
public void TimeSeriesQueryInputAsFloat() throws Exception public void timeSeriesQueryInputAsFloat() throws Exception
{ {
Sequence<Row> seq = timeSeriesHelper.createIndexAndRunQueryOnSegment( Sequence<Row> seq = timeSeriesHelper.createIndexAndRunQueryOnSegment(
new File(this.getClass().getClassLoader().getResource("quantiles/doubles_build_data.tsv").getFile()), new File(this.getClass().getClassLoader().getResource("quantiles/doubles_build_data.tsv").getFile()),

View File

@ -278,7 +278,7 @@ public class ThetaSketchSqlAggregatorTest extends CalciteTestBase
null, null,
null null
), ),
BaseCalciteQueryTest.NOT(BaseCalciteQueryTest.SELECTOR("dim2", "", null)) BaseCalciteQueryTest.not(BaseCalciteQueryTest.selector("dim2", "", null))
), ),
new SketchMergeAggregatorFactory( new SketchMergeAggregatorFactory(
"a3", "a3",

View File

@ -114,12 +114,12 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
ImmutableList.of( ImmutableList.of(
Druids.newTimeseriesQueryBuilder() Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1) .dataSource(CalciteTests.DATASOURCE1)
.intervals(QSS(Filtration.eternity())) .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL) .granularity(Granularities.ALL)
.filters( .filters(
new BloomDimFilter("dim1", BloomKFilterHolder.fromBloomKFilter(filter), null) new BloomDimFilter("dim1", BloomKFilterHolder.fromBloomKFilter(filter), null)
) )
.aggregators(AGGS(new CountAggregatorFactory("a0"))) .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT) .context(TIMESERIES_CONTEXT_DEFAULT)
.build() .build()
), ),
@ -146,7 +146,7 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
ImmutableList.of( ImmutableList.of(
Druids.newTimeseriesQueryBuilder() Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1) .dataSource(CalciteTests.DATASOURCE1)
.intervals(QSS(Filtration.eternity())) .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL) .granularity(Granularities.ALL)
.virtualColumns() .virtualColumns()
.filters( .filters(
@ -155,7 +155,7 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
createExprMacroTable() createExprMacroTable()
) )
) )
.aggregators(AGGS(new CountAggregatorFactory("a0"))) .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT) .context(TIMESERIES_CONTEXT_DEFAULT)
.build() .build()
), ),
@ -178,7 +178,7 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
ImmutableList.of( ImmutableList.of(
Druids.newTimeseriesQueryBuilder() Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1) .dataSource(CalciteTests.DATASOURCE1)
.intervals(QSS(Filtration.eternity())) .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL) .granularity(Granularities.ALL)
.virtualColumns() .virtualColumns()
.filters( .filters(
@ -187,7 +187,7 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
createExprMacroTable() createExprMacroTable()
) )
) )
.aggregators(AGGS(new CountAggregatorFactory("a0"))) .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT) .context(TIMESERIES_CONTEXT_DEFAULT)
.build() .build()
), ),
@ -214,7 +214,7 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
ImmutableList.of( ImmutableList.of(
Druids.newTimeseriesQueryBuilder() Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1) .dataSource(CalciteTests.DATASOURCE1)
.intervals(QSS(Filtration.eternity())) .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL) .granularity(Granularities.ALL)
.filters( .filters(
new OrDimFilter( new OrDimFilter(
@ -222,7 +222,7 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
new BloomDimFilter("dim2", BloomKFilterHolder.fromBloomKFilter(filter2), null) new BloomDimFilter("dim2", BloomKFilterHolder.fromBloomKFilter(filter2), null)
) )
) )
.aggregators(AGGS(new CountAggregatorFactory("a0"))) .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT) .context(TIMESERIES_CONTEXT_DEFAULT)
.build() .build()
), ),

View File

@ -28,10 +28,10 @@ import java.util.Map;
public class KafkaDataSourceMetadataTest public class KafkaDataSourceMetadataTest
{ {
private static final KafkaDataSourceMetadata KM0 = KM("foo", ImmutableMap.of()); private static final KafkaDataSourceMetadata KM0 = km("foo", ImmutableMap.of());
private static final KafkaDataSourceMetadata KM1 = KM("foo", ImmutableMap.of(0, 2L, 1, 3L)); private static final KafkaDataSourceMetadata KM1 = km("foo", ImmutableMap.of(0, 2L, 1, 3L));
private static final KafkaDataSourceMetadata KM2 = KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)); private static final KafkaDataSourceMetadata KM2 = km("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L));
private static final KafkaDataSourceMetadata KM3 = KM("foo", ImmutableMap.of(0, 2L, 2, 5L)); private static final KafkaDataSourceMetadata KM3 = km("foo", ImmutableMap.of(0, 2L, 2, 5L));
@Test @Test
public void testMatches() public void testMatches()
@ -70,27 +70,27 @@ public class KafkaDataSourceMetadataTest
public void testPlus() public void testPlus()
{ {
Assert.assertEquals( Assert.assertEquals(
KM("foo", ImmutableMap.of(0, 2L, 1, 3L, 2, 5L)), km("foo", ImmutableMap.of(0, 2L, 1, 3L, 2, 5L)),
KM1.plus(KM3) KM1.plus(KM3)
); );
Assert.assertEquals( Assert.assertEquals(
KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)), km("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)),
KM0.plus(KM2) KM0.plus(KM2)
); );
Assert.assertEquals( Assert.assertEquals(
KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)), km("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)),
KM1.plus(KM2) KM1.plus(KM2)
); );
Assert.assertEquals( Assert.assertEquals(
KM("foo", ImmutableMap.of(0, 2L, 1, 3L, 2, 5L)), km("foo", ImmutableMap.of(0, 2L, 1, 3L, 2, 5L)),
KM2.plus(KM1) KM2.plus(KM1)
); );
Assert.assertEquals( Assert.assertEquals(
KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)), km("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)),
KM2.plus(KM2) KM2.plus(KM2)
); );
} }
@ -99,32 +99,32 @@ public class KafkaDataSourceMetadataTest
public void testMinus() public void testMinus()
{ {
Assert.assertEquals( Assert.assertEquals(
KM("foo", ImmutableMap.of(1, 3L)), km("foo", ImmutableMap.of(1, 3L)),
KM1.minus(KM3) KM1.minus(KM3)
); );
Assert.assertEquals( Assert.assertEquals(
KM("foo", ImmutableMap.of()), km("foo", ImmutableMap.of()),
KM0.minus(KM2) KM0.minus(KM2)
); );
Assert.assertEquals( Assert.assertEquals(
KM("foo", ImmutableMap.of()), km("foo", ImmutableMap.of()),
KM1.minus(KM2) KM1.minus(KM2)
); );
Assert.assertEquals( Assert.assertEquals(
KM("foo", ImmutableMap.of(2, 5L)), km("foo", ImmutableMap.of(2, 5L)),
KM2.minus(KM1) KM2.minus(KM1)
); );
Assert.assertEquals( Assert.assertEquals(
KM("foo", ImmutableMap.of()), km("foo", ImmutableMap.of()),
KM2.minus(KM2) KM2.minus(KM2)
); );
} }
private static KafkaDataSourceMetadata KM(String topic, Map<Integer, Long> offsets) private static KafkaDataSourceMetadata km(String topic, Map<Integer, Long> offsets)
{ {
return new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, offsets)); return new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, offsets));
} }

View File

@ -139,7 +139,6 @@ import org.apache.druid.segment.loading.DataSegmentPusher;
import org.apache.druid.segment.loading.LocalDataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusher;
import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig;
import org.apache.druid.segment.loading.SegmentLoaderConfig; 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.loading.StorageLocationConfig;
import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl; import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl;
import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier;
@ -278,21 +277,21 @@ public class KafkaIndexTaskTest
private static List<ProducerRecord<byte[], byte[]>> generateRecords(String topic) private static List<ProducerRecord<byte[], byte[]>> generateRecords(String topic)
{ {
return ImmutableList.of( return ImmutableList.of(
new ProducerRecord<>(topic, 0, null, JB("2008", "a", "y", "10", "20.0", "1.0")), new ProducerRecord<>(topic, 0, null, jb("2008", "a", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, JB("2009", "b", "y", "10", "20.0", "1.0")), new ProducerRecord<>(topic, 0, null, jb("2009", "b", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, JB("2010", "c", "y", "10", "20.0", "1.0")), new ProducerRecord<>(topic, 0, null, jb("2010", "c", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, JB("2011", "d", "y", "10", "20.0", "1.0")), new ProducerRecord<>(topic, 0, null, jb("2011", "d", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, JB("2011", "e", "y", "10", "20.0", "1.0")), new ProducerRecord<>(topic, 0, null, jb("2011", "e", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")), new ProducerRecord<>(topic, 0, null, jb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable")), new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable")),
new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable2")), new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable2")),
new ProducerRecord<>(topic, 0, null, null), new ProducerRecord<>(topic, 0, null, null),
new ProducerRecord<>(topic, 0, null, JB("2013", "f", "y", "10", "20.0", "1.0")), new ProducerRecord<>(topic, 0, null, jb("2013", "f", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, JB("2049", "f", "y", "notanumber", "20.0", "1.0")), new ProducerRecord<>(topic, 0, null, jb("2049", "f", "y", "notanumber", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, JB("2049", "f", "y", "10", "notanumber", "1.0")), new ProducerRecord<>(topic, 0, null, jb("2049", "f", "y", "10", "notanumber", "1.0")),
new ProducerRecord<>(topic, 0, null, JB("2049", "f", "y", "10", "20.0", "notanumber")), new ProducerRecord<>(topic, 0, null, jb("2049", "f", "y", "10", "20.0", "notanumber")),
new ProducerRecord<>(topic, 1, null, JB("2012", "g", "y", "10", "20.0", "1.0")), new ProducerRecord<>(topic, 1, null, jb("2012", "g", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 1, null, JB("2011", "h", "y", "10", "20.0", "1.0")) new ProducerRecord<>(topic, 1, null, jb("2011", "h", "y", "10", "20.0", "1.0"))
); );
} }
@ -412,8 +411,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))), new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
@ -462,8 +461,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))), new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
@ -571,13 +570,13 @@ public class KafkaIndexTaskTest
Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
SegmentDescriptor desc3 = SD(task, "2010/P1D", 0); SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
SegmentDescriptor desc4 = SD(task, "2011/P1D", 0); SegmentDescriptor desc4 = sd(task, "2011/P1D", 0);
SegmentDescriptor desc5 = SD(task, "2011/P1D", 1); SegmentDescriptor desc5 = sd(task, "2011/P1D", 1);
SegmentDescriptor desc6 = SD(task, "2012/P1D", 0); SegmentDescriptor desc6 = sd(task, "2012/P1D", 0);
SegmentDescriptor desc7 = SD(task, "2013/P1D", 0); SegmentDescriptor desc7 = sd(task, "2013/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 2L))), new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 2L))),
@ -697,6 +696,7 @@ public class KafkaIndexTaskTest
} }
final Map<Integer, Long> nextOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); final Map<Integer, Long> nextOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets());
Assert.assertTrue(checkpoint2.getPartitionSequenceNumberMap().equals(nextOffsets)); Assert.assertTrue(checkpoint2.getPartitionSequenceNumberMap().equals(nextOffsets));
task.getRunner().setEndOffsets(nextOffsets, false); task.getRunner().setEndOffsets(nextOffsets, false);
@ -728,15 +728,20 @@ public class KafkaIndexTaskTest
Assert.assertEquals(8, task.getRunner().getRowIngestionMeters().getProcessed()); Assert.assertEquals(8, task.getRunner().getRowIngestionMeters().getProcessed());
Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable());
Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
SegmentDescriptor desc3 = SD(task, "2010/P1D", 0); SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
SegmentDescriptor desc4 = SD(task, "2011/P1D", 0); SegmentDescriptor desc4 = sd(task, "2011/P1D", 0);
SegmentDescriptor desc5 = SD(task, "2011/P1D", 1); SegmentDescriptor desc5 = sd(task, "2011/P1D", 1);
SegmentDescriptor desc6 = SD(task, "2012/P1D", 0); SegmentDescriptor desc6 = sd(task, "2012/P1D", 0);
SegmentDescriptor desc7 = SD(task, "2013/P1D", 0); SegmentDescriptor desc7 = sd(task, "2013/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 2L))),
metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 2L))), new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 2L))),
@ -845,8 +850,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L, 1, 0L))), new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L, 1, 0L))),
@ -866,13 +871,13 @@ public class KafkaIndexTaskTest
} }
List<ProducerRecord<byte[], byte[]>> records = ImmutableList.of( List<ProducerRecord<byte[], byte[]>> records = ImmutableList.of(
new ProducerRecord<>(topic, 0, null, JB("2008", "a", "y", "10", "20.0", "1.0")), new ProducerRecord<>(topic, 0, null, jb("2008", "a", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, JB("2009", "b", "y", "10", "20.0", "1.0")), new ProducerRecord<>(topic, 0, null, jb("2009", "b", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, JB("2010", "c", "y", "10", "20.0", "1.0")), new ProducerRecord<>(topic, 0, null, jb("2010", "c", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, JB("2011", "d", "y", "10", "20.0", "1.0")), new ProducerRecord<>(topic, 0, null, jb("2011", "d", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, JB("2011", "D", "y", "10", "20.0", "1.0")), new ProducerRecord<>(topic, 0, null, jb("2011", "D", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, JB("2012", "e", "y", "10", "20.0", "1.0")), new ProducerRecord<>(topic, 0, null, jb("2012", "e", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, JB("2009", "B", "y", "10", "20.0", "1.0")) new ProducerRecord<>(topic, 0, null, jb("2009", "B", "y", "10", "20.0", "1.0"))
); );
final String baseSequenceName = "sequence0"; final String baseSequenceName = "sequence0";
@ -974,8 +979,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))), new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
@ -1024,9 +1029,9 @@ public class KafkaIndexTaskTest
Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
SegmentDescriptor desc3 = SD(task, "2010/P1D", 0); SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))), new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
@ -1084,7 +1089,7 @@ public class KafkaIndexTaskTest
Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2009/P1D", 0); SegmentDescriptor desc1 = sd(task, "2009/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))), new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
@ -1165,8 +1170,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))), new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
@ -1213,8 +1218,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))), new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
@ -1308,10 +1313,10 @@ public class KafkaIndexTaskTest
Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
SegmentDescriptor desc3 = SD(task, "2013/P1D", 0); SegmentDescriptor desc3 = sd(task, "2013/P1D", 0);
SegmentDescriptor desc4 = SD(task, "2049/P1D", 0); SegmentDescriptor desc4 = sd(task, "2049/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 13L))), new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 13L))),
@ -1464,8 +1469,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata // Check published segments & metadata
SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))), new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
@ -1529,8 +1534,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata, should all be from the first task // Check published segments & metadata, should all be from the first task
SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))), new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
@ -1582,8 +1587,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode());
// Check published segments & metadata // Check published segments & metadata
SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
@ -1600,8 +1605,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata // Check published segments & metadata
SegmentDescriptor desc3 = SD(task2, "2011/P1D", 1); SegmentDescriptor desc3 = sd(task2, "2011/P1D", 1);
SegmentDescriptor desc4 = SD(task2, "2013/P1D", 0); SegmentDescriptor desc4 = sd(task2, "2013/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
@ -1644,11 +1649,11 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata // Check published segments & metadata
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
// desc3 will not be created in KafkaIndexTask (0.12.x) as it does not create per Kafka partition Druid segments // desc3 will not be created in KafkaIndexTask (0.12.x) as it does not create per Kafka partition Druid segments
SegmentDescriptor desc3 = SD(task, "2011/P1D", 1); SegmentDescriptor desc3 = sd(task, "2011/P1D", 1);
SegmentDescriptor desc4 = SD(task, "2012/P1D", 0); SegmentDescriptor desc4 = sd(task, "2012/P1D", 0);
Assert.assertEquals(isIncrementalHandoffSupported Assert.assertEquals(isIncrementalHandoffSupported
? ImmutableSet.of(desc1, desc2, desc4) ? ImmutableSet.of(desc1, desc2, desc4)
: ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); : ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
@ -1723,9 +1728,9 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata // Check published segments & metadata
SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
SegmentDescriptor desc3 = SD(task2, "2012/P1D", 0); SegmentDescriptor desc3 = sd(task2, "2012/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L, 1, 1L))), new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L, 1, 1L))),
@ -1821,8 +1826,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata // Check published segments & metadata
SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 6L))), new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 6L))),
@ -1910,8 +1915,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 6L))), new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 6L))),
@ -2038,8 +2043,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))), new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
@ -2179,10 +2184,10 @@ public class KafkaIndexTaskTest
Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
SegmentDescriptor desc3 = SD(task, "2013/P1D", 0); SegmentDescriptor desc3 = sd(task, "2013/P1D", 0);
SegmentDescriptor desc4 = SD(task, "2049/P1D", 0); SegmentDescriptor desc4 = sd(task, "2049/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 13L))), new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 13L))),
@ -2560,9 +2565,7 @@ public class KafkaIndexTaskTest
this::makeTimeseriesAndScanConglomerate, this::makeTimeseriesAndScanConglomerate,
Execs.directExecutor(), // queryExecutorService Execs.directExecutor(), // queryExecutorService
EasyMock.createMock(MonitorScheduler.class), EasyMock.createMock(MonitorScheduler.class),
new SegmentLoaderFactory( new SegmentLoaderFactory(null, testUtils.getTestObjectMapper()),
new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper())
),
testUtils.getTestObjectMapper(), testUtils.getTestObjectMapper(),
testUtils.getTestIndexIO(), testUtils.getTestIndexIO(),
MapCache.create(1024), MapCache.create(1024),
@ -2673,7 +2676,7 @@ public class KafkaIndexTaskTest
return results.isEmpty() ? 0L : DimensionHandlerUtils.nullToZero(results.get(0).getValue().getLongMetric("rows")); return results.isEmpty() ? 0L : DimensionHandlerUtils.nullToZero(results.get(0).getValue().getLongMetric("rows"));
} }
private static byte[] JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1) private static byte[] jb(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1)
{ {
try { try {
return new ObjectMapper().writeValueAsBytes( return new ObjectMapper().writeValueAsBytes(
@ -2692,7 +2695,7 @@ public class KafkaIndexTaskTest
} }
} }
private SegmentDescriptor SD(final Task task, final String intervalString, final int partitionNum) private SegmentDescriptor sd(final Task task, final String intervalString, final int partitionNum)
{ {
final Interval interval = Intervals.of(intervalString); final Interval interval = Intervals.of(intervalString);
return new SegmentDescriptor(interval, getLock(task, interval).getVersion(), partitionNum); return new SegmentDescriptor(interval, getLock(task, interval).getVersion(), partitionNum);

View File

@ -66,25 +66,25 @@ public class KafkaRecordSupplierTest
private static List<ProducerRecord<byte[], byte[]>> generateRecords(String topic) private static List<ProducerRecord<byte[], byte[]>> generateRecords(String topic)
{ {
return ImmutableList.of( return ImmutableList.of(
new ProducerRecord<>(topic, 0, null, JB("2008", "a", "y", "10", "20.0", "1.0")), new ProducerRecord<>(topic, 0, null, jb("2008", "a", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, JB("2009", "b", "y", "10", "20.0", "1.0")), new ProducerRecord<>(topic, 0, null, jb("2009", "b", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, JB("2010", "c", "y", "10", "20.0", "1.0")), new ProducerRecord<>(topic, 0, null, jb("2010", "c", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, JB("2011", "d", "y", "10", "20.0", "1.0")), new ProducerRecord<>(topic, 0, null, jb("2011", "d", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, JB("2011", "e", "y", "10", "20.0", "1.0")), new ProducerRecord<>(topic, 0, null, jb("2011", "e", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")), new ProducerRecord<>(topic, 0, null, jb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable")), new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable")),
new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable2")), new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable2")),
new ProducerRecord<>(topic, 0, null, null), new ProducerRecord<>(topic, 0, null, null),
new ProducerRecord<>(topic, 0, null, JB("2013", "f", "y", "10", "20.0", "1.0")), new ProducerRecord<>(topic, 0, null, jb("2013", "f", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, JB("2049", "f", "y", "notanumber", "20.0", "1.0")), new ProducerRecord<>(topic, 0, null, jb("2049", "f", "y", "notanumber", "20.0", "1.0")),
new ProducerRecord<>(topic, 1, null, JB("2049", "f", "y", "10", "notanumber", "1.0")), new ProducerRecord<>(topic, 1, null, jb("2049", "f", "y", "10", "notanumber", "1.0")),
new ProducerRecord<>(topic, 1, null, JB("2049", "f", "y", "10", "20.0", "notanumber")), new ProducerRecord<>(topic, 1, null, jb("2049", "f", "y", "10", "20.0", "notanumber")),
new ProducerRecord<>(topic, 1, null, JB("2012", "g", "y", "10", "20.0", "1.0")), new ProducerRecord<>(topic, 1, null, jb("2012", "g", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 1, null, JB("2011", "h", "y", "10", "20.0", "1.0")) new ProducerRecord<>(topic, 1, null, jb("2011", "h", "y", "10", "20.0", "1.0"))
); );
} }
private static byte[] JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1) private static byte[] jb(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1)
{ {
try { try {
return new ObjectMapper().writeValueAsBytes( return new ObjectMapper().writeValueAsBytes(

View File

@ -29,10 +29,10 @@ import java.util.Map;
public class KinesisDataSourceMetadataTest public class KinesisDataSourceMetadataTest
{ {
private static final KinesisDataSourceMetadata KM0 = KM("foo", ImmutableMap.of()); private static final KinesisDataSourceMetadata KM0 = km("foo", ImmutableMap.of());
private static final KinesisDataSourceMetadata KM1 = KM("foo", ImmutableMap.of("0", "2L", "1", "3L")); private static final KinesisDataSourceMetadata KM1 = km("foo", ImmutableMap.of("0", "2L", "1", "3L"));
private static final KinesisDataSourceMetadata KM2 = KM("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")); private static final KinesisDataSourceMetadata KM2 = km("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L"));
private static final KinesisDataSourceMetadata KM3 = KM("foo", ImmutableMap.of("0", "2L", "2", "5L")); private static final KinesisDataSourceMetadata KM3 = km("foo", ImmutableMap.of("0", "2L", "2", "5L"));
@Test @Test
public void testMatches() public void testMatches()
@ -71,27 +71,27 @@ public class KinesisDataSourceMetadataTest
public void testPlus() public void testPlus()
{ {
Assert.assertEquals( Assert.assertEquals(
KM("foo", ImmutableMap.of("0", "2L", "1", "3L", "2", "5L")), km("foo", ImmutableMap.of("0", "2L", "1", "3L", "2", "5L")),
KM1.plus(KM3) KM1.plus(KM3)
); );
Assert.assertEquals( Assert.assertEquals(
KM("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")), km("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")),
KM0.plus(KM2) KM0.plus(KM2)
); );
Assert.assertEquals( Assert.assertEquals(
KM("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")), km("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")),
KM1.plus(KM2) KM1.plus(KM2)
); );
Assert.assertEquals( Assert.assertEquals(
KM("foo", ImmutableMap.of("0", "2L", "1", "3L", "2", "5L")), km("foo", ImmutableMap.of("0", "2L", "1", "3L", "2", "5L")),
KM2.plus(KM1) KM2.plus(KM1)
); );
Assert.assertEquals( Assert.assertEquals(
KM("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")), km("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")),
KM2.plus(KM2) KM2.plus(KM2)
); );
} }
@ -100,32 +100,32 @@ public class KinesisDataSourceMetadataTest
public void testMinus() public void testMinus()
{ {
Assert.assertEquals( Assert.assertEquals(
KM("foo", ImmutableMap.of("1", "3L")), km("foo", ImmutableMap.of("1", "3L")),
KM1.minus(KM3) KM1.minus(KM3)
); );
Assert.assertEquals( Assert.assertEquals(
KM("foo", ImmutableMap.of()), km("foo", ImmutableMap.of()),
KM0.minus(KM2) KM0.minus(KM2)
); );
Assert.assertEquals( Assert.assertEquals(
KM("foo", ImmutableMap.of()), km("foo", ImmutableMap.of()),
KM1.minus(KM2) KM1.minus(KM2)
); );
Assert.assertEquals( Assert.assertEquals(
KM("foo", ImmutableMap.of("2", "5L")), km("foo", ImmutableMap.of("2", "5L")),
KM2.minus(KM1) KM2.minus(KM1)
); );
Assert.assertEquals( Assert.assertEquals(
KM("foo", ImmutableMap.of()), km("foo", ImmutableMap.of()),
KM2.minus(KM2) KM2.minus(KM2)
); );
} }
private static KinesisDataSourceMetadata KM(String stream, Map<String, String> sequences) private static KinesisDataSourceMetadata km(String stream, Map<String, String> sequences)
{ {
return new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, sequences)); return new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, sequences));
} }

View File

@ -135,7 +135,6 @@ import org.apache.druid.segment.loading.DataSegmentPusher;
import org.apache.druid.segment.loading.LocalDataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusher;
import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig;
import org.apache.druid.segment.loading.SegmentLoaderConfig; 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.loading.StorageLocationConfig;
import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl; import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl;
import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
@ -199,26 +198,26 @@ public class KinesisIndexTaskTest extends EasyMockSupport
private static String shardId0 = "0"; private static String shardId0 = "0";
private static KinesisRecordSupplier recordSupplier; private static KinesisRecordSupplier recordSupplier;
private static List<OrderedPartitionableRecord<String, String>> records = ImmutableList.of( private static List<OrderedPartitionableRecord<String, String>> records = ImmutableList.of(
new OrderedPartitionableRecord<>(stream, "1", "0", JB("2008", "a", "y", "10", "20.0", "1.0")), new OrderedPartitionableRecord<>(stream, "1", "0", jb("2008", "a", "y", "10", "20.0", "1.0")),
new OrderedPartitionableRecord<>(stream, "1", "1", JB("2009", "b", "y", "10", "20.0", "1.0")), new OrderedPartitionableRecord<>(stream, "1", "1", jb("2009", "b", "y", "10", "20.0", "1.0")),
new OrderedPartitionableRecord<>(stream, "1", "2", JB("2010", "c", "y", "10", "20.0", "1.0")), new OrderedPartitionableRecord<>(stream, "1", "2", jb("2010", "c", "y", "10", "20.0", "1.0")),
new OrderedPartitionableRecord<>(stream, "1", "3", JB("2011", "d", "y", "10", "20.0", "1.0")), new OrderedPartitionableRecord<>(stream, "1", "3", jb("2011", "d", "y", "10", "20.0", "1.0")),
new OrderedPartitionableRecord<>(stream, "1", "4", JB("2011", "e", "y", "10", "20.0", "1.0")), new OrderedPartitionableRecord<>(stream, "1", "4", jb("2011", "e", "y", "10", "20.0", "1.0")),
new OrderedPartitionableRecord<>( new OrderedPartitionableRecord<>(
stream, stream,
"1", "1",
"5", "5",
JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0") jb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")
), ),
new OrderedPartitionableRecord<>(stream, "1", "6", Collections.singletonList(StringUtils.toUtf8("unparseable"))), new OrderedPartitionableRecord<>(stream, "1", "6", Collections.singletonList(StringUtils.toUtf8("unparseable"))),
new OrderedPartitionableRecord<>(stream, "1", "7", Collections.singletonList(StringUtils.toUtf8("unparseable2"))), new OrderedPartitionableRecord<>(stream, "1", "7", Collections.singletonList(StringUtils.toUtf8("unparseable2"))),
new OrderedPartitionableRecord<>(stream, "1", "8", Collections.singletonList(StringUtils.toUtf8("{}"))), new OrderedPartitionableRecord<>(stream, "1", "8", Collections.singletonList(StringUtils.toUtf8("{}"))),
new OrderedPartitionableRecord<>(stream, "1", "9", JB("2013", "f", "y", "10", "20.0", "1.0")), new OrderedPartitionableRecord<>(stream, "1", "9", jb("2013", "f", "y", "10", "20.0", "1.0")),
new OrderedPartitionableRecord<>(stream, "1", "10", JB("2049", "f", "y", "notanumber", "20.0", "1.0")), new OrderedPartitionableRecord<>(stream, "1", "10", jb("2049", "f", "y", "notanumber", "20.0", "1.0")),
new OrderedPartitionableRecord<>(stream, "1", "11", JB("2049", "f", "y", "10", "notanumber", "1.0")), new OrderedPartitionableRecord<>(stream, "1", "11", jb("2049", "f", "y", "10", "notanumber", "1.0")),
new OrderedPartitionableRecord<>(stream, "1", "12", JB("2049", "f", "y", "10", "20.0", "notanumber")), new OrderedPartitionableRecord<>(stream, "1", "12", jb("2049", "f", "y", "10", "20.0", "notanumber")),
new OrderedPartitionableRecord<>(stream, "0", "0", JB("2012", "g", "y", "10", "20.0", "1.0")), new OrderedPartitionableRecord<>(stream, "0", "0", jb("2012", "g", "y", "10", "20.0", "1.0")),
new OrderedPartitionableRecord<>(stream, "0", "1", JB("2011", "h", "y", "10", "20.0", "1.0")) new OrderedPartitionableRecord<>(stream, "0", "1", jb("2011", "h", "y", "10", "20.0", "1.0"))
); );
private static ServiceEmitter emitter; private static ServiceEmitter emitter;
@ -405,8 +404,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>( new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(
@ -485,8 +484,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2011/P1D", 0); SegmentDescriptor desc1 = sd(task, "2011/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2012/P1D", 0); SegmentDescriptor desc2 = sd(task, "2012/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KinesisDataSourceMetadata( new KinesisDataSourceMetadata(
@ -610,13 +609,13 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
SegmentDescriptor desc3 = SD(task, "2010/P1D", 0); SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
SegmentDescriptor desc4 = SD(task, "2011/P1D", 0); SegmentDescriptor desc4 = sd(task, "2011/P1D", 0);
SegmentDescriptor desc5 = SD(task, "2011/P1D", 1); SegmentDescriptor desc5 = sd(task, "2011/P1D", 1);
SegmentDescriptor desc6 = SD(task, "2012/P1D", 0); SegmentDescriptor desc6 = sd(task, "2012/P1D", 0);
SegmentDescriptor desc7 = SD(task, "2013/P1D", 0); SegmentDescriptor desc7 = sd(task, "2013/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
@ -771,12 +770,12 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
SegmentDescriptor desc3 = SD(task, "2010/P1D", 0); SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
SegmentDescriptor desc4 = SD(task, "2011/P1D", 0); SegmentDescriptor desc4 = sd(task, "2011/P1D", 0);
SegmentDescriptor desc5 = SD(task, "2049/P1D", 0); SegmentDescriptor desc5 = sd(task, "2049/P1D", 0);
SegmentDescriptor desc7 = SD(task, "2013/P1D", 0); SegmentDescriptor desc7 = sd(task, "2013/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc7), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc7), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
@ -858,8 +857,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KinesisDataSourceMetadata( new KinesisDataSourceMetadata(
@ -941,9 +940,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
SegmentDescriptor desc3 = SD(task, "2010/P1D", 0); SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KinesisDataSourceMetadata( new KinesisDataSourceMetadata(
@ -1034,7 +1033,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2009/P1D", 0); SegmentDescriptor desc1 = sd(task, "2009/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KinesisDataSourceMetadata( new KinesisDataSourceMetadata(
@ -1172,8 +1171,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
@ -1249,8 +1248,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
@ -1396,10 +1395,10 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getUnparseable());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
SegmentDescriptor desc3 = SD(task, "2013/P1D", 0); SegmentDescriptor desc3 = sd(task, "2013/P1D", 0);
SegmentDescriptor desc4 = SD(task, "2049/P1D", 0); SegmentDescriptor desc4 = sd(task, "2049/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KinesisDataSourceMetadata( new KinesisDataSourceMetadata(
@ -1621,8 +1620,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata // Check published segments & metadata
SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KinesisDataSourceMetadata( new KinesisDataSourceMetadata(
@ -1732,8 +1731,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata, should all be from the first task // Check published segments & metadata, should all be from the first task
SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KinesisDataSourceMetadata( new KinesisDataSourceMetadata(
@ -1830,8 +1829,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode());
// Check published segments & metadata // Check published segments & metadata
SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
@ -1850,8 +1849,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata // Check published segments & metadata
SegmentDescriptor desc3 = SD(task2, "2011/P1D", 1); SegmentDescriptor desc3 = sd(task2, "2011/P1D", 1);
SegmentDescriptor desc4 = SD(task2, "2013/P1D", 0); SegmentDescriptor desc4 = sd(task2, "2013/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
@ -1929,9 +1928,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata // Check published segments & metadata
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
SegmentDescriptor desc4 = SD(task, "2012/P1D", 0); SegmentDescriptor desc4 = sd(task, "2012/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc4), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc4), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
@ -2044,10 +2043,10 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata // Check published segments & metadata
SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
SegmentDescriptor desc3 = SD(task2, "2011/P1D", 1); SegmentDescriptor desc3 = sd(task2, "2011/P1D", 1);
SegmentDescriptor desc4 = SD(task2, "2012/P1D", 0); SegmentDescriptor desc4 = sd(task2, "2012/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
@ -2197,8 +2196,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata // Check published segments & metadata
SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KinesisDataSourceMetadata( new KinesisDataSourceMetadata(
@ -2318,8 +2317,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>( new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(
@ -2408,8 +2407,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata // Check published metadata
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals( Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
@ -2765,9 +2764,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
this::makeTimeseriesOnlyConglomerate, this::makeTimeseriesOnlyConglomerate,
Execs.directExecutor(), // queryExecutorService Execs.directExecutor(), // queryExecutorService
EasyMock.createMock(MonitorScheduler.class), EasyMock.createMock(MonitorScheduler.class),
new SegmentLoaderFactory( new SegmentLoaderFactory(null, testUtils.getTestObjectMapper()),
new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper())
),
testUtils.getTestObjectMapper(), testUtils.getTestObjectMapper(),
testUtils.getTestIndexIO(), testUtils.getTestIndexIO(),
MapCache.create(1024), MapCache.create(1024),
@ -2880,7 +2877,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
return results.isEmpty() ? 0L : DimensionHandlerUtils.nullToZero(results.get(0).getValue().getLongMetric("rows")); return results.isEmpty() ? 0L : DimensionHandlerUtils.nullToZero(results.get(0).getValue().getLongMetric("rows"));
} }
private static List<byte[]> JB( private static List<byte[]> jb(
String timestamp, String timestamp,
String dim1, String dim1,
String dim2, String dim2,
@ -2906,7 +2903,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
} }
} }
private SegmentDescriptor SD(final Task task, final String intervalString, final int partitionNum) private SegmentDescriptor sd(final Task task, final String intervalString, final int partitionNum)
{ {
final Interval interval = Intervals.of(intervalString); final Interval interval = Intervals.of(intervalString);
return new SegmentDescriptor(interval, getLock(task, interval).getVersion(), partitionNum); return new SegmentDescriptor(interval, getLock(task, interval).getVersion(), partitionNum);

View File

@ -76,20 +76,20 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
private static Shard shard1; private static Shard shard1;
private static KinesisRecordSupplier recordSupplier; private static KinesisRecordSupplier recordSupplier;
private static List<Record> shard1Records = ImmutableList.of( private static List<Record> shard1Records = ImmutableList.of(
new Record().withData(JB("2011", "d", "y", "10", "20.0", "1.0")).withSequenceNumber("0"), new Record().withData(jb("2011", "d", "y", "10", "20.0", "1.0")).withSequenceNumber("0"),
new Record().withData(JB("2011", "e", "y", "10", "20.0", "1.0")).withSequenceNumber("1"), new Record().withData(jb("2011", "e", "y", "10", "20.0", "1.0")).withSequenceNumber("1"),
new Record().withData(JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")).withSequenceNumber("2"), new Record().withData(jb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")).withSequenceNumber("2"),
new Record().withData(ByteBuffer.wrap(StringUtils.toUtf8("unparseable"))).withSequenceNumber("3"), new Record().withData(ByteBuffer.wrap(StringUtils.toUtf8("unparseable"))).withSequenceNumber("3"),
new Record().withData(ByteBuffer.wrap(StringUtils.toUtf8("unparseable2"))).withSequenceNumber("4"), new Record().withData(ByteBuffer.wrap(StringUtils.toUtf8("unparseable2"))).withSequenceNumber("4"),
new Record().withData(ByteBuffer.wrap(StringUtils.toUtf8("{}"))).withSequenceNumber("5"), new Record().withData(ByteBuffer.wrap(StringUtils.toUtf8("{}"))).withSequenceNumber("5"),
new Record().withData(JB("2013", "f", "y", "10", "20.0", "1.0")).withSequenceNumber("6"), new Record().withData(jb("2013", "f", "y", "10", "20.0", "1.0")).withSequenceNumber("6"),
new Record().withData(JB("2049", "f", "y", "notanumber", "20.0", "1.0")).withSequenceNumber("7"), new Record().withData(jb("2049", "f", "y", "notanumber", "20.0", "1.0")).withSequenceNumber("7"),
new Record().withData(JB("2012", "g", "y", "10", "20.0", "1.0")).withSequenceNumber("8"), new Record().withData(jb("2012", "g", "y", "10", "20.0", "1.0")).withSequenceNumber("8"),
new Record().withData(JB("2011", "h", "y", "10", "20.0", "1.0")).withSequenceNumber("9") new Record().withData(jb("2011", "h", "y", "10", "20.0", "1.0")).withSequenceNumber("9")
); );
private static List<Record> shard0Records = ImmutableList.of( private static List<Record> shard0Records = ImmutableList.of(
new Record().withData(JB("2008", "a", "y", "10", "20.0", "1.0")).withSequenceNumber("0"), new Record().withData(jb("2008", "a", "y", "10", "20.0", "1.0")).withSequenceNumber("0"),
new Record().withData(JB("2009", "b", "y", "10", "20.0", "1.0")).withSequenceNumber("1") new Record().withData(jb("2009", "b", "y", "10", "20.0", "1.0")).withSequenceNumber("1")
); );
private static List<Object> allRecords = ImmutableList.builder() private static List<Object> allRecords = ImmutableList.builder()
.addAll(shard0Records.stream() .addAll(shard0Records.stream()
@ -120,7 +120,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
.toList())) .toList()))
.build(); .build();
private static ByteBuffer JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1) private static ByteBuffer jb(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1)
{ {
try { try {
return ByteBuffer.wrap(new ObjectMapper().writeValueAsBytes( return ByteBuffer.wrap(new ObjectMapper().writeValueAsBytes(

View File

@ -3560,7 +3560,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
); );
} }
private static List<byte[]> JB( private static List<byte[]> jb(
String timestamp, String timestamp,
String dim1, String dim1,
String dim2, String dim2,

View File

@ -43,6 +43,7 @@ import org.apache.druid.guice.JsonConfigProvider;
import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.LazySingleton;
import org.apache.druid.initialization.DruidModule; import org.apache.druid.initialization.DruidModule;
import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.URIs;
import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.logger.Logger;
import javax.annotation.Nullable; import javax.annotation.Nullable;
@ -184,7 +185,8 @@ public class S3StorageDruidModule implements DruidModule
final Protocol protocolFromClientConfig = parseProtocol(clientConfig.getProtocol()); final Protocol protocolFromClientConfig = parseProtocol(clientConfig.getProtocol());
final String endpointUrl = endpointConfig.getUrl(); final String endpointUrl = endpointConfig.getUrl();
if (StringUtils.isNotEmpty(endpointUrl)) { if (StringUtils.isNotEmpty(endpointUrl)) {
final URI uri = URI.create(endpointUrl); //noinspection ConstantConditions
final URI uri = URIs.parse(endpointUrl, protocolFromClientConfig.toString());
final Protocol protocol = parseProtocol(uri.getScheme()); final Protocol protocol = parseProtocol(uri.getScheme());
if (protocol != null && (protocol != protocolFromClientConfig)) { if (protocol != null && (protocol != protocolFromClientConfig)) {
log.warn("[%s] protocol will be used for endpoint [%s]", protocol, endpointUrl); log.warn("[%s] protocol will be used for endpoint [%s]", protocol, endpointUrl);

View File

@ -19,7 +19,10 @@
package org.apache.druid.indexing.common; package org.apache.druid.indexing.common;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Inject; import com.google.inject.Inject;
import org.apache.druid.guice.annotations.Json;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.loading.SegmentLoader; import org.apache.druid.segment.loading.SegmentLoader;
import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLoaderConfig;
import org.apache.druid.segment.loading.SegmentLoaderLocalCacheManager; import org.apache.druid.segment.loading.SegmentLoaderLocalCacheManager;
@ -29,23 +32,30 @@ import java.io.File;
import java.util.Collections; import java.util.Collections;
/** /**
*
*/ */
public class SegmentLoaderFactory public class SegmentLoaderFactory
{ {
private final SegmentLoaderLocalCacheManager loader; private final IndexIO indexIO;
private final ObjectMapper jsonMapper;
@Inject @Inject
public SegmentLoaderFactory( public SegmentLoaderFactory(
SegmentLoaderLocalCacheManager loader IndexIO indexIO,
@Json ObjectMapper mapper
) )
{ {
this.loader = loader; this.indexIO = indexIO;
this.jsonMapper = mapper;
} }
public SegmentLoader manufacturate(File storageDir) public SegmentLoader manufacturate(File storageDir)
{ {
return loader.withConfig( return new SegmentLoaderLocalCacheManager(
new SegmentLoaderConfig().withLocations(Collections.singletonList(new StorageLocationConfig().setPath(storageDir))) indexIO,
new SegmentLoaderConfig().withLocations(
Collections.singletonList(new StorageLocationConfig().setPath(storageDir))),
jsonMapper
); );
} }
} }

View File

@ -29,6 +29,7 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.BiMap; import com.google.common.collect.BiMap;
import com.google.common.collect.HashBiMap; import com.google.common.collect.HashBiMap;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import org.apache.druid.client.coordinator.CoordinatorClient;
import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionSchema;
import org.apache.druid.data.input.impl.DimensionSchema.MultiValueHandling; import org.apache.druid.data.input.impl.DimensionSchema.MultiValueHandling;
import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.DimensionsSpec;
@ -40,6 +41,8 @@ import org.apache.druid.data.input.impl.NoopInputRowParser;
import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema;
import org.apache.druid.data.input.impl.TimeAndDimsParseSpec; import org.apache.druid.data.input.impl.TimeAndDimsParseSpec;
import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.RetryPolicyFactory;
import org.apache.druid.indexing.common.SegmentLoaderFactory;
import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.actions.SegmentListUsedAction; import org.apache.druid.indexing.common.actions.SegmentListUsedAction;
import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TaskActionClient;
@ -132,6 +135,15 @@ public class CompactionTask extends AbstractTask
@JsonIgnore @JsonIgnore
private final RowIngestionMetersFactory rowIngestionMetersFactory; private final RowIngestionMetersFactory rowIngestionMetersFactory;
@JsonIgnore
private final CoordinatorClient coordinatorClient;
@JsonIgnore
private final SegmentLoaderFactory segmentLoaderFactory;
@JsonIgnore
private final RetryPolicyFactory retryPolicyFactory;
@JsonIgnore @JsonIgnore
private List<IndexTask> indexTaskSpecs; private List<IndexTask> indexTaskSpecs;
@ -153,7 +165,10 @@ public class CompactionTask extends AbstractTask
@JacksonInject ObjectMapper jsonMapper, @JacksonInject ObjectMapper jsonMapper,
@JacksonInject AuthorizerMapper authorizerMapper, @JacksonInject AuthorizerMapper authorizerMapper,
@JacksonInject ChatHandlerProvider chatHandlerProvider, @JacksonInject ChatHandlerProvider chatHandlerProvider,
@JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory,
@JacksonInject CoordinatorClient coordinatorClient,
@JacksonInject SegmentLoaderFactory segmentLoaderFactory,
@JacksonInject RetryPolicyFactory retryPolicyFactory
) )
{ {
super(getOrMakeId(id, TYPE, dataSource), null, taskResource, dataSource, context); super(getOrMakeId(id, TYPE, dataSource), null, taskResource, dataSource, context);
@ -186,6 +201,9 @@ public class CompactionTask extends AbstractTask
this.authorizerMapper = authorizerMapper; this.authorizerMapper = authorizerMapper;
this.chatHandlerProvider = chatHandlerProvider; this.chatHandlerProvider = chatHandlerProvider;
this.rowIngestionMetersFactory = rowIngestionMetersFactory; this.rowIngestionMetersFactory = rowIngestionMetersFactory;
this.coordinatorClient = coordinatorClient;
this.segmentLoaderFactory = segmentLoaderFactory;
this.retryPolicyFactory = retryPolicyFactory;
} }
@JsonProperty @JsonProperty
@ -278,20 +296,23 @@ public class CompactionTask extends AbstractTask
metricsSpec, metricsSpec,
keepSegmentGranularity, keepSegmentGranularity,
segmentGranularity, segmentGranularity,
jsonMapper jsonMapper,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
).stream() ).stream()
.map(spec -> new IndexTask( .map(spec -> new IndexTask(
getId(), getId(),
getGroupId(), getGroupId(),
getTaskResource(), getTaskResource(),
getDataSource(), getDataSource(),
spec, spec,
getContext(), getContext(),
authorizerMapper, authorizerMapper,
chatHandlerProvider, chatHandlerProvider,
rowIngestionMetersFactory rowIngestionMetersFactory
)) ))
.collect(Collectors.toList()); .collect(Collectors.toList());
} }
if (indexTaskSpecs.isEmpty()) { if (indexTaskSpecs.isEmpty()) {
@ -338,7 +359,10 @@ public class CompactionTask extends AbstractTask
@Nullable final AggregatorFactory[] metricsSpec, @Nullable final AggregatorFactory[] metricsSpec,
@Nullable final Boolean keepSegmentGranularity, @Nullable final Boolean keepSegmentGranularity,
@Nullable final Granularity segmentGranularity, @Nullable final Granularity segmentGranularity,
final ObjectMapper jsonMapper final ObjectMapper jsonMapper,
final CoordinatorClient coordinatorClient,
final SegmentLoaderFactory segmentLoaderFactory,
final RetryPolicyFactory retryPolicyFactory
) throws IOException, SegmentLoadingException ) throws IOException, SegmentLoadingException
{ {
Pair<Map<DataSegment, File>, List<TimelineObjectHolder<String, DataSegment>>> pair = prepareSegments( Pair<Map<DataSegment, File>, List<TimelineObjectHolder<String, DataSegment>>> pair = prepareSegments(
@ -379,7 +403,14 @@ public class CompactionTask extends AbstractTask
return Collections.singletonList( return Collections.singletonList(
new IndexIngestionSpec( new IndexIngestionSpec(
dataSchema, dataSchema,
createIoConfig(toolbox, dataSchema, segmentProvider.interval), createIoConfig(
toolbox,
dataSchema,
segmentProvider.interval,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
),
compactionTuningConfig compactionTuningConfig
) )
); );
@ -411,7 +442,14 @@ public class CompactionTask extends AbstractTask
specs.add( specs.add(
new IndexIngestionSpec( new IndexIngestionSpec(
dataSchema, dataSchema,
createIoConfig(toolbox, dataSchema, interval), createIoConfig(
toolbox,
dataSchema,
interval,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
),
compactionTuningConfig compactionTuningConfig
) )
); );
@ -438,7 +476,14 @@ public class CompactionTask extends AbstractTask
return Collections.singletonList( return Collections.singletonList(
new IndexIngestionSpec( new IndexIngestionSpec(
dataSchema, dataSchema,
createIoConfig(toolbox, dataSchema, segmentProvider.interval), createIoConfig(
toolbox,
dataSchema,
segmentProvider.interval,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
),
compactionTuningConfig compactionTuningConfig
) )
); );
@ -446,7 +491,14 @@ public class CompactionTask extends AbstractTask
} }
} }
private static IndexIOConfig createIoConfig(TaskToolbox toolbox, DataSchema dataSchema, Interval interval) private static IndexIOConfig createIoConfig(
TaskToolbox toolbox,
DataSchema dataSchema,
Interval interval,
CoordinatorClient coordinatorClient,
SegmentLoaderFactory segmentLoaderFactory,
RetryPolicyFactory retryPolicyFactory
)
{ {
return new IndexIOConfig( return new IndexIOConfig(
new IngestSegmentFirehoseFactory( new IngestSegmentFirehoseFactory(
@ -456,7 +508,10 @@ public class CompactionTask extends AbstractTask
// set dimensions and metrics names to make sure that the generated dataSchema is used for the firehose // set dimensions and metrics names to make sure that the generated dataSchema is used for the firehose
dataSchema.getParser().getParseSpec().getDimensionsSpec().getDimensionNames(), dataSchema.getParser().getParseSpec().getDimensionsSpec().getDimensionNames(),
Arrays.stream(dataSchema.getAggregators()).map(AggregatorFactory::getName).collect(Collectors.toList()), Arrays.stream(dataSchema.getAggregators()).map(AggregatorFactory::getName).collect(Collectors.toList()),
toolbox.getIndexIO() toolbox.getIndexIO(),
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
), ),
false false
); );
@ -811,7 +866,7 @@ public class CompactionTask extends AbstractTask
* targetCompactionSizeBytes cannot be used with {@link IndexTuningConfig#maxRowsPerSegment}, * targetCompactionSizeBytes cannot be used with {@link IndexTuningConfig#maxRowsPerSegment},
* {@link IndexTuningConfig#maxTotalRows}, or {@link IndexTuningConfig#numShards} together. * {@link IndexTuningConfig#maxTotalRows}, or {@link IndexTuningConfig#numShards} together.
* {@link #hasPartitionConfig} checks one of those configs is set. * {@link #hasPartitionConfig} checks one of those configs is set.
* * <p>
* This throws an {@link IllegalArgumentException} if targetCompactionSizeBytes is set and hasPartitionConfig * This throws an {@link IllegalArgumentException} if targetCompactionSizeBytes is set and hasPartitionConfig
* returns true. If targetCompactionSizeBytes is not set, this returns null or * returns true. If targetCompactionSizeBytes is not set, this returns null or
* {@link DataSourceCompactionConfig#DEFAULT_TARGET_COMPACTION_SIZE_BYTES} according to the result of * {@link DataSourceCompactionConfig#DEFAULT_TARGET_COMPACTION_SIZE_BYTES} according to the result of
@ -860,6 +915,9 @@ public class CompactionTask extends AbstractTask
private final AuthorizerMapper authorizerMapper; private final AuthorizerMapper authorizerMapper;
private final ChatHandlerProvider chatHandlerProvider; private final ChatHandlerProvider chatHandlerProvider;
private final RowIngestionMetersFactory rowIngestionMetersFactory; private final RowIngestionMetersFactory rowIngestionMetersFactory;
private final CoordinatorClient coordinatorClient;
private final SegmentLoaderFactory segmentLoaderFactory;
private final RetryPolicyFactory retryPolicyFactory;
@Nullable @Nullable
private Interval interval; private Interval interval;
@ -885,7 +943,10 @@ public class CompactionTask extends AbstractTask
ObjectMapper jsonMapper, ObjectMapper jsonMapper,
AuthorizerMapper authorizerMapper, AuthorizerMapper authorizerMapper,
ChatHandlerProvider chatHandlerProvider, ChatHandlerProvider chatHandlerProvider,
RowIngestionMetersFactory rowIngestionMetersFactory RowIngestionMetersFactory rowIngestionMetersFactory,
CoordinatorClient coordinatorClient,
SegmentLoaderFactory segmentLoaderFactory,
RetryPolicyFactory retryPolicyFactory
) )
{ {
this.dataSource = dataSource; this.dataSource = dataSource;
@ -893,6 +954,9 @@ public class CompactionTask extends AbstractTask
this.authorizerMapper = authorizerMapper; this.authorizerMapper = authorizerMapper;
this.chatHandlerProvider = chatHandlerProvider; this.chatHandlerProvider = chatHandlerProvider;
this.rowIngestionMetersFactory = rowIngestionMetersFactory; this.rowIngestionMetersFactory = rowIngestionMetersFactory;
this.coordinatorClient = coordinatorClient;
this.segmentLoaderFactory = segmentLoaderFactory;
this.retryPolicyFactory = retryPolicyFactory;
} }
public Builder interval(Interval interval) public Builder interval(Interval interval)
@ -968,7 +1032,10 @@ public class CompactionTask extends AbstractTask
jsonMapper, jsonMapper,
authorizerMapper, authorizerMapper,
chatHandlerProvider, chatHandlerProvider,
rowIngestionMetersFactory rowIngestionMetersFactory,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
); );
} }
} }

View File

@ -54,7 +54,6 @@ import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMeters;
import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor; import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor;
import org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory;
import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.JodaUtils;
@ -84,7 +83,6 @@ import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata;
import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
import org.apache.druid.segment.realtime.firehose.ChatHandler; import org.apache.druid.segment.realtime.firehose.ChatHandler;
import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
import org.apache.druid.segment.realtime.firehose.CombiningFirehoseFactory;
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Action;
import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.AuthorizerMapper;
@ -419,8 +417,6 @@ public class IndexTask extends AbstractTask implements ChatHandler
final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory(); final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory();
setFirehoseFactoryToolbox(firehoseFactory, toolbox);
final File firehoseTempDir = toolbox.getFirehoseTemporaryDir(); final File firehoseTempDir = toolbox.getFirehoseTemporaryDir();
// Firehose temporary directory is automatically removed when this IndexTask completes. // Firehose temporary directory is automatically removed when this IndexTask completes.
FileUtils.forceMkdir(firehoseTempDir); FileUtils.forceMkdir(firehoseTempDir);
@ -489,25 +485,6 @@ public class IndexTask extends AbstractTask implements ChatHandler
} }
} }
// pass toolbox to any IngestSegmentFirehoseFactory
private void setFirehoseFactoryToolbox(FirehoseFactory firehoseFactory, TaskToolbox toolbox)
{
if (firehoseFactory instanceof IngestSegmentFirehoseFactory) {
((IngestSegmentFirehoseFactory) firehoseFactory).setTaskToolbox(toolbox);
return;
}
if (firehoseFactory instanceof CombiningFirehoseFactory) {
for (FirehoseFactory delegateFactory : ((CombiningFirehoseFactory) firehoseFactory).getDelegateFactoryList()) {
if (delegateFactory instanceof IngestSegmentFirehoseFactory) {
((IngestSegmentFirehoseFactory) delegateFactory).setTaskToolbox(toolbox);
} else if (delegateFactory instanceof CombiningFirehoseFactory) {
setFirehoseFactoryToolbox(delegateFactory, toolbox);
}
}
}
}
private Map<String, TaskReport> getTaskCompletionReports() private Map<String, TaskReport> getTaskCompletionReports()
{ {
return TaskReport.buildTaskReports( return TaskReport.buildTaskReports(

View File

@ -44,7 +44,6 @@ import org.apache.druid.indexing.common.task.IndexTask;
import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.TaskResource;
import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.common.task.Tasks;
import org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory;
import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.StringUtils;
@ -185,11 +184,6 @@ public class ParallelIndexSubTask extends AbstractTask
{ {
final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory(); final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory();
if (firehoseFactory instanceof IngestSegmentFirehoseFactory) {
// pass toolbox to Firehose
((IngestSegmentFirehoseFactory) firehoseFactory).setTaskToolbox(toolbox);
}
final File firehoseTempDir = toolbox.getFirehoseTemporaryDir(); final File firehoseTempDir = toolbox.getFirehoseTemporaryDir();
// Firehose temporary directory is automatically removed when this IndexTask completes. // Firehose temporary directory is automatically removed when this IndexTask completes.
FileUtils.forceMkdir(firehoseTempDir); FileUtils.forceMkdir(firehoseTempDir);

View File

@ -30,16 +30,20 @@ import com.google.common.collect.BiMap;
import com.google.common.collect.HashBiMap; import com.google.common.collect.HashBiMap;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import org.apache.druid.client.coordinator.CoordinatorClient;
import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.Firehose;
import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.FirehoseFactory;
import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.InputRowParser;
import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.RetryPolicy;
import org.apache.druid.indexing.common.actions.SegmentListUsedAction; import org.apache.druid.indexing.common.RetryPolicyFactory;
import org.apache.druid.indexing.common.SegmentLoaderFactory;
import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.java.util.common.parsers.ParseException;
import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.DimFilter;
import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.QueryableIndexStorageAdapter;
import org.apache.druid.segment.loading.SegmentLoader;
import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.segment.loading.SegmentLoadingException;
import org.apache.druid.segment.realtime.firehose.IngestSegmentFirehose; import org.apache.druid.segment.realtime.firehose.IngestSegmentFirehose;
import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter; import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter;
@ -48,14 +52,17 @@ import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.TimelineObjectHolder;
import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.VersionedIntervalTimeline;
import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.PartitionChunk;
import org.joda.time.Duration;
import org.joda.time.Interval; import org.joda.time.Interval;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.ThreadLocalRandom;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.stream.IntStream; import java.util.stream.IntStream;
@ -68,7 +75,9 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
private final List<String> dimensions; private final List<String> dimensions;
private final List<String> metrics; private final List<String> metrics;
private final IndexIO indexIO; private final IndexIO indexIO;
private TaskToolbox taskToolbox; private final CoordinatorClient coordinatorClient;
private final SegmentLoaderFactory segmentLoaderFactory;
private final RetryPolicyFactory retryPolicyFactory;
@JsonCreator @JsonCreator
public IngestSegmentFirehoseFactory( public IngestSegmentFirehoseFactory(
@ -77,7 +86,10 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
@JsonProperty("filter") DimFilter dimFilter, @JsonProperty("filter") DimFilter dimFilter,
@JsonProperty("dimensions") List<String> dimensions, @JsonProperty("dimensions") List<String> dimensions,
@JsonProperty("metrics") List<String> metrics, @JsonProperty("metrics") List<String> metrics,
@JacksonInject IndexIO indexIO @JacksonInject IndexIO indexIO,
@JacksonInject CoordinatorClient coordinatorClient,
@JacksonInject SegmentLoaderFactory segmentLoaderFactory,
@JacksonInject RetryPolicyFactory retryPolicyFactory
) )
{ {
Preconditions.checkNotNull(dataSource, "dataSource"); Preconditions.checkNotNull(dataSource, "dataSource");
@ -88,6 +100,9 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
this.dimensions = dimensions; this.dimensions = dimensions;
this.metrics = metrics; this.metrics = metrics;
this.indexIO = Preconditions.checkNotNull(indexIO, "null IndexIO"); this.indexIO = Preconditions.checkNotNull(indexIO, "null IndexIO");
this.coordinatorClient = Preconditions.checkNotNull(coordinatorClient, "null CoordinatorClient");
this.segmentLoaderFactory = Preconditions.checkNotNull(segmentLoaderFactory, "null SegmentLoaderFactory");
this.retryPolicyFactory = Preconditions.checkNotNull(retryPolicyFactory, "null RetryPolicyFactory");
} }
@JsonProperty @JsonProperty
@ -120,23 +135,46 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
return metrics; return metrics;
} }
public void setTaskToolbox(TaskToolbox taskToolbox)
{
this.taskToolbox = taskToolbox;
}
@Override @Override
public Firehose connect(InputRowParser inputRowParser, File temporaryDirectory) throws ParseException public Firehose connect(InputRowParser inputRowParser, File temporaryDirectory) throws ParseException
{ {
log.info("Connecting firehose: dataSource[%s], interval[%s]", dataSource, interval); log.info("Connecting firehose: dataSource[%s], interval[%s]", dataSource, interval);
Preconditions.checkNotNull(taskToolbox, "taskToolbox is not set");
try { try {
final List<DataSegment> usedSegments = taskToolbox // This call used to use the TaskActionClient, so for compatibility we use the same retry configuration
.getTaskActionClient() // as TaskActionClient.
.submit(new SegmentListUsedAction(dataSource, interval, null)); final RetryPolicy retryPolicy = retryPolicyFactory.makeRetryPolicy();
final Map<DataSegment, File> segmentFileMap = taskToolbox.fetchSegments(usedSegments); List<DataSegment> usedSegments;
while (true) {
try {
usedSegments =
coordinatorClient.getDatabaseSegmentDataSourceSegments(dataSource, Collections.singletonList(interval));
break;
}
catch (Throwable e) {
log.warn(e, "Exception getting database segments");
final Duration delay = retryPolicy.getAndIncrementRetryDelay();
if (delay == null) {
throw e;
} else {
final long sleepTime = jitter(delay.getMillis());
log.info("Will try again in [%s].", new Duration(sleepTime).toString());
try {
Thread.sleep(sleepTime);
}
catch (InterruptedException e2) {
throw new RuntimeException(e2);
}
}
}
}
final SegmentLoader segmentLoader = segmentLoaderFactory.manufacturate(temporaryDirectory);
Map<DataSegment, File> segmentFileMap = Maps.newLinkedHashMap();
for (DataSegment segment : usedSegments) {
segmentFileMap.put(segment, segmentLoader.getSegmentFiles(segment));
}
final List<TimelineObjectHolder<String, DataSegment>> timeLineSegments = VersionedIntervalTimeline final List<TimelineObjectHolder<String, DataSegment>> timeLineSegments = VersionedIntervalTimeline
.forSegments(usedSegments) .forSegments(usedSegments)
.lookup(interval); .lookup(interval);
@ -201,11 +239,18 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
final TransformSpec transformSpec = TransformSpec.fromInputRowParser(inputRowParser); final TransformSpec transformSpec = TransformSpec.fromInputRowParser(inputRowParser);
return new IngestSegmentFirehose(adapters, transformSpec, dims, metricsList, dimFilter); return new IngestSegmentFirehose(adapters, transformSpec, dims, metricsList, dimFilter);
} }
catch (IOException | SegmentLoadingException e) { catch (SegmentLoadingException e) {
throw Throwables.propagate(e); throw Throwables.propagate(e);
} }
} }
private long jitter(long input)
{
final double jitter = ThreadLocalRandom.current().nextGaussian() * input / 4.0;
long retval = input + (long) jitter;
return retval < 0 ? 0 : retval;
}
@VisibleForTesting @VisibleForTesting
static List<String> getUniqueDimensions( static List<String> getUniqueDimensions(
List<TimelineObjectHolder<String, DataSegment>> timelineSegments, List<TimelineObjectHolder<String, DataSegment>> timelineSegments,
@ -260,7 +305,7 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
final BiMap<Integer, String> orderedMetrics = uniqueMetrics.inverse(); final BiMap<Integer, String> orderedMetrics = uniqueMetrics.inverse();
return IntStream.range(0, orderedMetrics.size()) return IntStream.range(0, orderedMetrics.size())
.mapToObj(orderedMetrics::get) .mapToObj(orderedMetrics::get)
.collect(Collectors.toList()); .collect(Collectors.toList());
} }
} }

View File

@ -76,6 +76,7 @@ public class TaskToolboxTest
private MonitorScheduler mockMonitorScheduler = EasyMock.createMock(MonitorScheduler.class); private MonitorScheduler mockMonitorScheduler = EasyMock.createMock(MonitorScheduler.class);
private ExecutorService mockQueryExecutorService = EasyMock.createMock(ExecutorService.class); private ExecutorService mockQueryExecutorService = EasyMock.createMock(ExecutorService.class);
private ObjectMapper ObjectMapper = new ObjectMapper(); private ObjectMapper ObjectMapper = new ObjectMapper();
private SegmentLoaderFactory mockSegmentLoaderFactory = EasyMock.createMock(SegmentLoaderFactory.class);
private SegmentLoaderLocalCacheManager mockSegmentLoaderLocalCacheManager = EasyMock.createMock(SegmentLoaderLocalCacheManager.class); private SegmentLoaderLocalCacheManager mockSegmentLoaderLocalCacheManager = EasyMock.createMock(SegmentLoaderLocalCacheManager.class);
private Task task = EasyMock.createMock(Task.class); private Task task = EasyMock.createMock(Task.class);
private IndexMergerV9 mockIndexMergerV9 = EasyMock.createMock(IndexMergerV9.class); private IndexMergerV9 mockIndexMergerV9 = EasyMock.createMock(IndexMergerV9.class);
@ -107,7 +108,7 @@ public class TaskToolboxTest
() -> mockQueryRunnerFactoryConglomerate, () -> mockQueryRunnerFactoryConglomerate,
mockQueryExecutorService, mockQueryExecutorService,
mockMonitorScheduler, mockMonitorScheduler,
new SegmentLoaderFactory(mockSegmentLoaderLocalCacheManager), mockSegmentLoaderFactory,
ObjectMapper, ObjectMapper,
mockIndexIO, mockIndexIO,
mockCache, mockCache,
@ -162,13 +163,13 @@ public class TaskToolboxTest
public void testFetchSegments() throws SegmentLoadingException, IOException public void testFetchSegments() throws SegmentLoadingException, IOException
{ {
File expectedFile = temporaryFolder.newFile(); File expectedFile = temporaryFolder.newFile();
EasyMock
.expect(mockSegmentLoaderFactory.manufacturate(EasyMock.anyObject()))
.andReturn(mockSegmentLoaderLocalCacheManager).anyTimes();
EasyMock EasyMock
.expect(mockSegmentLoaderLocalCacheManager.getSegmentFiles(EasyMock.anyObject())) .expect(mockSegmentLoaderLocalCacheManager.getSegmentFiles(EasyMock.anyObject()))
.andReturn(expectedFile).anyTimes(); .andReturn(expectedFile).anyTimes();
EasyMock EasyMock.replay(mockSegmentLoaderFactory, mockSegmentLoaderLocalCacheManager);
.expect(mockSegmentLoaderLocalCacheManager.withConfig(EasyMock.anyObject()))
.andReturn(mockSegmentLoaderLocalCacheManager).anyTimes();
EasyMock.replay(mockSegmentLoaderLocalCacheManager);
DataSegment dataSegment = DataSegment.builder().dataSource("source").interval(Intervals.of("2012-01-01/P1D")).version("1").size(1).build(); DataSegment dataSegment = DataSegment.builder().dataSource("source").interval(Intervals.of("2012-01-01/P1D")).version("1").size(1).build();
List<DataSegment> segments = ImmutableList.of List<DataSegment> segments = ImmutableList.of
( (

View File

@ -119,7 +119,6 @@ import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.RealtimeIOConfig; import org.apache.druid.segment.indexing.RealtimeIOConfig;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.druid.segment.loading.SegmentLoaderConfig; 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.loading.StorageLocationConfig;
import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier;
import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory;
@ -1607,9 +1606,7 @@ public class AppenderatorDriverRealtimeIndexTaskTest
() -> conglomerate, () -> conglomerate,
Execs.directExecutor(), // queryExecutorService Execs.directExecutor(), // queryExecutorService
EasyMock.createMock(MonitorScheduler.class), EasyMock.createMock(MonitorScheduler.class),
new SegmentLoaderFactory( new SegmentLoaderFactory(null, testUtils.getTestObjectMapper()),
new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper())
),
testUtils.getTestObjectMapper(), testUtils.getTestObjectMapper(),
testUtils.getTestIndexIO(), testUtils.getTestIndexIO(),
MapCache.create(1024), MapCache.create(1024),

View File

@ -23,11 +23,15 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.io.Files; import com.google.common.io.Files;
import org.apache.druid.client.coordinator.CoordinatorClient;
import org.apache.druid.data.input.impl.CSVParseSpec; import org.apache.druid.data.input.impl.CSVParseSpec;
import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.ParseSpec; import org.apache.druid.data.input.impl.ParseSpec;
import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.RetryPolicyConfig;
import org.apache.druid.indexing.common.RetryPolicyFactory;
import org.apache.druid.indexing.common.SegmentLoaderFactory;
import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.indexing.common.actions.LocalTaskActionClient; import org.apache.druid.indexing.common.actions.LocalTaskActionClient;
@ -52,6 +56,7 @@ import org.apache.druid.segment.loading.StorageLocationConfig;
import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.server.security.AuthTestUtils;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec;
import org.joda.time.Interval;
import org.junit.After; import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
@ -96,12 +101,24 @@ public class CompactionTaskRunTest extends IngestionTestBase
); );
private RowIngestionMetersFactory rowIngestionMetersFactory; private RowIngestionMetersFactory rowIngestionMetersFactory;
private CoordinatorClient coordinatorClient;
private SegmentLoaderFactory segmentLoaderFactory;
private ExecutorService exec; private ExecutorService exec;
private static RetryPolicyFactory retryPolicyFactory = new RetryPolicyFactory(new RetryPolicyConfig());
public CompactionTaskRunTest() public CompactionTaskRunTest()
{ {
TestUtils testUtils = new TestUtils(); TestUtils testUtils = new TestUtils();
rowIngestionMetersFactory = testUtils.getRowIngestionMetersFactory(); rowIngestionMetersFactory = testUtils.getRowIngestionMetersFactory();
coordinatorClient = new CoordinatorClient(null, null)
{
@Override
public List<DataSegment> getDatabaseSegmentDataSourceSegments(String dataSource, List<Interval> intervals)
{
return getStorageCoordinator().getUsedSegmentsForIntervals(dataSource, intervals);
}
};
segmentLoaderFactory = new SegmentLoaderFactory(getIndexIO(), getObjectMapper());
} }
@Before @Before
@ -126,7 +143,10 @@ public class CompactionTaskRunTest extends IngestionTestBase
getObjectMapper(), getObjectMapper(),
AuthTestUtils.TEST_AUTHORIZER_MAPPER, AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null, null,
rowIngestionMetersFactory rowIngestionMetersFactory,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
); );
final CompactionTask compactionTask = builder final CompactionTask compactionTask = builder
@ -156,7 +176,10 @@ public class CompactionTaskRunTest extends IngestionTestBase
getObjectMapper(), getObjectMapper(),
AuthTestUtils.TEST_AUTHORIZER_MAPPER, AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null, null,
rowIngestionMetersFactory rowIngestionMetersFactory,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
); );
final CompactionTask compactionTask1 = builder final CompactionTask compactionTask1 = builder
@ -200,7 +223,10 @@ public class CompactionTaskRunTest extends IngestionTestBase
getObjectMapper(), getObjectMapper(),
AuthTestUtils.TEST_AUTHORIZER_MAPPER, AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null, null,
rowIngestionMetersFactory rowIngestionMetersFactory,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
); );
final CompactionTask compactionTask1 = builder final CompactionTask compactionTask1 = builder
@ -248,7 +274,10 @@ public class CompactionTaskRunTest extends IngestionTestBase
getObjectMapper(), getObjectMapper(),
AuthTestUtils.TEST_AUTHORIZER_MAPPER, AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null, null,
rowIngestionMetersFactory rowIngestionMetersFactory,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
); );
// day segmentGranularity // day segmentGranularity

View File

@ -28,6 +28,7 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import org.apache.druid.client.coordinator.CoordinatorClient;
import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.FirehoseFactory;
import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionSchema;
import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.DimensionsSpec;
@ -41,6 +42,9 @@ import org.apache.druid.data.input.impl.TimeAndDimsParseSpec;
import org.apache.druid.guice.GuiceAnnotationIntrospector; import org.apache.druid.guice.GuiceAnnotationIntrospector;
import org.apache.druid.guice.GuiceInjectableValues; import org.apache.druid.guice.GuiceInjectableValues;
import org.apache.druid.guice.GuiceInjectors; import org.apache.druid.guice.GuiceInjectors;
import org.apache.druid.indexing.common.RetryPolicyConfig;
import org.apache.druid.indexing.common.RetryPolicyFactory;
import org.apache.druid.indexing.common.SegmentLoaderFactory;
import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.indexing.common.actions.SegmentListUsedAction; import org.apache.druid.indexing.common.actions.SegmentListUsedAction;
@ -153,12 +157,15 @@ public class CompactionTaskTest
private static List<AggregatorFactory> AGGREGATORS; private static List<AggregatorFactory> AGGREGATORS;
private static List<DataSegment> SEGMENTS; private static List<DataSegment> SEGMENTS;
private static RowIngestionMetersFactory rowIngestionMetersFactory = new TestUtils().getRowIngestionMetersFactory(); private static RowIngestionMetersFactory rowIngestionMetersFactory = new TestUtils().getRowIngestionMetersFactory();
private static Map<DataSegment, File> segmentMap = new HashMap<>();
private static CoordinatorClient coordinatorClient = new TestCoordinatorClient(segmentMap);
private static ObjectMapper objectMapper = setupInjectablesInObjectMapper(new DefaultObjectMapper()); private static ObjectMapper objectMapper = setupInjectablesInObjectMapper(new DefaultObjectMapper());
private static Map<DataSegment, File> segmentMap; private static RetryPolicyFactory retryPolicyFactory = new RetryPolicyFactory(new RetryPolicyConfig());
private final boolean keepSegmentGranularity; private final boolean keepSegmentGranularity;
private TaskToolbox toolbox; private TaskToolbox toolbox;
private SegmentLoaderFactory segmentLoaderFactory;
@BeforeClass @BeforeClass
public static void setupClass() public static void setupClass()
@ -202,7 +209,6 @@ public class CompactionTaskTest
AGGREGATORS.add(new FloatFirstAggregatorFactory("agg_3", "float_dim_3")); AGGREGATORS.add(new FloatFirstAggregatorFactory("agg_3", "float_dim_3"));
AGGREGATORS.add(new DoubleLastAggregatorFactory("agg_4", "double_dim_4")); AGGREGATORS.add(new DoubleLastAggregatorFactory("agg_4", "double_dim_4"));
segmentMap = new HashMap<>(SEGMENT_INTERVALS.size());
for (int i = 0; i < SEGMENT_INTERVALS.size(); i++) { for (int i = 0; i < SEGMENT_INTERVALS.size(); i++) {
final Interval segmentInterval = Intervals.of(StringUtils.format("2017-0%d-01/2017-0%d-01", (i + 1), (i + 2))); final Interval segmentInterval = Intervals.of(StringUtils.format("2017-0%d-01/2017-0%d-01", (i + 1), (i + 2)));
segmentMap.put( segmentMap.put(
@ -243,6 +249,8 @@ public class CompactionTaskTest
binder.bind(AuthorizerMapper.class).toInstance(AuthTestUtils.TEST_AUTHORIZER_MAPPER); binder.bind(AuthorizerMapper.class).toInstance(AuthTestUtils.TEST_AUTHORIZER_MAPPER);
binder.bind(ChatHandlerProvider.class).toInstance(new NoopChatHandlerProvider()); binder.bind(ChatHandlerProvider.class).toInstance(new NoopChatHandlerProvider());
binder.bind(RowIngestionMetersFactory.class).toInstance(rowIngestionMetersFactory); binder.bind(RowIngestionMetersFactory.class).toInstance(rowIngestionMetersFactory);
binder.bind(CoordinatorClient.class).toInstance(coordinatorClient);
binder.bind(SegmentLoaderFactory.class).toInstance(new SegmentLoaderFactory(null, objectMapper));
} }
) )
) )
@ -307,19 +315,21 @@ public class CompactionTaskTest
@Before @Before
public void setup() public void setup()
{ {
final IndexIO testIndexIO = new TestIndexIO(objectMapper, segmentMap);
toolbox = new TestTaskToolbox( toolbox = new TestTaskToolbox(
new TestTaskActionClient(new ArrayList<>(segmentMap.keySet())), new TestTaskActionClient(new ArrayList<>(segmentMap.keySet())),
new TestIndexIO(objectMapper, segmentMap), testIndexIO,
segmentMap segmentMap
); );
segmentLoaderFactory = new SegmentLoaderFactory(testIndexIO, objectMapper);
} }
@Parameters(name = "keepSegmentGranularity={0}") @Parameters(name = "keepSegmentGranularity={0}")
public static Collection<Object[]> parameters() public static Collection<Object[]> parameters()
{ {
return ImmutableList.of( return ImmutableList.of(
new Object[] {false}, new Object[]{false},
new Object[] {true} new Object[]{true}
); );
} }
@ -336,7 +346,10 @@ public class CompactionTaskTest
objectMapper, objectMapper,
AuthTestUtils.TEST_AUTHORIZER_MAPPER, AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null, null,
rowIngestionMetersFactory rowIngestionMetersFactory,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
); );
final CompactionTask task = builder final CompactionTask task = builder
.interval(COMPACTION_INTERVAL) .interval(COMPACTION_INTERVAL)
@ -357,7 +370,10 @@ public class CompactionTaskTest
objectMapper, objectMapper,
AuthTestUtils.TEST_AUTHORIZER_MAPPER, AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null, null,
rowIngestionMetersFactory rowIngestionMetersFactory,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
); );
final CompactionTask task = builder final CompactionTask task = builder
.segments(SEGMENTS) .segments(SEGMENTS)
@ -378,7 +394,10 @@ public class CompactionTaskTest
objectMapper, objectMapper,
AuthTestUtils.TEST_AUTHORIZER_MAPPER, AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null, null,
rowIngestionMetersFactory rowIngestionMetersFactory,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
); );
final CompactionTask task = builder final CompactionTask task = builder
@ -426,7 +445,10 @@ public class CompactionTaskTest
null, null,
keepSegmentGranularity, keepSegmentGranularity,
null, null,
objectMapper objectMapper,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
); );
final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration( final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(
keepSegmentGranularity keepSegmentGranularity
@ -440,7 +462,13 @@ public class CompactionTaskTest
) )
); );
Assert.assertEquals(6, ingestionSpecs.size()); Assert.assertEquals(6, ingestionSpecs.size());
assertIngestionSchema(ingestionSpecs, expectedDimensionsSpec, AGGREGATORS, SEGMENT_INTERVALS, Granularities.MONTH); assertIngestionSchema(
ingestionSpecs,
expectedDimensionsSpec,
AGGREGATORS,
SEGMENT_INTERVALS,
Granularities.MONTH
);
} else { } else {
Assert.assertEquals(1, ingestionSpecs.size()); Assert.assertEquals(1, ingestionSpecs.size());
assertIngestionSchema( assertIngestionSchema(
@ -491,7 +519,10 @@ public class CompactionTaskTest
null, null,
keepSegmentGranularity, keepSegmentGranularity,
null, null,
objectMapper objectMapper,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
); );
final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration( final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(
keepSegmentGranularity keepSegmentGranularity
@ -564,7 +595,10 @@ public class CompactionTaskTest
null, null,
keepSegmentGranularity, keepSegmentGranularity,
null, null,
objectMapper objectMapper,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
); );
final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration( final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(
keepSegmentGranularity keepSegmentGranularity
@ -637,7 +671,10 @@ public class CompactionTaskTest
null, null,
keepSegmentGranularity, keepSegmentGranularity,
null, null,
objectMapper objectMapper,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
); );
final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration( final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(
keepSegmentGranularity keepSegmentGranularity
@ -710,7 +747,10 @@ public class CompactionTaskTest
null, null,
keepSegmentGranularity, keepSegmentGranularity,
null, null,
objectMapper objectMapper,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
); );
if (keepSegmentGranularity) { if (keepSegmentGranularity) {
@ -760,7 +800,10 @@ public class CompactionTaskTest
customMetricsSpec, customMetricsSpec,
keepSegmentGranularity, keepSegmentGranularity,
null, null,
objectMapper objectMapper,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
); );
final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration( final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(
@ -805,7 +848,10 @@ public class CompactionTaskTest
null, null,
keepSegmentGranularity, keepSegmentGranularity,
null, null,
objectMapper objectMapper,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
); );
final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration( final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(
keepSegmentGranularity keepSegmentGranularity
@ -819,7 +865,13 @@ public class CompactionTaskTest
) )
); );
Assert.assertEquals(6, ingestionSpecs.size()); Assert.assertEquals(6, ingestionSpecs.size());
assertIngestionSchema(ingestionSpecs, expectedDimensionsSpec, AGGREGATORS, SEGMENT_INTERVALS, Granularities.MONTH); assertIngestionSchema(
ingestionSpecs,
expectedDimensionsSpec,
AGGREGATORS,
SEGMENT_INTERVALS,
Granularities.MONTH
);
} else { } else {
Assert.assertEquals(1, ingestionSpecs.size()); Assert.assertEquals(1, ingestionSpecs.size());
assertIngestionSchema( assertIngestionSchema(
@ -850,7 +902,10 @@ public class CompactionTaskTest
null, null,
keepSegmentGranularity, keepSegmentGranularity,
null, null,
objectMapper objectMapper,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
); );
} }
@ -871,7 +926,10 @@ public class CompactionTaskTest
null, null,
keepSegmentGranularity, keepSegmentGranularity,
null, null,
objectMapper objectMapper,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
); );
} }
@ -886,7 +944,10 @@ public class CompactionTaskTest
objectMapper, objectMapper,
AuthTestUtils.TEST_AUTHORIZER_MAPPER, AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null, null,
rowIngestionMetersFactory rowIngestionMetersFactory,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
); );
final CompactionTask task = builder final CompactionTask task = builder
@ -934,7 +995,10 @@ public class CompactionTaskTest
null, null,
keepSegmentGranularity, keepSegmentGranularity,
null, null,
objectMapper objectMapper,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
); );
} }
@ -949,7 +1013,10 @@ public class CompactionTaskTest
null, null,
null, null,
new PeriodGranularity(Period.months(3), null, null), new PeriodGranularity(Period.months(3), null, null),
objectMapper objectMapper,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
); );
final List<DimensionsSpec> expectedDimensionsSpec = ImmutableList.of( final List<DimensionsSpec> expectedDimensionsSpec = ImmutableList.of(
new DimensionsSpec(getDimensionSchema(new DoubleDimensionSchema("string_to_double"))) new DimensionsSpec(getDimensionSchema(new DoubleDimensionSchema("string_to_double")))
@ -982,7 +1049,10 @@ public class CompactionTaskTest
null, null,
false, false,
new PeriodGranularity(Period.months(3), null, null), new PeriodGranularity(Period.months(3), null, null),
objectMapper objectMapper,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
); );
final List<DimensionsSpec> expectedDimensionsSpec = ImmutableList.of( final List<DimensionsSpec> expectedDimensionsSpec = ImmutableList.of(
new DimensionsSpec(getDimensionSchema(new DoubleDimensionSchema("string_to_double"))) new DimensionsSpec(getDimensionSchema(new DoubleDimensionSchema("string_to_double")))
@ -1015,7 +1085,10 @@ public class CompactionTaskTest
null, null,
null, null,
null, null,
objectMapper objectMapper,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
); );
final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration( final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(
true true
@ -1048,7 +1121,10 @@ public class CompactionTaskTest
objectMapper, objectMapper,
AuthTestUtils.TEST_AUTHORIZER_MAPPER, AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null, null,
rowIngestionMetersFactory rowIngestionMetersFactory,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
); );
final CompactionTask task = builder final CompactionTask task = builder
.interval(COMPACTION_INTERVAL) .interval(COMPACTION_INTERVAL)
@ -1222,6 +1298,23 @@ public class CompactionTaskTest
} }
} }
private static class TestCoordinatorClient extends CoordinatorClient
{
private final Map<DataSegment, File> segmentMap;
TestCoordinatorClient(Map<DataSegment, File> segmentMap)
{
super(null, null);
this.segmentMap = segmentMap;
}
@Override
public List<DataSegment> getDatabaseSegmentDataSourceSegments(String dataSource, List<Interval> intervals)
{
return new ArrayList<>(segmentMap.keySet());
}
}
private static class TestTaskToolbox extends TaskToolbox private static class TestTaskToolbox extends TaskToolbox
{ {
private final Map<DataSegment, File> segmentFileMap; private final Map<DataSegment, File> segmentFileMap;

View File

@ -107,7 +107,6 @@ import org.apache.druid.segment.indexing.RealtimeIOConfig;
import org.apache.druid.segment.indexing.RealtimeTuningConfig; import org.apache.druid.segment.indexing.RealtimeTuningConfig;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.druid.segment.loading.SegmentLoaderConfig; 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.loading.StorageLocationConfig;
import org.apache.druid.segment.realtime.FireDepartment; import org.apache.druid.segment.realtime.FireDepartment;
import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier;
@ -1076,9 +1075,7 @@ public class RealtimeIndexTaskTest
() -> conglomerate, () -> conglomerate,
Execs.directExecutor(), // queryExecutorService Execs.directExecutor(), // queryExecutorService
EasyMock.createMock(MonitorScheduler.class), EasyMock.createMock(MonitorScheduler.class),
new SegmentLoaderFactory( new SegmentLoaderFactory(null, testUtils.getTestObjectMapper()),
new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper())
),
testUtils.getTestObjectMapper(), testUtils.getTestObjectMapper(),
testUtils.getTestIndexIO(), testUtils.getTestIndexIO(),
MapCache.create(1024), MapCache.create(1024),

View File

@ -29,6 +29,9 @@ import com.google.common.collect.ImmutableSet;
import com.google.common.io.Files; import com.google.common.io.Files;
import com.google.inject.Binder; import com.google.inject.Binder;
import com.google.inject.Module; import com.google.inject.Module;
import org.apache.druid.client.coordinator.CoordinatorClient;
import org.apache.druid.data.input.Firehose;
import org.apache.druid.data.input.FirehoseFactory;
import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.InputRowParser;
@ -39,21 +42,16 @@ import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.guice.GuiceAnnotationIntrospector; import org.apache.druid.guice.GuiceAnnotationIntrospector;
import org.apache.druid.guice.GuiceInjectableValues; import org.apache.druid.guice.GuiceInjectableValues;
import org.apache.druid.guice.GuiceInjectors; import org.apache.druid.guice.GuiceInjectors;
import org.apache.druid.indexing.common.RetryPolicyConfig;
import org.apache.druid.indexing.common.RetryPolicyFactory;
import org.apache.druid.indexing.common.SegmentLoaderFactory; import org.apache.druid.indexing.common.SegmentLoaderFactory;
import org.apache.druid.indexing.common.TaskToolboxFactory;
import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory;
import org.apache.druid.indexing.common.actions.TaskActionToolbox;
import org.apache.druid.indexing.common.actions.TaskAuditLogConfig;
import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.indexing.common.config.TaskStorageConfig; import org.apache.druid.indexing.common.config.TaskStorageConfig;
import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.NoopTask;
import org.apache.druid.indexing.common.task.NoopTestTaskFileWriter;
import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage;
import org.apache.druid.indexing.overlord.TaskLockbox; import org.apache.druid.indexing.overlord.TaskLockbox;
import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.TaskStorage;
import org.apache.druid.indexing.overlord.supervisor.SupervisorManager;
import org.apache.druid.java.util.common.IOE; import org.apache.druid.java.util.common.IOE;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.JodaUtils;
@ -61,7 +59,6 @@ import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.filter.SelectorDimFilter;
@ -72,16 +69,9 @@ import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.loading.DataSegmentArchiver;
import org.apache.druid.segment.loading.DataSegmentKiller;
import org.apache.druid.segment.loading.DataSegmentMover;
import org.apache.druid.segment.loading.DataSegmentPusher;
import org.apache.druid.segment.loading.LocalDataSegmentPuller; import org.apache.druid.segment.loading.LocalDataSegmentPuller;
import org.apache.druid.segment.loading.LocalLoadSpec; import org.apache.druid.segment.loading.LocalLoadSpec;
import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.realtime.firehose.CombiningFirehoseFactory;
import org.apache.druid.segment.loading.SegmentLoaderLocalCacheManager;
import org.apache.druid.segment.loading.StorageLocationConfig;
import org.apache.druid.segment.realtime.firehose.IngestSegmentFirehose;
import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory;
import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.ExpressionTransform;
import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.segment.transform.TransformSpec;
@ -103,15 +93,12 @@ import org.junit.runners.Parameterized;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.net.URI;
import java.nio.file.Paths; import java.nio.file.Paths;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.stream.IntStream; import java.util.stream.IntStream;
@ -170,157 +157,21 @@ public class IngestSegmentFirehoseFactoryTest
} }
INDEX_MERGER_V9.persist(index, persistDir, indexSpec, null); INDEX_MERGER_V9.persist(index, persistDir, indexSpec, null);
final IndexerSQLMetadataStorageCoordinator mdc = new IndexerSQLMetadataStorageCoordinator(null, null, null) final CoordinatorClient cc = new CoordinatorClient(null, null)
{ {
private final Set<DataSegment> published = new HashSet<>();
@Override @Override
public List<DataSegment> getUsedSegmentsForInterval(String dataSource, Interval interval) public List<DataSegment> getDatabaseSegmentDataSourceSegments(String dataSource, List<Interval> intervals)
{ {
return ImmutableList.copyOf(segmentSet); return ImmutableList.copyOf(segmentSet);
} }
@Override
public List<DataSegment> getUsedSegmentsForIntervals(String dataSource, List<Interval> interval)
{
return ImmutableList.copyOf(segmentSet);
}
@Override
public List<DataSegment> getUnusedSegmentsForInterval(String dataSource, Interval interval)
{
return ImmutableList.of();
}
@Override
public Set<DataSegment> announceHistoricalSegments(Set<DataSegment> segments)
{
Set<DataSegment> added = new HashSet<>();
for (final DataSegment segment : segments) {
if (published.add(segment)) {
added.add(segment);
}
}
return ImmutableSet.copyOf(added);
}
@Override
public void deleteSegments(Set<DataSegment> segments)
{
// do nothing
}
}; };
final LocalTaskActionClientFactory tac = new LocalTaskActionClientFactory(
TASK_STORAGE,
new TaskActionToolbox(
TASK_LOCKBOX,
TASK_STORAGE,
mdc,
newMockEmitter(),
EasyMock.createMock(SupervisorManager.class)
),
new TaskAuditLogConfig(false)
);
SegmentHandoffNotifierFactory notifierFactory = EasyMock.createNiceMock(SegmentHandoffNotifierFactory.class); SegmentHandoffNotifierFactory notifierFactory = EasyMock.createNiceMock(SegmentHandoffNotifierFactory.class);
EasyMock.replay(notifierFactory); EasyMock.replay(notifierFactory);
SegmentLoaderConfig segmentLoaderConfig = new SegmentLoaderConfig() final SegmentLoaderFactory slf = new SegmentLoaderFactory(null, MAPPER);
{ final RetryPolicyFactory retryPolicyFactory = new RetryPolicyFactory(new RetryPolicyConfig());
@Override
public List<StorageLocationConfig> getLocations()
{
return new ArrayList<>();
}
};
final TaskToolboxFactory taskToolboxFactory = new TaskToolboxFactory(
new TaskConfig(tmpDir.getAbsolutePath(), null, null, 50000, null, false, null, null),
tac,
newMockEmitter(),
new DataSegmentPusher()
{
@Deprecated
@Override
public String getPathForHadoop(String dataSource)
{
return getPathForHadoop();
}
@Override
public String getPathForHadoop()
{
throw new UnsupportedOperationException();
}
@Override
public DataSegment push(File file, DataSegment segment, boolean useUniquePath)
{
return segment;
}
@Override
public Map<String, Object> makeLoadSpec(URI uri)
{
throw new UnsupportedOperationException();
}
},
new DataSegmentKiller()
{
@Override
public void kill(DataSegment segments)
{
}
@Override
public void killAll()
{
throw new UnsupportedOperationException("not implemented");
}
},
new DataSegmentMover()
{
@Override
public DataSegment move(DataSegment dataSegment, Map<String, Object> targetLoadSpec)
{
return dataSegment;
}
},
new DataSegmentArchiver()
{
@Override
public DataSegment archive(DataSegment segment)
{
return segment;
}
@Override
public DataSegment restore(DataSegment segment)
{
return segment;
}
},
null, // segment announcer
null,
notifierFactory,
null, // query runner factory conglomerate corporation unionized collective
null, // query executor service
null, // monitor scheduler
new SegmentLoaderFactory(
new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, MAPPER)
),
MAPPER,
INDEX_IO,
null,
null,
null,
INDEX_MERGER_V9,
null,
null,
null,
null,
new NoopTestTaskFileWriter()
);
Collection<Object[]> values = new ArrayList<>(); Collection<Object[]> values = new ArrayList<>();
for (InputRowParser parser : Arrays.<InputRowParser>asList( for (InputRowParser parser : Arrays.<InputRowParser>asList(
ROW_PARSER, ROW_PARSER,
@ -342,27 +193,35 @@ public class IngestSegmentFirehoseFactoryTest
null, null,
ImmutableList.of(METRIC_LONG_NAME, METRIC_FLOAT_NAME) ImmutableList.of(METRIC_LONG_NAME, METRIC_FLOAT_NAME)
)) { )) {
final IngestSegmentFirehoseFactory factory = new IngestSegmentFirehoseFactory( for (Boolean wrapInCombining : Arrays.asList(false, true)) {
TASK.getDataSource(), final IngestSegmentFirehoseFactory isfFactory = new IngestSegmentFirehoseFactory(
Intervals.ETERNITY, TASK.getDataSource(),
new SelectorDimFilter(DIM_NAME, DIM_VALUE, null), Intervals.ETERNITY,
dim_names, new SelectorDimFilter(DIM_NAME, DIM_VALUE, null),
metric_names, dim_names,
INDEX_IO metric_names,
); INDEX_IO,
factory.setTaskToolbox(taskToolboxFactory.build(TASK)); cc,
values.add( slf,
new Object[]{ retryPolicyFactory
StringUtils.format( );
"DimNames[%s]MetricNames[%s]ParserDimNames[%s]", final FirehoseFactory factory = wrapInCombining
dim_names == null ? "null" : "dims", ? new CombiningFirehoseFactory(ImmutableList.of(isfFactory))
metric_names == null ? "null" : "metrics", : isfFactory;
parser == ROW_PARSER ? "dims" : "null" values.add(
), new Object[]{
factory, StringUtils.format(
parser "DimNames[%s]MetricNames[%s]ParserDimNames[%s]WrapInCombining[%s]",
} dim_names == null ? "null" : "dims",
); metric_names == null ? "null" : "metrics",
parser == ROW_PARSER ? "dims" : "null",
wrapInCombining
),
factory,
parser
}
);
}
} }
} }
} }
@ -407,7 +266,7 @@ public class IngestSegmentFirehoseFactoryTest
public IngestSegmentFirehoseFactoryTest( public IngestSegmentFirehoseFactoryTest(
String testName, String testName,
IngestSegmentFirehoseFactory factory, FirehoseFactory factory,
InputRowParser rowParser InputRowParser rowParser
) )
{ {
@ -436,7 +295,7 @@ public class IngestSegmentFirehoseFactoryTest
private static final File persistDir = Paths.get(tmpDir.getAbsolutePath(), "indexTestMerger").toFile(); private static final File persistDir = Paths.get(tmpDir.getAbsolutePath(), "indexTestMerger").toFile();
private static final List<DataSegment> segmentSet = new ArrayList<>(MAX_SHARD_NUMBER); private static final List<DataSegment> segmentSet = new ArrayList<>(MAX_SHARD_NUMBER);
private final IngestSegmentFirehoseFactory factory; private final FirehoseFactory<InputRowParser> factory;
private final InputRowParser rowParser; private final InputRowParser rowParser;
private static final InputRowParser<Map<String, Object>> ROW_PARSER = new MapInputRowParser( private static final InputRowParser<Map<String, Object>> ROW_PARSER = new MapInputRowParser(
@ -518,15 +377,20 @@ public class IngestSegmentFirehoseFactoryTest
@Test @Test
public void sanityTest() public void sanityTest()
{ {
Assert.assertEquals(TASK.getDataSource(), factory.getDataSource()); if (factory instanceof CombiningFirehoseFactory) {
if (factory.getDimensions() != null) { // This method tests IngestSegmentFirehoseFactory-specific methods.
Assert.assertArrayEquals(new String[]{DIM_NAME}, factory.getDimensions().toArray()); return;
} }
Assert.assertEquals(Intervals.ETERNITY, factory.getInterval()); final IngestSegmentFirehoseFactory isfFactory = (IngestSegmentFirehoseFactory) factory;
if (factory.getMetrics() != null) { Assert.assertEquals(TASK.getDataSource(), isfFactory.getDataSource());
if (isfFactory.getDimensions() != null) {
Assert.assertArrayEquals(new String[]{DIM_NAME}, isfFactory.getDimensions().toArray());
}
Assert.assertEquals(Intervals.ETERNITY, isfFactory.getInterval());
if (isfFactory.getMetrics() != null) {
Assert.assertEquals( Assert.assertEquals(
ImmutableSet.of(METRIC_LONG_NAME, METRIC_FLOAT_NAME), ImmutableSet.of(METRIC_LONG_NAME, METRIC_FLOAT_NAME),
ImmutableSet.copyOf(factory.getMetrics()) ImmutableSet.copyOf(isfFactory.getMetrics())
); );
} }
} }
@ -536,15 +400,17 @@ public class IngestSegmentFirehoseFactoryTest
{ {
Assert.assertEquals(MAX_SHARD_NUMBER.longValue(), segmentSet.size()); Assert.assertEquals(MAX_SHARD_NUMBER.longValue(), segmentSet.size());
Integer rowcount = 0; Integer rowcount = 0;
try (final IngestSegmentFirehose firehose = try (final Firehose firehose = factory.connect(rowParser, null)) {
(IngestSegmentFirehose)
factory.connect(rowParser, null)) {
while (firehose.hasMore()) { while (firehose.hasMore()) {
InputRow row = firehose.nextRow(); InputRow row = firehose.nextRow();
Assert.assertArrayEquals(new String[]{DIM_NAME}, row.getDimensions().toArray()); Assert.assertArrayEquals(new String[]{DIM_NAME}, row.getDimensions().toArray());
Assert.assertArrayEquals(new String[]{DIM_VALUE}, row.getDimension(DIM_NAME).toArray()); Assert.assertArrayEquals(new String[]{DIM_VALUE}, row.getDimension(DIM_NAME).toArray());
Assert.assertEquals(METRIC_LONG_VALUE.longValue(), row.getMetric(METRIC_LONG_NAME)); Assert.assertEquals(METRIC_LONG_VALUE.longValue(), row.getMetric(METRIC_LONG_NAME));
Assert.assertEquals(METRIC_FLOAT_VALUE, row.getMetric(METRIC_FLOAT_NAME).floatValue(), METRIC_FLOAT_VALUE * 0.0001); Assert.assertEquals(
METRIC_FLOAT_VALUE,
row.getMetric(METRIC_FLOAT_NAME).floatValue(),
METRIC_FLOAT_VALUE * 0.0001
);
++rowcount; ++rowcount;
} }
} }
@ -563,9 +429,8 @@ public class IngestSegmentFirehoseFactoryTest
) )
); );
int skipped = 0; int skipped = 0;
try (final IngestSegmentFirehose firehose = try (final Firehose firehose =
(IngestSegmentFirehose) factory.connect(transformSpec.decorate(rowParser), null)) {
factory.connect(transformSpec.decorate(rowParser), null)) {
while (firehose.hasMore()) { while (firehose.hasMore()) {
InputRow row = firehose.nextRow(); InputRow row = firehose.nextRow();
if (row == null) { if (row == null) {

View File

@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.common.io.Files; import com.google.common.io.Files;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.apache.druid.client.coordinator.CoordinatorClient;
import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.Firehose;
import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.MapBasedInputRow;
@ -34,20 +35,10 @@ import org.apache.druid.data.input.impl.InputRowParser;
import org.apache.druid.data.input.impl.JSONParseSpec; import org.apache.druid.data.input.impl.JSONParseSpec;
import org.apache.druid.data.input.impl.MapInputRowParser; import org.apache.druid.data.input.impl.MapInputRowParser;
import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.indexing.common.RetryPolicyConfig;
import org.apache.druid.indexing.common.RetryPolicyFactory;
import org.apache.druid.indexing.common.SegmentLoaderFactory; import org.apache.druid.indexing.common.SegmentLoaderFactory;
import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskLockType;
import org.apache.druid.indexing.common.TaskToolboxFactory;
import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.indexing.common.actions.LockAcquireAction;
import org.apache.druid.indexing.common.actions.SegmentListUsedAction;
import org.apache.druid.indexing.common.actions.TaskAction;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.actions.TaskActionClientFactory;
import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.indexing.common.task.NoopTask;
import org.apache.druid.indexing.common.task.NoopTestTaskFileWriter;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.JodaUtils;
@ -59,12 +50,8 @@ import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.incremental.IndexSizeExceededException;
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.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory;
import org.apache.druid.segment.transform.TransformSpec; 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.DataSegment;
import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.partition.LinearShardSpec; import org.apache.druid.timeline.partition.LinearShardSpec;
@ -165,7 +152,7 @@ public class IngestSegmentFirehoseFactoryTimelineTest
FileUtils.deleteDirectory(tmpDir); FileUtils.deleteDirectory(tmpDir);
} }
private static TestCase TC( private static TestCase tc(
String intervalString, String intervalString,
int expectedCount, int expectedCount,
long expectedSum, long expectedSum,
@ -187,7 +174,7 @@ public class IngestSegmentFirehoseFactoryTimelineTest
); );
} }
private static DataSegmentMaker DS( private static DataSegmentMaker ds(
String intervalString, String intervalString,
String version, String version,
int partitionNum, int partitionNum,
@ -197,7 +184,7 @@ public class IngestSegmentFirehoseFactoryTimelineTest
return new DataSegmentMaker(Intervals.of(intervalString), version, partitionNum, Arrays.asList(rows)); return new DataSegmentMaker(Intervals.of(intervalString), version, partitionNum, Arrays.asList(rows));
} }
private static InputRow IR(String timeString, long metricValue) private static InputRow ir(String timeString, long metricValue)
{ {
return new MapBasedInputRow( return new MapBasedInputRow(
DateTimes.of(timeString).getMillis(), DateTimes.of(timeString).getMillis(),
@ -249,115 +236,68 @@ public class IngestSegmentFirehoseFactoryTimelineTest
public static Collection<Object[]> constructorFeeder() public static Collection<Object[]> constructorFeeder()
{ {
final List<TestCase> testCases = ImmutableList.of( final List<TestCase> testCases = ImmutableList.of(
TC( tc(
"2000/2000T02", 3, 7, "2000/2000T02", 3, 7,
DS("2000/2000T01", "v1", 0, IR("2000", 1), IR("2000T00:01", 2)), ds("2000/2000T01", "v1", 0, ir("2000", 1), ir("2000T00:01", 2)),
DS("2000T01/2000T02", "v1", 0, IR("2000T01", 4)) ds("2000T01/2000T02", "v1", 0, ir("2000T01", 4))
) /* Adjacent segments */, ) /* Adjacent segments */,
TC( tc(
"2000/2000T02", 3, 7, "2000/2000T02", 3, 7,
DS("2000/2000T02", "v1", 0, IR("2000", 1), IR("2000T00:01", 2), IR("2000T01", 8)), ds("2000/2000T02", "v1", 0, ir("2000", 1), ir("2000T00:01", 2), ir("2000T01", 8)),
DS("2000T01/2000T02", "v2", 0, IR("2000T01:01", 4)) ds("2000T01/2000T02", "v2", 0, ir("2000T01:01", 4))
) /* 1H segment overlaid on top of 2H segment */, ) /* 1H segment overlaid on top of 2H segment */,
TC( tc(
"2000/2000-01-02", 4, 23, "2000/2000-01-02", 4, 23,
DS("2000/2000-01-02", "v1", 0, IR("2000", 1), IR("2000T00:01", 2), IR("2000T01", 8), IR("2000T02", 16)), ds("2000/2000-01-02", "v1", 0, ir("2000", 1), ir("2000T00:01", 2), ir("2000T01", 8), ir("2000T02", 16)),
DS("2000T01/2000T02", "v2", 0, IR("2000T01:01", 4)) ds("2000T01/2000T02", "v2", 0, ir("2000T01:01", 4))
) /* 1H segment overlaid on top of 1D segment */, ) /* 1H segment overlaid on top of 1D segment */,
TC( tc(
"2000/2000T02", 4, 15, "2000/2000T02", 4, 15,
DS("2000/2000T02", "v1", 0, IR("2000", 1), IR("2000T00:01", 2), IR("2000T01", 8)), ds("2000/2000T02", "v1", 0, ir("2000", 1), ir("2000T00:01", 2), ir("2000T01", 8)),
DS("2000/2000T02", "v1", 1, IR("2000T01:01", 4)) ds("2000/2000T02", "v1", 1, ir("2000T01:01", 4))
) /* Segment set with two segments for the same interval */, ) /* Segment set with two segments for the same interval */,
TC( tc(
"2000T01/2000T02", 1, 2, "2000T01/2000T02", 1, 2,
DS("2000/2000T03", "v1", 0, IR("2000", 1), IR("2000T01", 2), IR("2000T02", 4)) ds("2000/2000T03", "v1", 0, ir("2000", 1), ir("2000T01", 2), ir("2000T02", 4))
) /* Segment wider than desired interval */, ) /* Segment wider than desired interval */,
TC( tc(
"2000T02/2000T04", 2, 12, "2000T02/2000T04", 2, 12,
DS("2000/2000T03", "v1", 0, IR("2000", 1), IR("2000T01", 2), IR("2000T02", 4)), ds("2000/2000T03", "v1", 0, ir("2000", 1), ir("2000T01", 2), ir("2000T02", 4)),
DS("2000T03/2000T04", "v1", 0, IR("2000T03", 8)) ds("2000T03/2000T04", "v1", 0, ir("2000T03", 8))
) /* Segment intersecting desired interval */ ) /* Segment intersecting desired interval */
); );
final List<Object[]> constructors = new ArrayList<>(); final List<Object[]> constructors = new ArrayList<>();
for (final TestCase testCase : testCases) { for (final TestCase testCase : testCases) {
final TaskActionClient taskActionClient = new TaskActionClient() SegmentHandoffNotifierFactory notifierFactory = EasyMock.createNiceMock(SegmentHandoffNotifierFactory.class);
EasyMock.replay(notifierFactory);
final SegmentLoaderFactory slf = new SegmentLoaderFactory(null, MAPPER);
final RetryPolicyFactory retryPolicyFactory = new RetryPolicyFactory(new RetryPolicyConfig());
final CoordinatorClient cc = new CoordinatorClient(null, null)
{ {
@Override @Override
public <RetType> RetType submit(TaskAction<RetType> taskAction) public List<DataSegment> getDatabaseSegmentDataSourceSegments(String dataSource, List<Interval> intervals)
{ {
if (taskAction instanceof SegmentListUsedAction) { // Expect the interval we asked for
// Expect the interval we asked for if (intervals.equals(ImmutableList.of(testCase.interval))) {
final SegmentListUsedAction action = (SegmentListUsedAction) taskAction; return ImmutableList.copyOf(testCase.segments);
if (action.getIntervals().equals(ImmutableList.of(testCase.interval))) {
return (RetType) ImmutableList.copyOf(testCase.segments);
} else {
throw new IllegalArgumentException("WTF");
}
} else if (taskAction instanceof LockAcquireAction) {
return (RetType) new TaskLock(TaskLockType.EXCLUSIVE, null, DATA_SOURCE, Intervals.of("2000/2001"), "v1", 0);
} else { } else {
throw new UnsupportedOperationException(); throw new IllegalArgumentException("WTF");
} }
} }
}; };
SegmentHandoffNotifierFactory notifierFactory = EasyMock.createNiceMock(SegmentHandoffNotifierFactory.class);
EasyMock.replay(notifierFactory);
SegmentLoaderConfig segmentLoaderConfig = new SegmentLoaderConfig()
{
@Override
public List<StorageLocationConfig> getLocations()
{
return new ArrayList<>();
}
};
final TaskToolboxFactory taskToolboxFactory = new TaskToolboxFactory(
new TaskConfig(testCase.tmpDir.getAbsolutePath(), null, null, 50000, null, false, null, null),
new TaskActionClientFactory()
{
@Override
public TaskActionClient create(Task task)
{
return taskActionClient;
}
},
new NoopServiceEmitter(),
null, // segment pusher
null, // segment killer
null, // segment mover
null, // segment archiver
null, // segment announcer,
null,
notifierFactory,
null, // query runner factory conglomerate corporation unionized collective
null, // query executor service
null, // monitor scheduler
new SegmentLoaderFactory(
new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, MAPPER)
),
MAPPER,
INDEX_IO,
null,
null,
null,
INDEX_MERGER_V9,
null,
null,
null,
null,
new NoopTestTaskFileWriter()
);
final IngestSegmentFirehoseFactory factory = new IngestSegmentFirehoseFactory( final IngestSegmentFirehoseFactory factory = new IngestSegmentFirehoseFactory(
DATA_SOURCE, DATA_SOURCE,
testCase.interval, testCase.interval,
new TrueDimFilter(), new TrueDimFilter(),
Arrays.asList(DIMENSIONS), Arrays.asList(DIMENSIONS),
Arrays.asList(METRICS), Arrays.asList(METRICS),
INDEX_IO INDEX_IO,
cc,
slf,
retryPolicyFactory
); );
factory.setTaskToolbox(taskToolboxFactory.build(NoopTask.create(DATA_SOURCE)));
constructors.add( constructors.add(
new Object[]{ new Object[]{

View File

@ -37,7 +37,6 @@ import org.apache.druid.segment.loading.NoopDataSegmentArchiver;
import org.apache.druid.segment.loading.NoopDataSegmentKiller; import org.apache.druid.segment.loading.NoopDataSegmentKiller;
import org.apache.druid.segment.loading.NoopDataSegmentMover; import org.apache.druid.segment.loading.NoopDataSegmentMover;
import org.apache.druid.segment.loading.NoopDataSegmentPusher; import org.apache.druid.segment.loading.NoopDataSegmentPusher;
import org.apache.druid.segment.loading.SegmentLoaderLocalCacheManager;
import org.apache.druid.server.DruidNode; import org.apache.druid.server.DruidNode;
import org.apache.druid.server.coordination.NoopDataSegmentAnnouncer; import org.apache.druid.server.coordination.NoopDataSegmentAnnouncer;
import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.initialization.ServerConfig;
@ -94,7 +93,7 @@ public class SingleTaskBackgroundRunnerTest
null, null,
null, null,
null, null,
new SegmentLoaderFactory(EasyMock.createMock(SegmentLoaderLocalCacheManager.class)), new SegmentLoaderFactory(null, utils.getTestObjectMapper()),
utils.getTestObjectMapper(), utils.getTestObjectMapper(),
utils.getTestIndexIO(), utils.getTestIndexIO(),
null, null,

View File

@ -106,7 +106,6 @@ import org.apache.druid.segment.loading.DataSegmentPusher;
import org.apache.druid.segment.loading.LocalDataSegmentKiller; import org.apache.druid.segment.loading.LocalDataSegmentKiller;
import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig;
import org.apache.druid.segment.loading.SegmentLoaderConfig; 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.loading.StorageLocationConfig;
import org.apache.druid.segment.realtime.FireDepartment; import org.apache.druid.segment.realtime.FireDepartment;
import org.apache.druid.segment.realtime.FireDepartmentTest; import org.apache.druid.segment.realtime.FireDepartmentTest;
@ -195,16 +194,16 @@ public class TaskLifecycleTest
private static DateTime now = DateTimes.nowUtc(); private static DateTime now = DateTimes.nowUtc();
private static final Iterable<InputRow> realtimeIdxTaskInputRows = ImmutableList.of( private static final Iterable<InputRow> realtimeIdxTaskInputRows = ImmutableList.of(
IR(now.toString("YYYY-MM-dd'T'HH:mm:ss"), "test_dim1", "test_dim2", 1.0f), ir(now.toString("YYYY-MM-dd'T'HH:mm:ss"), "test_dim1", "test_dim2", 1.0f),
IR(now.plus(new Period(Hours.ONE)).toString("YYYY-MM-dd'T'HH:mm:ss"), "test_dim1", "test_dim2", 2.0f), ir(now.plus(new Period(Hours.ONE)).toString("YYYY-MM-dd'T'HH:mm:ss"), "test_dim1", "test_dim2", 2.0f),
IR(now.plus(new Period(Hours.TWO)).toString("YYYY-MM-dd'T'HH:mm:ss"), "test_dim1", "test_dim2", 3.0f) ir(now.plus(new Period(Hours.TWO)).toString("YYYY-MM-dd'T'HH:mm:ss"), "test_dim1", "test_dim2", 3.0f)
); );
private static final Iterable<InputRow> IdxTaskInputRows = ImmutableList.of( private static final Iterable<InputRow> IdxTaskInputRows = ImmutableList.of(
IR("2010-01-01T01", "x", "y", 1), ir("2010-01-01T01", "x", "y", 1),
IR("2010-01-01T01", "x", "z", 1), ir("2010-01-01T01", "x", "z", 1),
IR("2010-01-02T01", "a", "b", 2), ir("2010-01-02T01", "a", "b", 2),
IR("2010-01-02T01", "a", "c", 1) ir("2010-01-02T01", "a", "c", 1)
); );
@Rule @Rule
@ -241,7 +240,7 @@ public class TaskLifecycleTest
return new NoopServiceEmitter(); return new NoopServiceEmitter();
} }
private static InputRow IR(String dt, String dim1, String dim2, float met) private static InputRow ir(String dt, String dim1, String dim2, float met)
{ {
return new MapBasedInputRow( return new MapBasedInputRow(
DateTimes.of(dt).getMillis(), DateTimes.of(dt).getMillis(),
@ -611,9 +610,7 @@ public class TaskLifecycleTest
() -> queryRunnerFactoryConglomerate, // query runner factory conglomerate corporation unionized collective () -> queryRunnerFactoryConglomerate, // query runner factory conglomerate corporation unionized collective
Execs.directExecutor(), // query executor service Execs.directExecutor(), // query executor service
monitorScheduler, // monitor scheduler monitorScheduler, // monitor scheduler
new SegmentLoaderFactory( new SegmentLoaderFactory(null, new DefaultObjectMapper()),
new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, new DefaultObjectMapper())
),
MAPPER, MAPPER,
INDEX_IO, INDEX_IO,
MapCache.create(0), MapCache.create(0),

View File

@ -41,7 +41,6 @@ import org.apache.druid.indexing.overlord.TestTaskRunner;
import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexMergerV9;
import org.apache.druid.segment.loading.SegmentLoaderConfig; 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.loading.StorageLocationConfig;
import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory;
import org.apache.druid.server.coordination.ChangeRequestHistory; import org.apache.druid.server.coordination.ChangeRequestHistory;
@ -120,7 +119,7 @@ public class WorkerTaskManagerTest
null, null,
null, null,
null, null,
new SegmentLoaderFactory(new SegmentLoaderLocalCacheManager(null, loaderConfig, jsonMapper)), new SegmentLoaderFactory(null, jsonMapper),
jsonMapper, jsonMapper,
indexIO, indexIO,
null, null,

View File

@ -46,9 +46,6 @@ import org.apache.druid.indexing.overlord.TestRemoteTaskRunnerConfig;
import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexMergerV9;
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.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory;
import org.apache.druid.server.DruidNode; import org.apache.druid.server.DruidNode;
import org.apache.druid.server.initialization.IndexerZkConfig; import org.apache.druid.server.initialization.IndexerZkConfig;
@ -62,10 +59,10 @@ import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import java.util.ArrayList;
import java.util.List; import java.util.List;
/** /**
*
*/ */
public class WorkerTaskMonitorTest public class WorkerTaskMonitorTest
{ {
@ -169,20 +166,8 @@ public class WorkerTaskMonitorTest
new TaskToolboxFactory( new TaskToolboxFactory(
taskConfig, taskConfig,
taskActionClientFactory, taskActionClientFactory,
null, null, null, null, null, null, null, notifierFactory, null, null, null, new SegmentLoaderFactory( null, null, null, null, null, null, null, notifierFactory, null, null, null,
new SegmentLoaderLocalCacheManager( new SegmentLoaderFactory(null, jsonMapper),
null,
new SegmentLoaderConfig()
{
@Override
public List<StorageLocationConfig> getLocations()
{
return new ArrayList<>();
}
},
jsonMapper
)
),
jsonMapper, jsonMapper,
indexIO, indexIO,
null, null,

View File

@ -42,6 +42,7 @@ import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ComplexColumn; import org.apache.druid.segment.column.ComplexColumn;
import org.apache.druid.segment.column.DictionaryEncodedColumn;
import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.data.IndexedInts;
import org.apache.druid.segment.serde.ComplexMetricSerde; import org.apache.druid.segment.serde.ComplexMetricSerde;
@ -194,30 +195,38 @@ public class SegmentAnalyzer
final ColumnHolder columnHolder final ColumnHolder columnHolder
) )
{ {
long size = 0;
Comparable min = null; Comparable min = null;
Comparable max = null; Comparable max = null;
long size = 0;
final int cardinality;
if (capabilities.hasBitmapIndexes()) {
final BitmapIndex bitmapIndex = columnHolder.getBitmapIndex();
cardinality = bitmapIndex.getCardinality();
if (!capabilities.hasBitmapIndexes()) { if (analyzingSize()) {
return ColumnAnalysis.error("string_no_bitmap"); for (int i = 0; i < cardinality; ++i) {
} String value = bitmapIndex.getValue(i);
if (value != null) {
final BitmapIndex bitmapIndex = columnHolder.getBitmapIndex(); size += StringUtils.estimatedBinaryLengthAsUTF8(value) * bitmapIndex.getBitmap(bitmapIndex.getIndex(value))
final int cardinality = bitmapIndex.getCardinality(); .size();
}
if (analyzingSize()) {
for (int i = 0; i < cardinality; ++i) {
String value = bitmapIndex.getValue(i);
if (value != null) {
size += StringUtils.estimatedBinaryLengthAsUTF8(value) * bitmapIndex.getBitmap(bitmapIndex.getIndex(value)).size();
} }
} }
}
if (analyzingMinMax() && cardinality > 0) { if (analyzingMinMax() && cardinality > 0) {
min = NullHandling.nullToEmptyIfNeeded(bitmapIndex.getValue(0)); min = NullHandling.nullToEmptyIfNeeded(bitmapIndex.getValue(0));
max = NullHandling.nullToEmptyIfNeeded(bitmapIndex.getValue(cardinality - 1)); max = NullHandling.nullToEmptyIfNeeded(bitmapIndex.getValue(cardinality - 1));
}
} else if (capabilities.isDictionaryEncoded()) {
// fallback if no bitmap index
DictionaryEncodedColumn<String> theColumn = (DictionaryEncodedColumn<String>) columnHolder.getColumn();
cardinality = theColumn.getCardinality();
if (analyzingMinMax() && cardinality > 0) {
min = NullHandling.nullToEmptyIfNeeded(theColumn.lookupName(0));
max = NullHandling.nullToEmptyIfNeeded(theColumn.lookupName(cardinality - 1));
}
} else {
cardinality = 0;
} }
return new ColumnAnalysis( return new ColumnAnalysis(

View File

@ -394,7 +394,7 @@ public class GroupByLimitPushDownInsufficientBufferTest
strategySelector, strategySelector,
new GroupByQueryQueryToolChest( new GroupByQueryQueryToolChest(
strategySelector, strategySelector,
NoopIntervalChunkingQueryRunnerDecorator() noopIntervalChunkingQueryRunnerDecorator()
) )
); );
@ -402,7 +402,7 @@ public class GroupByLimitPushDownInsufficientBufferTest
tooSmallStrategySelector, tooSmallStrategySelector,
new GroupByQueryQueryToolChest( new GroupByQueryQueryToolChest(
tooSmallStrategySelector, tooSmallStrategySelector,
NoopIntervalChunkingQueryRunnerDecorator() noopIntervalChunkingQueryRunnerDecorator()
) )
); );
} }
@ -679,7 +679,7 @@ public class GroupByLimitPushDownInsufficientBufferTest
} }
}; };
public static IntervalChunkingQueryRunnerDecorator NoopIntervalChunkingQueryRunnerDecorator() public static IntervalChunkingQueryRunnerDecorator noopIntervalChunkingQueryRunnerDecorator()
{ {
return new IntervalChunkingQueryRunnerDecorator(null, null, null) return new IntervalChunkingQueryRunnerDecorator(null, null, null)
{ {

View File

@ -426,7 +426,7 @@ public class GroupByLimitPushDownMultiNodeMergeTest
strategySelector, strategySelector,
new GroupByQueryQueryToolChest( new GroupByQueryQueryToolChest(
strategySelector, strategySelector,
NoopIntervalChunkingQueryRunnerDecorator() noopIntervalChunkingQueryRunnerDecorator()
) )
); );
@ -434,7 +434,7 @@ public class GroupByLimitPushDownMultiNodeMergeTest
strategySelector2, strategySelector2,
new GroupByQueryQueryToolChest( new GroupByQueryQueryToolChest(
strategySelector2, strategySelector2,
NoopIntervalChunkingQueryRunnerDecorator() noopIntervalChunkingQueryRunnerDecorator()
) )
); );
} }
@ -780,7 +780,7 @@ public class GroupByLimitPushDownMultiNodeMergeTest
} }
}; };
public static IntervalChunkingQueryRunnerDecorator NoopIntervalChunkingQueryRunnerDecorator() public static IntervalChunkingQueryRunnerDecorator noopIntervalChunkingQueryRunnerDecorator()
{ {
return new IntervalChunkingQueryRunnerDecorator(null, null, null) return new IntervalChunkingQueryRunnerDecorator(null, null, null)
{ {

View File

@ -287,7 +287,7 @@ public class GroupByMultiSegmentTest
strategySelector, strategySelector,
new GroupByQueryQueryToolChest( new GroupByQueryQueryToolChest(
strategySelector, strategySelector,
NoopIntervalChunkingQueryRunnerDecorator() noopIntervalChunkingQueryRunnerDecorator()
) )
); );
} }
@ -419,7 +419,7 @@ public class GroupByMultiSegmentTest
} }
}; };
public static IntervalChunkingQueryRunnerDecorator NoopIntervalChunkingQueryRunnerDecorator() public static IntervalChunkingQueryRunnerDecorator noopIntervalChunkingQueryRunnerDecorator()
{ {
return new IntervalChunkingQueryRunnerDecorator(null, null, null) { return new IntervalChunkingQueryRunnerDecorator(null, null, null) {
@Override @Override

View File

@ -36,7 +36,7 @@ public class LookupConfigTest
public TemporaryFolder temporaryFolder = new TemporaryFolder(); public TemporaryFolder temporaryFolder = new TemporaryFolder();
@Test @Test
public void TestSerDesr() throws IOException public void testSerDesr() throws IOException
{ {
LookupConfig lookupConfig = new LookupConfig(temporaryFolder.newFile().getAbsolutePath()); LookupConfig lookupConfig = new LookupConfig(temporaryFolder.newFile().getAbsolutePath());
Assert.assertEquals( Assert.assertEquals(

View File

@ -83,10 +83,16 @@ public class SegmentMetadataQueryTest
public static QueryRunner makeMMappedQueryRunner( public static QueryRunner makeMMappedQueryRunner(
SegmentId segmentId, SegmentId segmentId,
boolean rollup, boolean rollup,
boolean bitmaps,
QueryRunnerFactory factory QueryRunnerFactory factory
) )
{ {
QueryableIndex index = rollup ? TestIndex.getMMappedTestIndex() : TestIndex.getNoRollupMMappedTestIndex(); QueryableIndex index;
if (bitmaps) {
index = rollup ? TestIndex.getMMappedTestIndex() : TestIndex.getNoRollupMMappedTestIndex();
} else {
index = TestIndex.getNoBitmapMMappedTestIndex();
}
return QueryRunnerTestHelper.makeQueryRunner( return QueryRunnerTestHelper.makeQueryRunner(
factory, factory,
segmentId, segmentId,
@ -99,10 +105,16 @@ public class SegmentMetadataQueryTest
public static QueryRunner makeIncrementalIndexQueryRunner( public static QueryRunner makeIncrementalIndexQueryRunner(
SegmentId segmentId, SegmentId segmentId,
boolean rollup, boolean rollup,
boolean bitmaps,
QueryRunnerFactory factory QueryRunnerFactory factory
) )
{ {
IncrementalIndex index = rollup ? TestIndex.getIncrementalTestIndex() : TestIndex.getNoRollupIncrementalTestIndex(); IncrementalIndex index;
if (bitmaps) {
index = rollup ? TestIndex.getIncrementalTestIndex() : TestIndex.getNoRollupIncrementalTestIndex();
} else {
index = TestIndex.getNoBitmapIncrementalTestIndex();
}
return QueryRunnerTestHelper.makeQueryRunner( return QueryRunnerTestHelper.makeQueryRunner(
factory, factory,
segmentId, segmentId,
@ -121,17 +133,19 @@ public class SegmentMetadataQueryTest
private final SegmentMetadataQuery testQuery; private final SegmentMetadataQuery testQuery;
private final SegmentAnalysis expectedSegmentAnalysis1; private final SegmentAnalysis expectedSegmentAnalysis1;
private final SegmentAnalysis expectedSegmentAnalysis2; private final SegmentAnalysis expectedSegmentAnalysis2;
private final boolean bitmaps;
@Parameterized.Parameters(name = "mmap1 = {0}, mmap2 = {1}, rollup1 = {2}, rollup2 = {3}, differentIds = {4}") @Parameterized.Parameters(name = "mmap1 = {0}, mmap2 = {1}, rollup1 = {2}, rollup2 = {3}, differentIds = {4}, bitmaps={5}")
public static Collection<Object[]> constructorFeeder() public static Collection<Object[]> constructorFeeder()
{ {
return ImmutableList.of( return ImmutableList.of(
new Object[]{true, true, true, true, false}, new Object[]{true, true, true, true, false, true},
new Object[]{true, false, true, false, false}, new Object[]{true, false, true, false, false, true},
new Object[]{false, true, true, false, false}, new Object[]{false, true, true, false, false, true},
new Object[]{false, false, false, false, false}, new Object[]{false, false, false, false, false, true},
new Object[]{false, false, true, true, false}, new Object[]{false, false, true, true, false, true},
new Object[]{false, false, false, true, true} new Object[]{false, false, false, true, true, true},
new Object[]{true, true, false, false, false, false}
); );
} }
@ -140,22 +154,24 @@ public class SegmentMetadataQueryTest
boolean mmap2, boolean mmap2,
boolean rollup1, boolean rollup1,
boolean rollup2, boolean rollup2,
boolean differentIds boolean differentIds,
boolean bitmaps
) )
{ {
final SegmentId id1 = SegmentId.dummy(differentIds ? "testSegment1" : "testSegment"); final SegmentId id1 = SegmentId.dummy(differentIds ? "testSegment1" : "testSegment");
final SegmentId id2 = SegmentId.dummy(differentIds ? "testSegment2" : "testSegment"); final SegmentId id2 = SegmentId.dummy(differentIds ? "testSegment2" : "testSegment");
this.runner1 = mmap1 this.runner1 = mmap1
? makeMMappedQueryRunner(id1, rollup1, FACTORY) ? makeMMappedQueryRunner(id1, rollup1, bitmaps, FACTORY)
: makeIncrementalIndexQueryRunner(id1, rollup1, FACTORY); : makeIncrementalIndexQueryRunner(id1, rollup1, bitmaps, FACTORY);
this.runner2 = mmap2 this.runner2 = mmap2
? makeMMappedQueryRunner(id2, rollup2, FACTORY) ? makeMMappedQueryRunner(id2, rollup2, bitmaps, FACTORY)
: makeIncrementalIndexQueryRunner(id2, rollup2, FACTORY); : makeIncrementalIndexQueryRunner(id2, rollup2, bitmaps, FACTORY);
this.mmap1 = mmap1; this.mmap1 = mmap1;
this.mmap2 = mmap2; this.mmap2 = mmap2;
this.rollup1 = rollup1; this.rollup1 = rollup1;
this.rollup2 = rollup2; this.rollup2 = rollup2;
this.differentIds = differentIds; this.differentIds = differentIds;
this.bitmaps = bitmaps;
testQuery = Druids.newSegmentMetadataQueryBuilder() testQuery = Druids.newSegmentMetadataQueryBuilder()
.dataSource("testing") .dataSource("testing")
.intervals("2013/2014") .intervals("2013/2014")
@ -169,6 +185,16 @@ public class SegmentMetadataQueryTest
.merge(true) .merge(true)
.build(); .build();
int preferedSize1 = 0;
int placementSize2 = 0;
int overallSize1 = 119691;
int overallSize2 = 119691;
if (bitmaps) {
preferedSize1 = mmap1 ? 10881 : 10764;
placementSize2 = mmap2 ? 10881 : 0;
overallSize1 = mmap1 ? 167493 : 168188;
overallSize2 = mmap2 ? 167493 : 168188;
}
expectedSegmentAnalysis1 = new SegmentAnalysis( expectedSegmentAnalysis1 = new SegmentAnalysis(
id1.toString(), id1.toString(),
ImmutableList.of(Intervals.of("2011-01-12T00:00:00.000Z/2011-04-15T00:00:00.001Z")), ImmutableList.of(Intervals.of("2011-01-12T00:00:00.000Z/2011-04-15T00:00:00.001Z")),
@ -187,7 +213,7 @@ public class SegmentMetadataQueryTest
new ColumnAnalysis( new ColumnAnalysis(
ValueType.STRING.toString(), ValueType.STRING.toString(),
false, false,
mmap1 ? 10881 : 10764, preferedSize1,
1, 1,
"preferred", "preferred",
"preferred", "preferred",
@ -203,7 +229,7 @@ public class SegmentMetadataQueryTest
null, null,
null null
) )
), mmap1 ? 167493 : 168188, ), overallSize1,
1209, 1209,
null, null,
null, null,
@ -228,7 +254,7 @@ public class SegmentMetadataQueryTest
new ColumnAnalysis( new ColumnAnalysis(
ValueType.STRING.toString(), ValueType.STRING.toString(),
false, false,
mmap2 ? 10881 : 0, placementSize2,
1, 1,
null, null,
null, null,
@ -245,7 +271,7 @@ public class SegmentMetadataQueryTest
null null
) )
// null_column will be included only for incremental index, which makes a little bigger result than expected // null_column will be included only for incremental index, which makes a little bigger result than expected
), mmap2 ? 167493 : 168188, ), overallSize2,
1209, 1209,
null, null,
null, null,
@ -470,10 +496,16 @@ public class SegmentMetadataQueryTest
@Test @Test
public void testSegmentMetadataQueryWithDefaultAnalysisMerge() public void testSegmentMetadataQueryWithDefaultAnalysisMerge()
{ {
int size1 = 0;
int size2 = 0;
if (bitmaps) {
size1 = mmap1 ? 10881 : 10764;
size2 = mmap2 ? 10881 : 10764;
}
ColumnAnalysis analysis = new ColumnAnalysis( ColumnAnalysis analysis = new ColumnAnalysis(
ValueType.STRING.toString(), ValueType.STRING.toString(),
false, false,
(mmap1 ? 10881 : 10764) + (mmap2 ? 10881 : 10764), size1 + size2,
1, 1,
"preferred", "preferred",
"preferred", "preferred",
@ -485,10 +517,16 @@ public class SegmentMetadataQueryTest
@Test @Test
public void testSegmentMetadataQueryWithDefaultAnalysisMerge2() public void testSegmentMetadataQueryWithDefaultAnalysisMerge2()
{ {
int size1 = 0;
int size2 = 0;
if (bitmaps) {
size1 = mmap1 ? 6882 : 6808;
size2 = mmap2 ? 6882 : 6808;
}
ColumnAnalysis analysis = new ColumnAnalysis( ColumnAnalysis analysis = new ColumnAnalysis(
ValueType.STRING.toString(), ValueType.STRING.toString(),
false, false,
(mmap1 ? 6882 : 6808) + (mmap2 ? 6882 : 6808), size1 + size2,
3, 3,
"spot", "spot",
"upfront", "upfront",
@ -500,10 +538,16 @@ public class SegmentMetadataQueryTest
@Test @Test
public void testSegmentMetadataQueryWithDefaultAnalysisMerge3() public void testSegmentMetadataQueryWithDefaultAnalysisMerge3()
{ {
int size1 = 0;
int size2 = 0;
if (bitmaps) {
size1 = mmap1 ? 9765 : 9660;
size2 = mmap2 ? 9765 : 9660;
}
ColumnAnalysis analysis = new ColumnAnalysis( ColumnAnalysis analysis = new ColumnAnalysis(
ValueType.STRING.toString(), ValueType.STRING.toString(),
false, false,
(mmap1 ? 9765 : 9660) + (mmap2 ? 9765 : 9660), size1 + size2,
9, 9,
"automotive", "automotive",
"travel", "travel",

View File

@ -20,6 +20,7 @@
package org.apache.druid.segment; package org.apache.druid.segment;
import com.google.common.base.Supplier; import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
import com.google.common.base.Throwables; import com.google.common.base.Throwables;
import com.google.common.io.CharSource; import com.google.common.io.CharSource;
import com.google.common.io.LineProcessor; import com.google.common.io.LineProcessor;
@ -111,12 +112,31 @@ public class TestIndex
new StringDimensionSchema("null_column") new StringDimensionSchema("null_column")
); );
public static final List<DimensionSchema> DIMENSION_SCHEMAS_NO_BITMAP = Arrays.asList(
new StringDimensionSchema("market", null, false),
new StringDimensionSchema("quality", null, false),
new LongDimensionSchema("qualityLong"),
new FloatDimensionSchema("qualityFloat"),
new DoubleDimensionSchema("qualityDouble"),
new StringDimensionSchema("qualityNumericString", null, false),
new StringDimensionSchema("placement", null, false),
new StringDimensionSchema("placementish", null, false),
new StringDimensionSchema("partial_null_column", null, false),
new StringDimensionSchema("null_column", null, false)
);
public static final DimensionsSpec DIMENSIONS_SPEC = new DimensionsSpec( public static final DimensionsSpec DIMENSIONS_SPEC = new DimensionsSpec(
DIMENSION_SCHEMAS, DIMENSION_SCHEMAS,
null, null,
null null
); );
public static final DimensionsSpec DIMENSIONS_SPEC_NO_BITMAPS = new DimensionsSpec(
DIMENSION_SCHEMAS_NO_BITMAP,
null,
null
);
public static final String[] DOUBLE_METRICS = new String[]{"index", "indexMin", "indexMaxPlusTen"}; public static final String[] DOUBLE_METRICS = new String[]{"index", "indexMin", "indexMaxPlusTen"};
public static final String[] FLOAT_METRICS = new String[]{"indexFloat", "indexMinFloat", "indexMaxFloat"}; public static final String[] FLOAT_METRICS = new String[]{"indexFloat", "indexMinFloat", "indexMaxFloat"};
private static final Logger log = new Logger(TestIndex.class); private static final Logger log = new Logger(TestIndex.class);
@ -147,107 +167,95 @@ public class TestIndex
} }
} }
private static IncrementalIndex realtimeIndex = null; private static Supplier<IncrementalIndex> realtimeIndex = Suppliers.memoize(
private static IncrementalIndex noRollupRealtimeIndex = null; () -> makeRealtimeIndex("druid.sample.numeric.tsv")
private static QueryableIndex mmappedIndex = null; );
private static QueryableIndex noRollupMmappedIndex = null; private static Supplier<IncrementalIndex> noRollupRealtimeIndex = Suppliers.memoize(
private static QueryableIndex mergedRealtime = null; () -> makeRealtimeIndex("druid.sample.numeric.tsv", false)
);
private static Supplier<IncrementalIndex> noBitmapRealtimeIndex = Suppliers.memoize(
() -> makeRealtimeIndex("druid.sample.numeric.tsv", false, false)
);
private static Supplier<QueryableIndex> mmappedIndex = Suppliers.memoize(
() -> persistRealtimeAndLoadMMapped(realtimeIndex.get())
);
private static Supplier<QueryableIndex> noRollupMmappedIndex = Suppliers.memoize(
() -> persistRealtimeAndLoadMMapped(noRollupRealtimeIndex.get())
);
private static Supplier<QueryableIndex> noBitmapMmappedIndex = Suppliers.memoize(
() -> persistRealtimeAndLoadMMapped(noBitmapRealtimeIndex.get())
);
private static Supplier<QueryableIndex> mergedRealtime = Suppliers.memoize(() -> {
try {
IncrementalIndex top = makeRealtimeIndex("druid.sample.numeric.tsv.top");
IncrementalIndex bottom = makeRealtimeIndex("druid.sample.numeric.tsv.bottom");
File tmpFile = File.createTempFile("yay", "who");
tmpFile.delete();
File topFile = new File(tmpFile, "top");
File bottomFile = new File(tmpFile, "bottom");
File mergedFile = new File(tmpFile, "merged");
topFile.mkdirs();
topFile.deleteOnExit();
bottomFile.mkdirs();
bottomFile.deleteOnExit();
mergedFile.mkdirs();
mergedFile.deleteOnExit();
INDEX_MERGER.persist(top, DATA_INTERVAL, topFile, indexSpec, null);
INDEX_MERGER.persist(bottom, DATA_INTERVAL, bottomFile, indexSpec, null);
return INDEX_IO.loadIndex(
INDEX_MERGER.mergeQueryableIndex(
Arrays.asList(INDEX_IO.loadIndex(topFile), INDEX_IO.loadIndex(bottomFile)),
true,
METRIC_AGGS,
mergedFile,
indexSpec,
null
)
);
}
catch (IOException e) {
throw Throwables.propagate(e);
}
});
public static IncrementalIndex getIncrementalTestIndex() public static IncrementalIndex getIncrementalTestIndex()
{ {
synchronized (log) { return realtimeIndex.get();
if (realtimeIndex != null) {
return realtimeIndex;
}
}
return realtimeIndex = makeRealtimeIndex("druid.sample.numeric.tsv");
} }
public static IncrementalIndex getNoRollupIncrementalTestIndex() public static IncrementalIndex getNoRollupIncrementalTestIndex()
{ {
synchronized (log) { return noRollupRealtimeIndex.get();
if (noRollupRealtimeIndex != null) { }
return noRollupRealtimeIndex;
}
}
return noRollupRealtimeIndex = makeRealtimeIndex("druid.sample.numeric.tsv", false); public static IncrementalIndex getNoBitmapIncrementalTestIndex()
{
return noBitmapRealtimeIndex.get();
} }
public static QueryableIndex getMMappedTestIndex() public static QueryableIndex getMMappedTestIndex()
{ {
synchronized (log) { return mmappedIndex.get();
if (mmappedIndex != null) {
return mmappedIndex;
}
}
IncrementalIndex incrementalIndex = getIncrementalTestIndex();
mmappedIndex = persistRealtimeAndLoadMMapped(incrementalIndex);
return mmappedIndex;
} }
public static QueryableIndex getNoRollupMMappedTestIndex() public static QueryableIndex getNoRollupMMappedTestIndex()
{ {
synchronized (log) { return noRollupMmappedIndex.get();
if (noRollupMmappedIndex != null) { }
return noRollupMmappedIndex;
}
}
IncrementalIndex incrementalIndex = getNoRollupIncrementalTestIndex(); public static QueryableIndex getNoBitmapMMappedTestIndex()
noRollupMmappedIndex = persistRealtimeAndLoadMMapped(incrementalIndex); {
return noBitmapMmappedIndex.get();
return noRollupMmappedIndex;
} }
public static QueryableIndex mergedRealtimeIndex() public static QueryableIndex mergedRealtimeIndex()
{ {
synchronized (log) { return mergedRealtime.get();
if (mergedRealtime != null) {
return mergedRealtime;
}
try {
IncrementalIndex top = makeRealtimeIndex("druid.sample.numeric.tsv.top");
IncrementalIndex bottom = makeRealtimeIndex("druid.sample.numeric.tsv.bottom");
File tmpFile = File.createTempFile("yay", "who");
tmpFile.delete();
File topFile = new File(tmpFile, "top");
File bottomFile = new File(tmpFile, "bottom");
File mergedFile = new File(tmpFile, "merged");
topFile.mkdirs();
topFile.deleteOnExit();
bottomFile.mkdirs();
bottomFile.deleteOnExit();
mergedFile.mkdirs();
mergedFile.deleteOnExit();
INDEX_MERGER.persist(top, DATA_INTERVAL, topFile, indexSpec, null);
INDEX_MERGER.persist(bottom, DATA_INTERVAL, bottomFile, indexSpec, null);
mergedRealtime = INDEX_IO.loadIndex(
INDEX_MERGER.mergeQueryableIndex(
Arrays.asList(INDEX_IO.loadIndex(topFile), INDEX_IO.loadIndex(bottomFile)),
true,
METRIC_AGGS,
mergedFile,
indexSpec,
null
)
);
return mergedRealtime;
}
catch (IOException e) {
throw Throwables.propagate(e);
}
}
} }
public static IncrementalIndex makeRealtimeIndex(final String resourceFilename) public static IncrementalIndex makeRealtimeIndex(final String resourceFilename)
@ -256,6 +264,11 @@ public class TestIndex
} }
public static IncrementalIndex makeRealtimeIndex(final String resourceFilename, boolean rollup) public static IncrementalIndex makeRealtimeIndex(final String resourceFilename, boolean rollup)
{
return makeRealtimeIndex(resourceFilename, rollup, true);
}
public static IncrementalIndex makeRealtimeIndex(final String resourceFilename, boolean rollup, boolean bitmap)
{ {
final URL resource = TestIndex.class.getClassLoader().getResource(resourceFilename); final URL resource = TestIndex.class.getClassLoader().getResource(resourceFilename);
if (resource == null) { if (resource == null) {
@ -263,20 +276,20 @@ public class TestIndex
} }
log.info("Realtime loading index file[%s]", resource); log.info("Realtime loading index file[%s]", resource);
CharSource stream = Resources.asByteSource(resource).asCharSource(StandardCharsets.UTF_8); CharSource stream = Resources.asByteSource(resource).asCharSource(StandardCharsets.UTF_8);
return makeRealtimeIndex(stream, rollup); return makeRealtimeIndex(stream, rollup, bitmap);
} }
public static IncrementalIndex makeRealtimeIndex(final CharSource source) public static IncrementalIndex makeRealtimeIndex(final CharSource source)
{ {
return makeRealtimeIndex(source, true); return makeRealtimeIndex(source, true, true);
} }
public static IncrementalIndex makeRealtimeIndex(final CharSource source, boolean rollup) public static IncrementalIndex makeRealtimeIndex(final CharSource source, boolean rollup, boolean bitmap)
{ {
final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder() final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder()
.withMinTimestamp(DateTimes.of("2011-01-12T00:00:00.000Z").getMillis()) .withMinTimestamp(DateTimes.of("2011-01-12T00:00:00.000Z").getMillis())
.withTimestampSpec(new TimestampSpec("ds", "auto", null)) .withTimestampSpec(new TimestampSpec("ds", "auto", null))
.withDimensionsSpec(DIMENSIONS_SPEC) .withDimensionsSpec(bitmap ? DIMENSIONS_SPEC : DIMENSIONS_SPEC_NO_BITMAPS)
.withVirtualColumns(VIRTUAL_COLUMNS) .withVirtualColumns(VIRTUAL_COLUMNS)
.withMetrics(METRIC_AGGS) .withMetrics(METRIC_AGGS)
.withRollup(rollup) .withRollup(rollup)

View File

@ -117,22 +117,22 @@ public class ExpressionFilterTest extends BaseFilterTest
@Test @Test
public void testOneSingleValuedStringColumn() public void testOneSingleValuedStringColumn()
{ {
assertFilterMatches(EDF("dim3 == ''"), ImmutableList.of("0")); assertFilterMatches(edf("dim3 == ''"), ImmutableList.of("0"));
assertFilterMatches(EDF("dim3 == '1'"), ImmutableList.of("3", "4", "6")); assertFilterMatches(edf("dim3 == '1'"), ImmutableList.of("3", "4", "6"));
assertFilterMatches(EDF("dim3 == 'a'"), ImmutableList.of("7")); assertFilterMatches(edf("dim3 == 'a'"), ImmutableList.of("7"));
assertFilterMatches(EDF("dim3 == 1"), ImmutableList.of("3", "4", "6")); assertFilterMatches(edf("dim3 == 1"), ImmutableList.of("3", "4", "6"));
assertFilterMatches(EDF("dim3 == 1.0"), ImmutableList.of("3", "4", "6")); assertFilterMatches(edf("dim3 == 1.0"), ImmutableList.of("3", "4", "6"));
assertFilterMatches(EDF("dim3 == 1.234"), ImmutableList.of("9")); assertFilterMatches(edf("dim3 == 1.234"), ImmutableList.of("9"));
assertFilterMatches(EDF("dim3 < '2'"), ImmutableList.of("0", "1", "3", "4", "6", "9")); assertFilterMatches(edf("dim3 < '2'"), ImmutableList.of("0", "1", "3", "4", "6", "9"));
if (NullHandling.replaceWithDefault()) { if (NullHandling.replaceWithDefault()) {
assertFilterMatches(EDF("dim3 < 2"), ImmutableList.of("0", "3", "4", "6", "7", "9")); assertFilterMatches(edf("dim3 < 2"), ImmutableList.of("0", "3", "4", "6", "7", "9"));
assertFilterMatches(EDF("dim3 < 2.0"), ImmutableList.of("0", "3", "4", "6", "7", "9")); assertFilterMatches(edf("dim3 < 2.0"), ImmutableList.of("0", "3", "4", "6", "7", "9"));
} else { } else {
// Empty String and "a" will not match // Empty String and "a" will not match
assertFilterMatches(EDF("dim3 < 2"), ImmutableList.of("3", "4", "6", "9")); assertFilterMatches(edf("dim3 < 2"), ImmutableList.of("3", "4", "6", "9"));
assertFilterMatches(EDF("dim3 < 2.0"), ImmutableList.of("3", "4", "6", "9")); assertFilterMatches(edf("dim3 < 2.0"), ImmutableList.of("3", "4", "6", "9"));
} }
assertFilterMatches(EDF("like(dim3, '1%')"), ImmutableList.of("1", "3", "4", "6", "9")); assertFilterMatches(edf("like(dim3, '1%')"), ImmutableList.of("1", "3", "4", "6", "9"));
} }
@Test @Test
@ -141,124 +141,124 @@ public class ExpressionFilterTest extends BaseFilterTest
// Expressions currently treat multi-valued arrays as nulls. // Expressions currently treat multi-valued arrays as nulls.
// This test is just documenting the current behavior, not necessarily saying it makes sense. // This test is just documenting the current behavior, not necessarily saying it makes sense.
if (NullHandling.replaceWithDefault()) { if (NullHandling.replaceWithDefault()) {
assertFilterMatches(EDF("dim4 == ''"), ImmutableList.of("0", "1", "2", "4", "5", "6", "7", "8")); assertFilterMatches(edf("dim4 == ''"), ImmutableList.of("0", "1", "2", "4", "5", "6", "7", "8"));
} else { } else {
assertFilterMatches(EDF("dim4 == ''"), ImmutableList.of("2")); assertFilterMatches(edf("dim4 == ''"), ImmutableList.of("2"));
// AS per SQL standard null == null returns false. // AS per SQL standard null == null returns false.
assertFilterMatches(EDF("dim4 == null"), ImmutableList.of()); assertFilterMatches(edf("dim4 == null"), ImmutableList.of());
} }
assertFilterMatches(EDF("dim4 == '1'"), ImmutableList.of()); assertFilterMatches(edf("dim4 == '1'"), ImmutableList.of());
assertFilterMatches(EDF("dim4 == '3'"), ImmutableList.of("3")); assertFilterMatches(edf("dim4 == '3'"), ImmutableList.of("3"));
} }
@Test @Test
public void testOneLongColumn() public void testOneLongColumn()
{ {
if (NullHandling.replaceWithDefault()) { if (NullHandling.replaceWithDefault()) {
assertFilterMatches(EDF("dim1 == ''"), ImmutableList.of("0")); assertFilterMatches(edf("dim1 == ''"), ImmutableList.of("0"));
} else { } else {
// A long does not match empty string // A long does not match empty string
assertFilterMatches(EDF("dim1 == ''"), ImmutableList.of()); assertFilterMatches(edf("dim1 == ''"), ImmutableList.of());
} }
assertFilterMatches(EDF("dim1 == '1'"), ImmutableList.of("1")); assertFilterMatches(edf("dim1 == '1'"), ImmutableList.of("1"));
assertFilterMatches(EDF("dim1 == 2"), ImmutableList.of("2")); assertFilterMatches(edf("dim1 == 2"), ImmutableList.of("2"));
assertFilterMatches(EDF("dim1 < '2'"), ImmutableList.of("0", "1")); assertFilterMatches(edf("dim1 < '2'"), ImmutableList.of("0", "1"));
assertFilterMatches(EDF("dim1 < 2"), ImmutableList.of("0", "1")); assertFilterMatches(edf("dim1 < 2"), ImmutableList.of("0", "1"));
assertFilterMatches(EDF("dim1 < 2.0"), ImmutableList.of("0", "1")); assertFilterMatches(edf("dim1 < 2.0"), ImmutableList.of("0", "1"));
assertFilterMatches(EDF("like(dim1, '1%')"), ImmutableList.of("1")); assertFilterMatches(edf("like(dim1, '1%')"), ImmutableList.of("1"));
} }
@Test @Test
public void testOneFloatColumn() public void testOneFloatColumn()
{ {
if (NullHandling.replaceWithDefault()) { if (NullHandling.replaceWithDefault()) {
assertFilterMatches(EDF("dim2 == ''"), ImmutableList.of("0")); assertFilterMatches(edf("dim2 == ''"), ImmutableList.of("0"));
} else { } else {
// A float does not match empty string // A float does not match empty string
assertFilterMatches(EDF("dim2 == ''"), ImmutableList.of()); assertFilterMatches(edf("dim2 == ''"), ImmutableList.of());
} }
assertFilterMatches(EDF("dim2 == '1'"), ImmutableList.of("1")); assertFilterMatches(edf("dim2 == '1'"), ImmutableList.of("1"));
assertFilterMatches(EDF("dim2 == 2"), ImmutableList.of("2")); assertFilterMatches(edf("dim2 == 2"), ImmutableList.of("2"));
assertFilterMatches(EDF("dim2 < '2'"), ImmutableList.of("0", "1")); assertFilterMatches(edf("dim2 < '2'"), ImmutableList.of("0", "1"));
assertFilterMatches(EDF("dim2 < 2"), ImmutableList.of("0", "1")); assertFilterMatches(edf("dim2 < 2"), ImmutableList.of("0", "1"));
assertFilterMatches(EDF("dim2 < 2.0"), ImmutableList.of("0", "1")); assertFilterMatches(edf("dim2 < 2.0"), ImmutableList.of("0", "1"));
assertFilterMatches(EDF("like(dim2, '1%')"), ImmutableList.of("1")); assertFilterMatches(edf("like(dim2, '1%')"), ImmutableList.of("1"));
} }
@Test @Test
public void testConstantExpression() public void testConstantExpression()
{ {
assertFilterMatches(EDF("1 + 1"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")); assertFilterMatches(edf("1 + 1"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
assertFilterMatches(EDF("0 + 0"), ImmutableList.of()); assertFilterMatches(edf("0 + 0"), ImmutableList.of());
} }
@Test @Test
public void testCompareColumns() public void testCompareColumns()
{ {
// String vs string // String vs string
assertFilterMatches(EDF("dim0 == dim3"), ImmutableList.of("2", "5", "8")); assertFilterMatches(edf("dim0 == dim3"), ImmutableList.of("2", "5", "8"));
if (NullHandling.replaceWithDefault()) { if (NullHandling.replaceWithDefault()) {
// String vs long // String vs long
assertFilterMatches(EDF("dim1 == dim3"), ImmutableList.of("0", "2", "5", "8")); assertFilterMatches(edf("dim1 == dim3"), ImmutableList.of("0", "2", "5", "8"));
// String vs float // String vs float
assertFilterMatches(EDF("dim2 == dim3"), ImmutableList.of("0", "2", "5", "8")); assertFilterMatches(edf("dim2 == dim3"), ImmutableList.of("0", "2", "5", "8"));
} else { } else {
// String vs long // String vs long
assertFilterMatches(EDF("dim1 == dim3"), ImmutableList.of("2", "5", "8")); assertFilterMatches(edf("dim1 == dim3"), ImmutableList.of("2", "5", "8"));
// String vs float // String vs float
assertFilterMatches(EDF("dim2 == dim3"), ImmutableList.of("2", "5", "8")); assertFilterMatches(edf("dim2 == dim3"), ImmutableList.of("2", "5", "8"));
} }
// String vs. multi-value string // String vs. multi-value string
// Expressions currently treat multi-valued arrays as nulls. // Expressions currently treat multi-valued arrays as nulls.
// This test is just documenting the current behavior, not necessarily saying it makes sense. // This test is just documenting the current behavior, not necessarily saying it makes sense.
assertFilterMatches(EDF("dim0 == dim4"), ImmutableList.of("3")); assertFilterMatches(edf("dim0 == dim4"), ImmutableList.of("3"));
} }
@Test @Test
public void testMissingColumn() public void testMissingColumn()
{ {
if (NullHandling.replaceWithDefault()) { if (NullHandling.replaceWithDefault()) {
assertFilterMatches(EDF("missing == ''"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")); assertFilterMatches(edf("missing == ''"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
} else { } else {
// AS per SQL standard null == null returns false. // AS per SQL standard null == null returns false.
assertFilterMatches(EDF("missing == null"), ImmutableList.of()); assertFilterMatches(edf("missing == null"), ImmutableList.of());
} }
assertFilterMatches(EDF("missing == '1'"), ImmutableList.of()); assertFilterMatches(edf("missing == '1'"), ImmutableList.of());
assertFilterMatches(EDF("missing == 2"), ImmutableList.of()); assertFilterMatches(edf("missing == 2"), ImmutableList.of());
if (NullHandling.replaceWithDefault()) { if (NullHandling.replaceWithDefault()) {
// missing equivaluent to 0 // missing equivaluent to 0
assertFilterMatches(EDF("missing < '2'"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")); assertFilterMatches(edf("missing < '2'"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
assertFilterMatches(EDF("missing < 2"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")); assertFilterMatches(edf("missing < 2"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
assertFilterMatches(EDF("missing < 2.0"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")); assertFilterMatches(edf("missing < 2.0"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
} else { } else {
// missing equivalent to null // missing equivalent to null
assertFilterMatches(EDF("missing < '2'"), ImmutableList.of()); assertFilterMatches(edf("missing < '2'"), ImmutableList.of());
assertFilterMatches(EDF("missing < 2"), ImmutableList.of()); assertFilterMatches(edf("missing < 2"), ImmutableList.of());
assertFilterMatches(EDF("missing < 2.0"), ImmutableList.of()); assertFilterMatches(edf("missing < 2.0"), ImmutableList.of());
} }
assertFilterMatches(EDF("missing > '2'"), ImmutableList.of()); assertFilterMatches(edf("missing > '2'"), ImmutableList.of());
assertFilterMatches(EDF("missing > 2"), ImmutableList.of()); assertFilterMatches(edf("missing > 2"), ImmutableList.of());
assertFilterMatches(EDF("missing > 2.0"), ImmutableList.of()); assertFilterMatches(edf("missing > 2.0"), ImmutableList.of());
assertFilterMatches(EDF("like(missing, '1%')"), ImmutableList.of()); assertFilterMatches(edf("like(missing, '1%')"), ImmutableList.of());
} }
@Test @Test
public void testGetRequiredColumn() public void testGetRequiredColumn()
{ {
Assert.assertEquals(EDF("like(dim1, '1%')").getRequiredColumns(), Sets.newHashSet("dim1")); Assert.assertEquals(edf("like(dim1, '1%')").getRequiredColumns(), Sets.newHashSet("dim1"));
Assert.assertEquals(EDF("dim2 == '1'").getRequiredColumns(), Sets.newHashSet("dim2")); Assert.assertEquals(edf("dim2 == '1'").getRequiredColumns(), Sets.newHashSet("dim2"));
Assert.assertEquals(EDF("dim3 < '2'").getRequiredColumns(), Sets.newHashSet("dim3")); Assert.assertEquals(edf("dim3 < '2'").getRequiredColumns(), Sets.newHashSet("dim3"));
Assert.assertEquals(EDF("dim4 == ''").getRequiredColumns(), Sets.newHashSet("dim4")); Assert.assertEquals(edf("dim4 == ''").getRequiredColumns(), Sets.newHashSet("dim4"));
Assert.assertEquals(EDF("1 + 1").getRequiredColumns(), new HashSet<>()); Assert.assertEquals(edf("1 + 1").getRequiredColumns(), new HashSet<>());
Assert.assertEquals(EDF("dim0 == dim3").getRequiredColumns(), Sets.newHashSet("dim0", "dim3")); Assert.assertEquals(edf("dim0 == dim3").getRequiredColumns(), Sets.newHashSet("dim0", "dim3"));
Assert.assertEquals(EDF("missing == ''").getRequiredColumns(), Sets.newHashSet("missing")); Assert.assertEquals(edf("missing == ''").getRequiredColumns(), Sets.newHashSet("missing"));
} }
private static ExpressionDimFilter EDF(final String expression) private static ExpressionDimFilter edf(final String expression)
{ {
return new ExpressionDimFilter(expression, TestExprMacroTable.INSTANCE); return new ExpressionDimFilter(expression, TestExprMacroTable.INSTANCE);
} }

View File

@ -28,11 +28,13 @@ import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.http.client.response.FullResponseHolder; import org.apache.druid.java.util.http.client.response.FullResponseHolder;
import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.timeline.DataSegment;
import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpMethod;
import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.jboss.netty.handler.codec.http.HttpResponseStatus;
import org.joda.time.Interval; import org.joda.time.Interval;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import javax.ws.rs.core.MediaType;
import java.util.List; import java.util.List;
public class CoordinatorClient public class CoordinatorClient
@ -95,13 +97,15 @@ public class CoordinatorClient
{ {
try { try {
FullResponseHolder response = druidLeaderClient.go( FullResponseHolder response = druidLeaderClient.go(
druidLeaderClient.makeRequest(HttpMethod.GET, druidLeaderClient.makeRequest(
StringUtils.format( HttpMethod.GET,
"/druid/coordinator/v1/datasources/%s/intervals/%s/serverview?partial=%s", StringUtils.format(
dataSource, "/druid/coordinator/v1/datasources/%s/intervals/%s/serverview?partial=%s",
interval.toString().replace('/', '_'), StringUtils.urlEncode(dataSource),
incompleteOk interval.toString().replace('/', '_'),
)) incompleteOk
)
)
); );
if (!response.getStatus().equals(HttpResponseStatus.OK)) { if (!response.getStatus().equals(HttpResponseStatus.OK)) {
@ -121,4 +125,35 @@ public class CoordinatorClient
throw new RuntimeException(e); throw new RuntimeException(e);
} }
} }
public List<DataSegment> getDatabaseSegmentDataSourceSegments(String dataSource, List<Interval> intervals)
{
try {
FullResponseHolder response = druidLeaderClient.go(
druidLeaderClient.makeRequest(
HttpMethod.POST,
StringUtils.format(
"/druid/coordinator/v1/metadata/datasources/%s/segments?full",
StringUtils.urlEncode(dataSource)
)
).setContent(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(intervals))
);
if (!response.getStatus().equals(HttpResponseStatus.OK)) {
throw new ISE(
"Error while fetching database segment data source segments status[%s] content[%s]",
response.getStatus(),
response.getContent()
);
}
return jsonMapper.readValue(
response.getContent(), new TypeReference<List<DataSegment>>()
{
}
);
}
catch (Exception e) {
throw new RuntimeException(e);
}
}
} }

View File

@ -58,6 +58,9 @@ public class SegmentLoaderLocalCacheManager implements SegmentLoader
} }
}; };
// Note that we only create this via injection in historical and realtime nodes. Peons create these
// objects via SegmentLoaderFactory objects, so that they can store segments in task-specific
// directories rather than statically configured directories.
@Inject @Inject
public SegmentLoaderLocalCacheManager( public SegmentLoaderLocalCacheManager(
IndexIO indexIO, IndexIO indexIO,
@ -79,11 +82,6 @@ public class SegmentLoaderLocalCacheManager implements SegmentLoader
} }
} }
public SegmentLoaderLocalCacheManager withConfig(SegmentLoaderConfig config)
{
return new SegmentLoaderLocalCacheManager(indexIO, config, jsonMapper);
}
@Override @Override
public boolean isSegmentLoaded(final DataSegment segment) public boolean isSegmentLoaded(final DataSegment segment)
{ {

View File

@ -100,8 +100,8 @@ public class AppenderatorPlumberTest
// getDataSource // getDataSource
Assert.assertEquals(AppenderatorTester.DATASOURCE, appenderator.getDataSource()); Assert.assertEquals(AppenderatorTester.DATASOURCE, appenderator.getDataSource());
InputRow[] rows = new InputRow[] {AppenderatorTest.IR("2000", "foo", 1), InputRow[] rows = new InputRow[] {AppenderatorTest.ir("2000", "foo", 1),
AppenderatorTest.IR("2000", "bar", 2), AppenderatorTest.IR("2000", "qux", 4)}; AppenderatorTest.ir("2000", "bar", 2), AppenderatorTest.ir("2000", "qux", 4)};
// add // add
Assert.assertEquals(1, plumber.add(rows[0], null).getRowCount()); Assert.assertEquals(1, plumber.add(rows[0], null).getRowCount());

View File

@ -58,9 +58,9 @@ import java.util.concurrent.atomic.AtomicInteger;
public class AppenderatorTest public class AppenderatorTest
{ {
private static final List<SegmentIdWithShardSpec> IDENTIFIERS = ImmutableList.of( private static final List<SegmentIdWithShardSpec> IDENTIFIERS = ImmutableList.of(
SI("2000/2001", "A", 0), si("2000/2001", "A", 0),
SI("2000/2001", "A", 1), si("2000/2001", "A", 1),
SI("2001/2002", "A", 0) si("2001/2002", "A", 0)
); );
@Test @Test
@ -83,21 +83,21 @@ public class AppenderatorTest
commitMetadata.put("x", "1"); commitMetadata.put("x", "1");
Assert.assertEquals( Assert.assertEquals(
1, 1,
appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier) appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier)
.getNumRowsInSegment() .getNumRowsInSegment()
); );
commitMetadata.put("x", "2"); commitMetadata.put("x", "2");
Assert.assertEquals( Assert.assertEquals(
2, 2,
appenderator.add(IDENTIFIERS.get(0), IR("2000", "bar", 2), committerSupplier) appenderator.add(IDENTIFIERS.get(0), ir("2000", "bar", 2), committerSupplier)
.getNumRowsInSegment() .getNumRowsInSegment()
); );
commitMetadata.put("x", "3"); commitMetadata.put("x", "3");
Assert.assertEquals( Assert.assertEquals(
1, 1,
appenderator.add(IDENTIFIERS.get(1), IR("2000", "qux", 4), committerSupplier) appenderator.add(IDENTIFIERS.get(1), ir("2000", "qux", 4), committerSupplier)
.getNumRowsInSegment() .getNumRowsInSegment()
); );
@ -173,14 +173,14 @@ public class AppenderatorTest
}; };
appenderator.startJob(); appenderator.startJob();
appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier); appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier);
//expectedSizeInBytes = 44(map overhead) + 28 (TimeAndDims overhead) + 56 (aggregator metrics) + 10 (dimsKeySize) = 138 + 1 byte when null handling is enabled //expectedSizeInBytes = 44(map overhead) + 28 (TimeAndDims overhead) + 56 (aggregator metrics) + 10 (dimsKeySize) = 138 + 1 byte when null handling is enabled
int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0; int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0;
Assert.assertEquals( Assert.assertEquals(
138 + nullHandlingOverhead, 138 + nullHandlingOverhead,
((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(0))
); );
appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier);
Assert.assertEquals( Assert.assertEquals(
138 + nullHandlingOverhead, 138 + nullHandlingOverhead,
((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(1))
@ -216,11 +216,11 @@ public class AppenderatorTest
}; };
appenderator.startJob(); appenderator.startJob();
appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier); appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier);
//expectedSizeInBytes = 44(map overhead) + 28 (TimeAndDims overhead) + 56 (aggregator metrics) + 10 (dimsKeySize) = 138 //expectedSizeInBytes = 44(map overhead) + 28 (TimeAndDims overhead) + 56 (aggregator metrics) + 10 (dimsKeySize) = 138
int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0; int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0;
Assert.assertEquals(138 + nullHandlingOverhead, ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory()); Assert.assertEquals(138 + nullHandlingOverhead, ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory());
appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier);
Assert.assertEquals( Assert.assertEquals(
276 + 2 * nullHandlingOverhead, 276 + 2 * nullHandlingOverhead,
((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory() ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory()
@ -258,7 +258,7 @@ public class AppenderatorTest
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.startJob(); appenderator.startJob();
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier); appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier);
//we still calculate the size even when ignoring it to make persist decision //we still calculate the size even when ignoring it to make persist decision
int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0; int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0;
Assert.assertEquals( Assert.assertEquals(
@ -266,7 +266,7 @@ public class AppenderatorTest
((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(0))
); );
Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier);
Assert.assertEquals( Assert.assertEquals(
276 + 2 * nullHandlingOverhead, 276 + 2 * nullHandlingOverhead,
((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory() ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory()
@ -310,17 +310,17 @@ public class AppenderatorTest
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.startJob(); appenderator.startJob();
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier); appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier);
Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier);
Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier);
Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.add(IDENTIFIERS.get(0), IR("2000", "baz", 1), committerSupplier); appenderator.add(IDENTIFIERS.get(0), ir("2000", "baz", 1), committerSupplier);
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.add(IDENTIFIERS.get(1), IR("2000", "qux", 1), committerSupplier); appenderator.add(IDENTIFIERS.get(1), ir("2000", "qux", 1), committerSupplier);
Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.add(IDENTIFIERS.get(0), IR("2000", "bob", 1), committerSupplier); appenderator.add(IDENTIFIERS.get(0), ir("2000", "bob", 1), committerSupplier);
Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.persistAll(committerSupplier.get()); appenderator.persistAll(committerSupplier.get());
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
@ -356,17 +356,17 @@ public class AppenderatorTest
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.startJob(); appenderator.startJob();
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier, false); appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier, false);
Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier, false); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier, false);
Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier, false); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier, false);
Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.add(IDENTIFIERS.get(0), IR("2000", "baz", 1), committerSupplier, false); appenderator.add(IDENTIFIERS.get(0), ir("2000", "baz", 1), committerSupplier, false);
Assert.assertEquals(3, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(3, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.add(IDENTIFIERS.get(1), IR("2000", "qux", 1), committerSupplier, false); appenderator.add(IDENTIFIERS.get(1), ir("2000", "qux", 1), committerSupplier, false);
Assert.assertEquals(4, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(4, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.add(IDENTIFIERS.get(0), IR("2000", "bob", 1), committerSupplier, false); appenderator.add(IDENTIFIERS.get(0), ir("2000", "bob", 1), committerSupplier, false);
Assert.assertEquals(5, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(5, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.persistAll(committerSupplier.get()); appenderator.persistAll(committerSupplier.get());
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
@ -409,15 +409,15 @@ public class AppenderatorTest
appenderator.startJob(); appenderator.startJob();
eventCount.incrementAndGet(); eventCount.incrementAndGet();
appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier); appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier);
eventCount.incrementAndGet(); eventCount.incrementAndGet();
appenderator.add(IDENTIFIERS.get(0), IR("2000", "bar", 2), committerSupplier); appenderator.add(IDENTIFIERS.get(0), ir("2000", "bar", 2), committerSupplier);
eventCount.incrementAndGet(); eventCount.incrementAndGet();
appenderator.add(IDENTIFIERS.get(0), IR("2000", "baz", 3), committerSupplier); appenderator.add(IDENTIFIERS.get(0), ir("2000", "baz", 3), committerSupplier);
eventCount.incrementAndGet(); eventCount.incrementAndGet();
appenderator.add(IDENTIFIERS.get(0), IR("2000", "qux", 4), committerSupplier); appenderator.add(IDENTIFIERS.get(0), ir("2000", "qux", 4), committerSupplier);
eventCount.incrementAndGet(); eventCount.incrementAndGet();
appenderator.add(IDENTIFIERS.get(0), IR("2000", "bob", 5), committerSupplier); appenderator.add(IDENTIFIERS.get(0), ir("2000", "bob", 5), committerSupplier);
appenderator.close(); appenderator.close();
try (final AppenderatorTester tester2 = new AppenderatorTester( try (final AppenderatorTester tester2 = new AppenderatorTester(
@ -445,9 +445,9 @@ public class AppenderatorTest
Assert.assertEquals(0, appenderator.getTotalRowCount()); Assert.assertEquals(0, appenderator.getTotalRowCount());
appenderator.startJob(); appenderator.startJob();
Assert.assertEquals(0, appenderator.getTotalRowCount()); Assert.assertEquals(0, appenderator.getTotalRowCount());
appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier); appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier);
Assert.assertEquals(1, appenderator.getTotalRowCount()); Assert.assertEquals(1, appenderator.getTotalRowCount());
appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier);
Assert.assertEquals(2, appenderator.getTotalRowCount()); Assert.assertEquals(2, appenderator.getTotalRowCount());
appenderator.persistAll(committerSupplier.get()).get(); appenderator.persistAll(committerSupplier.get()).get();
@ -457,13 +457,13 @@ public class AppenderatorTest
appenderator.drop(IDENTIFIERS.get(1)).get(); appenderator.drop(IDENTIFIERS.get(1)).get();
Assert.assertEquals(0, appenderator.getTotalRowCount()); Assert.assertEquals(0, appenderator.getTotalRowCount());
appenderator.add(IDENTIFIERS.get(2), IR("2001", "bar", 1), committerSupplier); appenderator.add(IDENTIFIERS.get(2), ir("2001", "bar", 1), committerSupplier);
Assert.assertEquals(1, appenderator.getTotalRowCount()); Assert.assertEquals(1, appenderator.getTotalRowCount());
appenderator.add(IDENTIFIERS.get(2), IR("2001", "baz", 1), committerSupplier); appenderator.add(IDENTIFIERS.get(2), ir("2001", "baz", 1), committerSupplier);
Assert.assertEquals(2, appenderator.getTotalRowCount()); Assert.assertEquals(2, appenderator.getTotalRowCount());
appenderator.add(IDENTIFIERS.get(2), IR("2001", "qux", 1), committerSupplier); appenderator.add(IDENTIFIERS.get(2), ir("2001", "qux", 1), committerSupplier);
Assert.assertEquals(3, appenderator.getTotalRowCount()); Assert.assertEquals(3, appenderator.getTotalRowCount());
appenderator.add(IDENTIFIERS.get(2), IR("2001", "bob", 1), committerSupplier); appenderator.add(IDENTIFIERS.get(2), ir("2001", "bob", 1), committerSupplier);
Assert.assertEquals(4, appenderator.getTotalRowCount()); Assert.assertEquals(4, appenderator.getTotalRowCount());
appenderator.persistAll(committerSupplier.get()).get(); appenderator.persistAll(committerSupplier.get()).get();
@ -483,13 +483,13 @@ public class AppenderatorTest
final Appenderator appenderator = tester.getAppenderator(); final Appenderator appenderator = tester.getAppenderator();
appenderator.startJob(); appenderator.startJob();
appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), Suppliers.ofInstance(Committers.nil())); appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), Suppliers.ofInstance(Committers.nil()));
appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 2), Suppliers.ofInstance(Committers.nil())); appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 2), Suppliers.ofInstance(Committers.nil()));
appenderator.add(IDENTIFIERS.get(1), IR("2000", "foo", 4), Suppliers.ofInstance(Committers.nil())); appenderator.add(IDENTIFIERS.get(1), ir("2000", "foo", 4), Suppliers.ofInstance(Committers.nil()));
appenderator.add(IDENTIFIERS.get(2), IR("2001", "foo", 8), Suppliers.ofInstance(Committers.nil())); appenderator.add(IDENTIFIERS.get(2), ir("2001", "foo", 8), Suppliers.ofInstance(Committers.nil()));
appenderator.add(IDENTIFIERS.get(2), IR("2001T01", "foo", 16), Suppliers.ofInstance(Committers.nil())); appenderator.add(IDENTIFIERS.get(2), ir("2001T01", "foo", 16), Suppliers.ofInstance(Committers.nil()));
appenderator.add(IDENTIFIERS.get(2), IR("2001T02", "foo", 32), Suppliers.ofInstance(Committers.nil())); appenderator.add(IDENTIFIERS.get(2), ir("2001T02", "foo", 32), Suppliers.ofInstance(Committers.nil()));
appenderator.add(IDENTIFIERS.get(2), IR("2001T03", "foo", 64), Suppliers.ofInstance(Committers.nil())); appenderator.add(IDENTIFIERS.get(2), ir("2001T03", "foo", 64), Suppliers.ofInstance(Committers.nil()));
// Query1: 2000/2001 // Query1: 2000/2001
final TimeseriesQuery query1 = Druids.newTimeseriesQueryBuilder() final TimeseriesQuery query1 = Druids.newTimeseriesQueryBuilder()
@ -619,13 +619,13 @@ public class AppenderatorTest
final Appenderator appenderator = tester.getAppenderator(); final Appenderator appenderator = tester.getAppenderator();
appenderator.startJob(); appenderator.startJob();
appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), Suppliers.ofInstance(Committers.nil())); appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), Suppliers.ofInstance(Committers.nil()));
appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 2), Suppliers.ofInstance(Committers.nil())); appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 2), Suppliers.ofInstance(Committers.nil()));
appenderator.add(IDENTIFIERS.get(1), IR("2000", "foo", 4), Suppliers.ofInstance(Committers.nil())); appenderator.add(IDENTIFIERS.get(1), ir("2000", "foo", 4), Suppliers.ofInstance(Committers.nil()));
appenderator.add(IDENTIFIERS.get(2), IR("2001", "foo", 8), Suppliers.ofInstance(Committers.nil())); appenderator.add(IDENTIFIERS.get(2), ir("2001", "foo", 8), Suppliers.ofInstance(Committers.nil()));
appenderator.add(IDENTIFIERS.get(2), IR("2001T01", "foo", 16), Suppliers.ofInstance(Committers.nil())); appenderator.add(IDENTIFIERS.get(2), ir("2001T01", "foo", 16), Suppliers.ofInstance(Committers.nil()));
appenderator.add(IDENTIFIERS.get(2), IR("2001T02", "foo", 32), Suppliers.ofInstance(Committers.nil())); appenderator.add(IDENTIFIERS.get(2), ir("2001T02", "foo", 32), Suppliers.ofInstance(Committers.nil()));
appenderator.add(IDENTIFIERS.get(2), IR("2001T03", "foo", 64), Suppliers.ofInstance(Committers.nil())); appenderator.add(IDENTIFIERS.get(2), ir("2001T03", "foo", 64), Suppliers.ofInstance(Committers.nil()));
// Query1: segment #2 // Query1: segment #2
final TimeseriesQuery query1 = Druids.newTimeseriesQueryBuilder() final TimeseriesQuery query1 = Druids.newTimeseriesQueryBuilder()
@ -742,7 +742,7 @@ public class AppenderatorTest
} }
} }
private static SegmentIdWithShardSpec SI(String interval, String version, int partitionNum) private static SegmentIdWithShardSpec si(String interval, String version, int partitionNum)
{ {
return new SegmentIdWithShardSpec( return new SegmentIdWithShardSpec(
AppenderatorTester.DATASOURCE, AppenderatorTester.DATASOURCE,
@ -752,7 +752,7 @@ public class AppenderatorTest
); );
} }
static InputRow IR(String ts, String dim, long met) static InputRow ir(String ts, String dim, long met)
{ {
return new MapBasedInputRow( return new MapBasedInputRow(
DateTimes.of(ts).getMillis(), DateTimes.of(ts).getMillis(),

View File

@ -167,9 +167,9 @@ public class DefaultOfflineAppenderatorFactoryTest
new LinearShardSpec(0) new LinearShardSpec(0)
); );
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.add(identifier, AppenderatorTest.IR("2000", "bar", 1), Suppliers.ofInstance(Committers.nil())); appenderator.add(identifier, AppenderatorTest.ir("2000", "bar", 1), Suppliers.ofInstance(Committers.nil()));
Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.add(identifier, AppenderatorTest.IR("2000", "baz", 1), Suppliers.ofInstance(Committers.nil())); appenderator.add(identifier, AppenderatorTest.ir("2000", "baz", 1), Suppliers.ofInstance(Committers.nil()));
Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.close(); appenderator.close();
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());

View File

@ -91,7 +91,6 @@ import org.apache.druid.segment.loading.DataSegmentMover;
import org.apache.druid.segment.loading.OmniDataSegmentArchiver; import org.apache.druid.segment.loading.OmniDataSegmentArchiver;
import org.apache.druid.segment.loading.OmniDataSegmentKiller; import org.apache.druid.segment.loading.OmniDataSegmentKiller;
import org.apache.druid.segment.loading.OmniDataSegmentMover; import org.apache.druid.segment.loading.OmniDataSegmentMover;
import org.apache.druid.segment.loading.SegmentLoaderConfig;
import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider;
import org.apache.druid.segment.realtime.firehose.ServiceAnnouncingChatHandlerProvider; import org.apache.druid.segment.realtime.firehose.ServiceAnnouncingChatHandlerProvider;
@ -109,7 +108,6 @@ import org.eclipse.jetty.server.Server;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Properties; import java.util.Properties;
import java.util.Set; import java.util.Set;
@ -255,12 +253,6 @@ public class CliPeon extends GuiceRunnable
.to(CoordinatorBasedSegmentHandoffNotifierFactory.class) .to(CoordinatorBasedSegmentHandoffNotifierFactory.class)
.in(LazySingleton.class); .in(LazySingleton.class);
// Override the default SegmentLoaderConfig because we don't actually care about the
// configuration based locations. This will override them anyway. This is also stopping
// configuration of other parameters, but I don't think that's actually a problem.
// Note, if that is actually not a problem, then that probably means we have the wrong abstraction.
binder.bind(SegmentLoaderConfig.class)
.toInstance(new SegmentLoaderConfig().withLocations(Collections.emptyList()));
binder.bind(CoordinatorClient.class).in(LazySingleton.class); binder.bind(CoordinatorClient.class).in(LazySingleton.class);
binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class); binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class);

View File

@ -66,6 +66,7 @@ public class RouterJettyServerInitializer implements JettyServerInitializer
"/", "/",
"/coordinator-console/*", "/coordinator-console/*",
"/public/*", "/public/*",
"/assets/*",
"/old-console/*", "/old-console/*",
"/pages/*", "/pages/*",
"/unified-console.html", "/unified-console.html",

View File

@ -362,7 +362,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
final DatabaseMetaData metaData = client.getMetaData(); final DatabaseMetaData metaData = client.getMetaData();
Assert.assertEquals( Assert.assertEquals(
ImmutableList.of( ImmutableList.of(
ROW(Pair.of("TABLE_CAT", "druid")) row(Pair.of("TABLE_CAT", "druid"))
), ),
getRows(metaData.getCatalogs()) getRows(metaData.getCatalogs())
); );
@ -374,7 +374,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
final DatabaseMetaData metaData = client.getMetaData(); final DatabaseMetaData metaData = client.getMetaData();
Assert.assertEquals( Assert.assertEquals(
ImmutableList.of( ImmutableList.of(
ROW(Pair.of("TABLE_CATALOG", "druid"), Pair.of("TABLE_SCHEM", "druid")) row(Pair.of("TABLE_CATALOG", "druid"), Pair.of("TABLE_SCHEM", "druid"))
), ),
getRows(metaData.getSchemas(null, "druid")) getRows(metaData.getSchemas(null, "druid"))
); );
@ -386,19 +386,19 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
final DatabaseMetaData metaData = client.getMetaData(); final DatabaseMetaData metaData = client.getMetaData();
Assert.assertEquals( Assert.assertEquals(
ImmutableList.of( ImmutableList.of(
ROW( row(
Pair.of("TABLE_CAT", "druid"), Pair.of("TABLE_CAT", "druid"),
Pair.of("TABLE_NAME", CalciteTests.DATASOURCE1), Pair.of("TABLE_NAME", CalciteTests.DATASOURCE1),
Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_TYPE", "TABLE") Pair.of("TABLE_TYPE", "TABLE")
), ),
ROW( row(
Pair.of("TABLE_CAT", "druid"), Pair.of("TABLE_CAT", "druid"),
Pair.of("TABLE_NAME", CalciteTests.DATASOURCE2), Pair.of("TABLE_NAME", CalciteTests.DATASOURCE2),
Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_TYPE", "TABLE") Pair.of("TABLE_TYPE", "TABLE")
), ),
ROW( row(
Pair.of("TABLE_CAT", "druid"), Pair.of("TABLE_CAT", "druid"),
Pair.of("TABLE_NAME", CalciteTests.DATASOURCE3), Pair.of("TABLE_NAME", CalciteTests.DATASOURCE3),
Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_SCHEM", "druid"),
@ -418,25 +418,25 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
final DatabaseMetaData metaData = superuserClient.getMetaData(); final DatabaseMetaData metaData = superuserClient.getMetaData();
Assert.assertEquals( Assert.assertEquals(
ImmutableList.of( ImmutableList.of(
ROW( row(
Pair.of("TABLE_CAT", "druid"), Pair.of("TABLE_CAT", "druid"),
Pair.of("TABLE_NAME", CalciteTests.DATASOURCE1), Pair.of("TABLE_NAME", CalciteTests.DATASOURCE1),
Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_TYPE", "TABLE") Pair.of("TABLE_TYPE", "TABLE")
), ),
ROW( row(
Pair.of("TABLE_CAT", "druid"), Pair.of("TABLE_CAT", "druid"),
Pair.of("TABLE_NAME", CalciteTests.DATASOURCE2), Pair.of("TABLE_NAME", CalciteTests.DATASOURCE2),
Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_TYPE", "TABLE") Pair.of("TABLE_TYPE", "TABLE")
), ),
ROW( row(
Pair.of("TABLE_CAT", "druid"), Pair.of("TABLE_CAT", "druid"),
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE), Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_TYPE", "TABLE") Pair.of("TABLE_TYPE", "TABLE")
), ),
ROW( row(
Pair.of("TABLE_CAT", "druid"), Pair.of("TABLE_CAT", "druid"),
Pair.of("TABLE_NAME", CalciteTests.DATASOURCE3), Pair.of("TABLE_NAME", CalciteTests.DATASOURCE3),
Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_SCHEM", "druid"),
@ -456,7 +456,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
final DatabaseMetaData metaData = client.getMetaData(); final DatabaseMetaData metaData = client.getMetaData();
Assert.assertEquals( Assert.assertEquals(
ImmutableList.of( ImmutableList.of(
ROW( row(
Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", "foo"), Pair.of("TABLE_NAME", "foo"),
Pair.of("COLUMN_NAME", "__time"), Pair.of("COLUMN_NAME", "__time"),
@ -464,7 +464,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "TIMESTAMP"), Pair.of("TYPE_NAME", "TIMESTAMP"),
Pair.of("IS_NULLABLE", "NO") Pair.of("IS_NULLABLE", "NO")
), ),
ROW( row(
Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", "foo"), Pair.of("TABLE_NAME", "foo"),
Pair.of("COLUMN_NAME", "cnt"), Pair.of("COLUMN_NAME", "cnt"),
@ -472,7 +472,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "BIGINT"), Pair.of("TYPE_NAME", "BIGINT"),
Pair.of("IS_NULLABLE", "NO") Pair.of("IS_NULLABLE", "NO")
), ),
ROW( row(
Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", "foo"), Pair.of("TABLE_NAME", "foo"),
Pair.of("COLUMN_NAME", "dim1"), Pair.of("COLUMN_NAME", "dim1"),
@ -480,7 +480,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "VARCHAR"), Pair.of("TYPE_NAME", "VARCHAR"),
Pair.of("IS_NULLABLE", "YES") Pair.of("IS_NULLABLE", "YES")
), ),
ROW( row(
Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", "foo"), Pair.of("TABLE_NAME", "foo"),
Pair.of("COLUMN_NAME", "dim2"), Pair.of("COLUMN_NAME", "dim2"),
@ -488,7 +488,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "VARCHAR"), Pair.of("TYPE_NAME", "VARCHAR"),
Pair.of("IS_NULLABLE", "YES") Pair.of("IS_NULLABLE", "YES")
), ),
ROW( row(
Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", "foo"), Pair.of("TABLE_NAME", "foo"),
Pair.of("COLUMN_NAME", "dim3"), Pair.of("COLUMN_NAME", "dim3"),
@ -496,7 +496,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "VARCHAR"), Pair.of("TYPE_NAME", "VARCHAR"),
Pair.of("IS_NULLABLE", "YES") Pair.of("IS_NULLABLE", "YES")
), ),
ROW( row(
Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", "foo"), Pair.of("TABLE_NAME", "foo"),
Pair.of("COLUMN_NAME", "m1"), Pair.of("COLUMN_NAME", "m1"),
@ -504,7 +504,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "FLOAT"), Pair.of("TYPE_NAME", "FLOAT"),
Pair.of("IS_NULLABLE", "NO") Pair.of("IS_NULLABLE", "NO")
), ),
ROW( row(
Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", "foo"), Pair.of("TABLE_NAME", "foo"),
Pair.of("COLUMN_NAME", "m2"), Pair.of("COLUMN_NAME", "m2"),
@ -512,7 +512,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "DOUBLE"), Pair.of("TYPE_NAME", "DOUBLE"),
Pair.of("IS_NULLABLE", "NO") Pair.of("IS_NULLABLE", "NO")
), ),
ROW( row(
Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", "foo"), Pair.of("TABLE_NAME", "foo"),
Pair.of("COLUMN_NAME", "unique_dim1"), Pair.of("COLUMN_NAME", "unique_dim1"),
@ -547,7 +547,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
final DatabaseMetaData metaData = superuserClient.getMetaData(); final DatabaseMetaData metaData = superuserClient.getMetaData();
Assert.assertEquals( Assert.assertEquals(
ImmutableList.of( ImmutableList.of(
ROW( row(
Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE), Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
Pair.of("COLUMN_NAME", "__time"), Pair.of("COLUMN_NAME", "__time"),
@ -555,7 +555,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "TIMESTAMP"), Pair.of("TYPE_NAME", "TIMESTAMP"),
Pair.of("IS_NULLABLE", "NO") Pair.of("IS_NULLABLE", "NO")
), ),
ROW( row(
Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE), Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
Pair.of("COLUMN_NAME", "cnt"), Pair.of("COLUMN_NAME", "cnt"),
@ -563,7 +563,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "BIGINT"), Pair.of("TYPE_NAME", "BIGINT"),
Pair.of("IS_NULLABLE", "NO") Pair.of("IS_NULLABLE", "NO")
), ),
ROW( row(
Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE), Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
Pair.of("COLUMN_NAME", "dim1"), Pair.of("COLUMN_NAME", "dim1"),
@ -571,7 +571,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "VARCHAR"), Pair.of("TYPE_NAME", "VARCHAR"),
Pair.of("IS_NULLABLE", "YES") Pair.of("IS_NULLABLE", "YES")
), ),
ROW( row(
Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE), Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
Pair.of("COLUMN_NAME", "dim2"), Pair.of("COLUMN_NAME", "dim2"),
@ -579,7 +579,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "VARCHAR"), Pair.of("TYPE_NAME", "VARCHAR"),
Pair.of("IS_NULLABLE", "YES") Pair.of("IS_NULLABLE", "YES")
), ),
ROW( row(
Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE), Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
Pair.of("COLUMN_NAME", "m1"), Pair.of("COLUMN_NAME", "m1"),
@ -587,7 +587,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "FLOAT"), Pair.of("TYPE_NAME", "FLOAT"),
Pair.of("IS_NULLABLE", "NO") Pair.of("IS_NULLABLE", "NO")
), ),
ROW( row(
Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE), Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
Pair.of("COLUMN_NAME", "m2"), Pair.of("COLUMN_NAME", "m2"),
@ -595,7 +595,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "DOUBLE"), Pair.of("TYPE_NAME", "DOUBLE"),
Pair.of("IS_NULLABLE", "NO") Pair.of("IS_NULLABLE", "NO")
), ),
ROW( row(
Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE), Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
Pair.of("COLUMN_NAME", "unique_dim1"), Pair.of("COLUMN_NAME", "unique_dim1"),
@ -928,7 +928,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
} }
} }
private static Map<String, Object> ROW(final Pair<String, ?>... entries) private static Map<String, Object> row(final Pair<String, ?>... entries)
{ {
final Map<String, Object> m = new HashMap<>(); final Map<String, Object> m = new HashMap<>();
for (Pair<String, ?> entry : entries) { for (Pair<String, ?> entry : entries) {

View File

@ -243,70 +243,70 @@ public class BaseCalciteQueryTest extends CalciteTestBase
} }
// Generate timestamps for expected results // Generate timestamps for expected results
public static long T(final String timeString) public static long t(final String timeString)
{ {
return Calcites.jodaToCalciteTimestamp(DateTimes.of(timeString), DateTimeZone.UTC); return Calcites.jodaToCalciteTimestamp(DateTimes.of(timeString), DateTimeZone.UTC);
} }
// Generate timestamps for expected results // Generate timestamps for expected results
public static long T(final String timeString, final String timeZoneString) public static long t(final String timeString, final String timeZoneString)
{ {
final DateTimeZone timeZone = DateTimes.inferTzFromString(timeZoneString); final DateTimeZone timeZone = DateTimes.inferTzFromString(timeZoneString);
return Calcites.jodaToCalciteTimestamp(new DateTime(timeString, timeZone), timeZone); return Calcites.jodaToCalciteTimestamp(new DateTime(timeString, timeZone), timeZone);
} }
// Generate day numbers for expected results // Generate day numbers for expected results
public static int D(final String dayString) public static int d(final String dayString)
{ {
return (int) (Intervals.utc(T("1970"), T(dayString)).toDurationMillis() / (86400L * 1000L)); return (int) (Intervals.utc(t("1970"), t(dayString)).toDurationMillis() / (86400L * 1000L));
} }
public static QuerySegmentSpec QSS(final Interval... intervals) public static QuerySegmentSpec querySegmentSpec(final Interval... intervals)
{ {
return new MultipleIntervalSegmentSpec(Arrays.asList(intervals)); return new MultipleIntervalSegmentSpec(Arrays.asList(intervals));
} }
public static AndDimFilter AND(DimFilter... filters) public static AndDimFilter and(DimFilter... filters)
{ {
return new AndDimFilter(Arrays.asList(filters)); return new AndDimFilter(Arrays.asList(filters));
} }
public static OrDimFilter OR(DimFilter... filters) public static OrDimFilter or(DimFilter... filters)
{ {
return new OrDimFilter(Arrays.asList(filters)); return new OrDimFilter(Arrays.asList(filters));
} }
public static NotDimFilter NOT(DimFilter filter) public static NotDimFilter not(DimFilter filter)
{ {
return new NotDimFilter(filter); return new NotDimFilter(filter);
} }
public static InDimFilter IN(String dimension, List<String> values, ExtractionFn extractionFn) public static InDimFilter in(String dimension, List<String> values, ExtractionFn extractionFn)
{ {
return new InDimFilter(dimension, values, extractionFn); return new InDimFilter(dimension, values, extractionFn);
} }
public static SelectorDimFilter SELECTOR(final String fieldName, final String value, final ExtractionFn extractionFn) public static SelectorDimFilter selector(final String fieldName, final String value, final ExtractionFn extractionFn)
{ {
return new SelectorDimFilter(fieldName, value, extractionFn); return new SelectorDimFilter(fieldName, value, extractionFn);
} }
public static ExpressionDimFilter EXPRESSION_FILTER(final String expression) public static ExpressionDimFilter expressionFilter(final String expression)
{ {
return new ExpressionDimFilter(expression, CalciteTests.createExprMacroTable()); return new ExpressionDimFilter(expression, CalciteTests.createExprMacroTable());
} }
public static DimFilter NUMERIC_SELECTOR( public static DimFilter numeric_Selector(
final String fieldName, final String fieldName,
final String value, final String value,
final ExtractionFn extractionFn final ExtractionFn extractionFn
) )
{ {
// We use Bound filters for numeric equality to achieve "10.0" = "10" // We use Bound filters for numeric equality to achieve "10.0" = "10"
return BOUND(fieldName, value, value, false, false, extractionFn, StringComparators.NUMERIC); return bound(fieldName, value, value, false, false, extractionFn, StringComparators.NUMERIC);
} }
public static BoundDimFilter BOUND( public static BoundDimFilter bound(
final String fieldName, final String fieldName,
final String lower, final String lower,
final String upper, final String upper,
@ -319,7 +319,7 @@ public class BaseCalciteQueryTest extends CalciteTestBase
return new BoundDimFilter(fieldName, lower, upper, lowerStrict, upperStrict, null, extractionFn, comparator); return new BoundDimFilter(fieldName, lower, upper, lowerStrict, upperStrict, null, extractionFn, comparator);
} }
public static BoundDimFilter TIME_BOUND(final Object intervalObj) public static BoundDimFilter timeBound(final Object intervalObj)
{ {
final Interval interval = new Interval(intervalObj, ISOChronology.getInstanceUTC()); final Interval interval = new Interval(intervalObj, ISOChronology.getInstanceUTC());
return new BoundDimFilter( return new BoundDimFilter(
@ -329,32 +329,32 @@ public class BaseCalciteQueryTest extends CalciteTestBase
false, false,
true, true,
null, null,
null, null,
StringComparators.NUMERIC StringComparators.NUMERIC
); );
} }
public static CascadeExtractionFn CASCADE(final ExtractionFn... fns) public static CascadeExtractionFn cascade(final ExtractionFn... fns)
{ {
return new CascadeExtractionFn(fns); return new CascadeExtractionFn(fns);
} }
public static List<DimensionSpec> DIMS(final DimensionSpec... dimensionSpecs) public static List<DimensionSpec> dimensionSpec(final DimensionSpec... dimensionSpecs)
{ {
return Arrays.asList(dimensionSpecs); return Arrays.asList(dimensionSpecs);
} }
public static List<AggregatorFactory> AGGS(final AggregatorFactory... aggregators) public static List<AggregatorFactory> aggregators(final AggregatorFactory... aggregators)
{ {
return Arrays.asList(aggregators); return Arrays.asList(aggregators);
} }
public static DimFilterHavingSpec HAVING(final DimFilter filter) public static DimFilterHavingSpec having(final DimFilter filter)
{ {
return new DimFilterHavingSpec(filter, true); return new DimFilterHavingSpec(filter, true);
} }
public static ExpressionVirtualColumn EXPRESSION_VIRTUAL_COLUMN( public static ExpressionVirtualColumn expression_Virtual_Column(
final String name, final String name,
final String expression, final String expression,
final ValueType outputType final ValueType outputType
@ -363,7 +363,7 @@ public class BaseCalciteQueryTest extends CalciteTestBase
return new ExpressionVirtualColumn(name, expression, outputType, CalciteTests.createExprMacroTable()); return new ExpressionVirtualColumn(name, expression, outputType, CalciteTests.createExprMacroTable());
} }
public static ExpressionPostAggregator EXPRESSION_POST_AGG(final String name, final String expression) public static ExpressionPostAggregator expresionPostAgg(final String name, final String expression)
{ {
return new ExpressionPostAggregator(name, expression, null, CalciteTests.createExprMacroTable()); return new ExpressionPostAggregator(name, expression, null, CalciteTests.createExprMacroTable());
} }

View File

@ -2,6 +2,7 @@ node/
node_modules/ node_modules/
resources/ resources/
public/ public/
assets/
lib/*.css lib/*.css
coordinator-console/ coordinator-console/

File diff suppressed because it is too large Load Diff

View File

@ -17,9 +17,9 @@
"start": "webpack-dev-server --hot --open" "start": "webpack-dev-server --hot --open"
}, },
"dependencies": { "dependencies": {
"@blueprintjs/core": "^3.12.0", "@blueprintjs/core": "1.0.1",
"@types/hjson": "^2.4.0",
"axios": "^0.18.0", "axios": "^0.18.0",
"brace": "^0.11.1",
"classnames": "^2.2.6", "classnames": "^2.2.6",
"d3-array": "^2.0.3", "d3-array": "^2.0.3",
"druid-console": "^0.0.2", "druid-console": "^0.0.2",
@ -28,16 +28,20 @@
"hjson": "^3.1.2", "hjson": "^3.1.2",
"lodash.debounce": "^4.0.8", "lodash.debounce": "^4.0.8",
"numeral": "^2.0.6", "numeral": "^2.0.6",
"react": "^16.7.0", "prop-types": "^15.7.2",
"react-dom": "^16.7.0", "react": "^16.8.3",
"react-ace": "^6.4.0",
"react-addons-css-transition-group": "^15.6.2",
"react-dom": "^16.8.3",
"react-router": "^4.3.1", "react-router": "^4.3.1",
"react-router-dom": "^4.3.1", "react-router-dom": "^4.3.1",
"react-table": "^6.8.6", "react-table": "^6.9.2",
"tslib": "^1.9.3" "tslib": "^1.9.3"
}, },
"devDependencies": { "devDependencies": {
"@types/classnames": "^2.2.7", "@types/classnames": "^2.2.7",
"@types/d3-array": "^1.2.4", "@types/d3-array": "^1.2.4",
"@types/hjson": "^2.4.0",
"@types/jest": "^23.3.13", "@types/jest": "^23.3.13",
"@types/lodash.debounce": "^4.0.4", "@types/lodash.debounce": "^4.0.4",
"@types/mocha": "^5.2.5", "@types/mocha": "^5.2.5",

View File

@ -23,6 +23,10 @@ cp -r ./node_modules/druid-console/coordinator-console .
cp -r ./node_modules/druid-console/pages . cp -r ./node_modules/druid-console/pages .
cp ./node_modules/druid-console/index.html . cp ./node_modules/druid-console/index.html .
echo "Copying blueprint assets in..."
sed 's|url("assets|url("/assets|g' "./node_modules/@blueprintjs/core/dist/blueprint.css" > lib/blueprint.css
cp -r "./node_modules/@blueprintjs/core/dist/assets" .
echo "Transpiling ReactTable CSS..." echo "Transpiling ReactTable CSS..."
PATH="./target/node:$PATH" ./node_modules/.bin/stylus lib/react-table.styl -o lib/react-table.css PATH="./target/node:$PATH" ./node_modules/.bin/stylus lib/react-table.styl -o lib/react-table.css

View File

@ -17,9 +17,10 @@
# limitations under the License. # limitations under the License.
rm -rf \ rm -rf \
lib/react-table.css \ lib/*.css \
node_modules \ node_modules \
coordinator-console \ coordinator-console \
pages \ pages \
public \ public \
assets \
index.html index.html

View File

@ -24,3 +24,4 @@ cp -r coordinator-console "$1"
cp -r old-console "$1" cp -r old-console "$1"
cp -r pages "$1" cp -r pages "$1"
cp -r public "$1" cp -r public "$1"
cp -r assets "$1"

View File

@ -19,20 +19,8 @@
import { resolveSrv } from 'dns'; import { resolveSrv } from 'dns';
import * as React from 'react'; import * as React from 'react';
import axios from 'axios'; import axios from 'axios';
import { import { InputGroup } from "@blueprintjs/core";
FormGroup, import { HTMLSelect, FormGroup, NumericInput, TagInput } from "../components/filler";
Button,
InputGroup,
Dialog,
NumericInput,
Classes,
Tooltip,
AnchorButton,
TagInput,
Intent,
ButtonGroup,
HTMLSelect
} from "@blueprintjs/core";
interface Field { interface Field {
name: string; name: string;
@ -67,7 +55,7 @@ export class AutoForm<T> extends React.Component<AutoFormProps<T>, AutoFormState
const { model, onChange } = this.props; const { model, onChange } = this.props;
return <NumericInput return <NumericInput
value={(model as any)[field.name]} value={(model as any)[field.name]}
onValueChange={v => { onValueChange={(v: any) => {
if (isNaN(v)) return; if (isNaN(v)) return;
onChange(Object.assign({}, model, { [field.name]: v })); onChange(Object.assign({}, model, { [field.name]: v }));
}} }}
@ -79,7 +67,7 @@ export class AutoForm<T> extends React.Component<AutoFormProps<T>, AutoFormState
const { model, onChange } = this.props; const { model, onChange } = this.props;
return <NumericInput return <NumericInput
value={(model as any)[field.name]} value={(model as any)[field.name]}
onValueChange={v => { onValueChange={(v: number) => {
if (isNaN(v)) return; if (isNaN(v)) return;
onChange(Object.assign({}, model, { [field.name]: v })); onChange(Object.assign({}, model, { [field.name]: v }));
}} }}
@ -102,12 +90,14 @@ export class AutoForm<T> extends React.Component<AutoFormProps<T>, AutoFormState
private renderBooleanInput(field: Field): JSX.Element { private renderBooleanInput(field: Field): JSX.Element {
const { model, onChange } = this.props; const { model, onChange } = this.props;
return <HTMLSelect return <HTMLSelect
options={["True", "False"]}
value={(model as any)[field.name] === true ? "True" : "False"} value={(model as any)[field.name] === true ? "True" : "False"}
onChange={e => { onChange={(e: any) => {
onChange(Object.assign({}, model, { [field.name]: e.currentTarget.value === "True" })); onChange(Object.assign({}, model, { [field.name]: e.currentTarget.value === "True" }));
}} }}
/> >
<option value="True">True</option>
<option value="False">False</option>
</HTMLSelect>
} }
private renderStringArrayInput(field: Field): JSX.Element { private renderStringArrayInput(field: Field): JSX.Element {
@ -118,7 +108,7 @@ export class AutoForm<T> extends React.Component<AutoFormProps<T>, AutoFormState
onChange={(v: any) => { onChange={(v: any) => {
onChange(Object.assign({}, model, { [field.name]: v })); onChange(Object.assign({}, model, { [field.name]: v }));
}} }}
addOnBlur={true} fill
/>; />;
} }

View File

@ -0,0 +1,27 @@
/*
* 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.
*/
.pt-select {
&.pt-fill {
flex: 1;
}
}
.form-group {
margin: 0 0 15px;
}

View File

@ -0,0 +1,259 @@
/*
* 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.
*/
import { Button } from '@blueprintjs/core';
import * as React from 'react';
import classNames from 'classnames';
import './filler.scss';
export const IconNames = {
ERROR: "error" as "error",
PLUS: "plus" as "plus",
REFRESH: "refresh" as "refresh",
APPLICATION: "application" as "application",
GRAPH: "graph" as "graph",
MAP: "map" as "map",
TH: "th" as "th",
USER: "user" as "user",
GIT_BRANCH: "git-branch" as "git-branch",
COG: "cog" as "cog",
MULTI_SELECT: "multi-select" as "multi-select",
STACKED_CHART: "stacked-chart" as "stacked-chart",
GANTT_CHART: "gantt-chart" as "gantt-chart",
DATABASE: "database" as "database",
SETTINGS: "settings" as "settings",
HELP: "help" as "help",
SHARE: "share" as "share",
CROSS: "cross" as "cross",
ARROW_LEFT: "arrow-left" as "arrow-left",
CARET_RIGHT: "caret-right" as "caret-right",
TICK: "tick" as "tick",
ARROW_RIGHT: "right-arrow" as "right-arrow",
TRASH: "trash" as "trash",
CARET_DOWN: "caret-down" as "caret-down",
ARROW_UP: "arrow-up" as "arrow-up",
ARROW_DOWN: "arrow-down" as "arrow-down",
};
export type IconNames = typeof IconNames[keyof typeof IconNames];
export class H5 extends React.Component<{}, {}> {
render() {
const { children } = this.props;
return <h5>{children}</h5>;
}
}
export class Card extends React.Component<{ interactive?: boolean }, {}> {
render() {
const { interactive, children } = this.props;
return <div className={classNames("pt-card", { 'pt-interactive': interactive })}>
{children}
</div>;
}
}
export class Icon extends React.Component<{ icon: string, color?: string }, {}> {
render() {
const { color, icon } = this.props;
return <span className={classNames('pt-icon-standard', 'pt-icon-' + icon)} style={{ color }}/>;
}
}
export class ControlGroup extends React.Component<{}, {}> {
render() {
return <div className="pt-control-group" {...this.props}/>;
}
}
export class ButtonGroup extends React.Component<{ vertical?: boolean, fixed?: boolean }, {}> {
render() {
const { vertical, fixed, children } = this.props;
return <div className={classNames("pt-button-group", { 'pt-vertical': vertical, 'pt-fixed': fixed })}>
{children}
</div>;
}
}
export class Label extends React.Component<{}, {}> {
render() {
const { children } = this.props;
return <label className="pt-label">{children}</label>;
}
}
export class FormGroup extends React.Component<{ className?: string, label?: string }, {}> {
render() {
const { className, label, children } = this.props;
return <div className={classNames("form-group", className)}>
{ label ? <Label>{label}</Label> : null }
{children}
</div>;
}
}
export const Alignment = {
LEFT: "left" as "left",
RIGHT: "right" as "right",
};
export type Alignment = typeof Alignment[keyof typeof Alignment];
export class Navbar extends React.Component<{ className?: string }, {}> {
render() {
const { className, children } = this.props;
return <nav className={classNames("pt-navbar", className)}>
{children}
</nav>;
}
}
export class NavbarGroup extends React.Component<{ align: Alignment }, {}> {
render() {
const { align, children } = this.props;
return <div className={classNames('pt-navbar-group', 'pt-align-' + align)}>
{children}
</div>;
}
}
export class NavbarDivider extends React.Component<{}, {}> {
render() {
return <span className="pt-navbar-divider"/>;
}
}
export class HTMLSelect extends React.Component<{ key?: string; style?: any; onChange: any; value: any; fill?: boolean }, {}> {
render() {
const { key, style, onChange, value, fill, children } = this.props;
return <div className={classNames("pt-select", { 'pt-fill': fill })} key={key} style={style}>
<select onChange={onChange} value={value}>{children}</select>
</div>;
}
}
export class TextArea extends React.Component<{ className?: string; onChange?: any; value?: string }, {}> {
render() {
const { className, value, onChange } = this.props;
return <textarea
className={classNames("pt-input", className)}
value={value}
onChange={onChange}
/>;
}
}
export interface NumericInputProps {
value: number | null;
onValueChange: any;
min?: number;
max?: number;
stepSize?: number;
majorStepSize?: number
}
export class NumericInput extends React.Component<NumericInputProps, { stringValue: string }> {
static defaultProps = {
stepSize: 1,
majorStepSize: 10
}
constructor(props: NumericInputProps) {
super(props);
this.state = {
stringValue: typeof props.value === 'number' ? String(props.value) : ''
}
}
private constrain(n: number): number {
const { min, max } = this.props;
if (typeof min === 'number') n = Math.max(n, min);
if (typeof max === 'number') n = Math.min(n, max);
return n
}
private handleChange = (e: any) => {
let stringValue = e.target.value.replace(/[^\d.+-]/g, '');
let numValue = parseFloat(stringValue);
if (isNaN(numValue)) {
this.setState({ stringValue });
} else {
numValue = this.constrain(numValue);
stringValue = String(numValue);
this.setState({ stringValue });
this.props.onValueChange(numValue);
}
}
private handleClick = (e: any, direction: number) => {
const { stepSize, majorStepSize } = this.props;
const { stringValue } = this.state;
const diff = direction * (e.shiftKey ? majorStepSize as number : stepSize as number);
const numValue = this.constrain((parseFloat(stringValue) || 0) + diff);
this.setState({ stringValue: String(numValue) });
this.props.onValueChange(numValue);
}
render() {
const { stringValue } = this.state;
return <ControlGroup>
<input className="pt-input" value={stringValue} onChange={this.handleChange}/>
<ButtonGroup fixed>
<Button iconName="caret-up" onClick={(e: any) => this.handleClick(e, +1)}/>
<Button iconName="caret-down" onClick={(e: any) => this.handleClick(e, -1)}/>
</ButtonGroup>
</ControlGroup>;
}
}
export interface TagInputProps {
values: string[];
onChange: any;
fill?: boolean;
}
export class TagInput extends React.Component<TagInputProps, { stringValue: string }> {
constructor(props: TagInputProps) {
super(props);
this.state = {
stringValue: Array.isArray(props.values) ? props.values.join(', ') : ''
}
}
handleChange = (e: any) => {
let stringValue = e.target.value;
let newValues = stringValue.split(',').map((v: string) => v.trim());
let newValuesFiltered = newValues.filter(Boolean);
this.setState({
stringValue: newValues.length === newValuesFiltered.length ? newValues.join(', ') : stringValue
});
this.props.onChange(newValuesFiltered);
}
render() {
const { fill } = this.props;
const { stringValue } = this.state;
return <input
className={classNames("pt-input", {'pt-fill': fill })}
value={stringValue}
onChange={this.handleChange}
/>;
}
}

View File

@ -17,9 +17,10 @@
*/ */
.header-bar { .header-bar {
overflow: hidden; z-index: 10;
.logo { .logo {
position: relative;
width: 100px; width: 100px;
height: 50px; height: 50px;
@ -31,4 +32,13 @@
height: 75px; height: 75px;
} }
} }
.config-popover .pt-popover-content,
.legacy-popover .pt-popover-content {
width: 240px;
}
.help-popover .pt-popover-content {
width: 180px;
}
} }

View File

@ -17,12 +17,9 @@
*/ */
import * as React from 'react'; import * as React from 'react';
import { import classNames from 'classnames';
Button, import { Button, Classes, AnchorButton, Popover, Position, Menu, MenuItem } from "@blueprintjs/core";
Classes, import { IconNames, NavbarGroup, Alignment, NavbarDivider, Navbar } from "../components/filler";
AnchorButton, NavbarGroup, Alignment, NavbarHeading, NavbarDivider, Popover, Position, Navbar, Menu, MenuItem
} from "@blueprintjs/core";
import { IconNames } from "@blueprintjs/icons";
import { AboutDialog } from "../dialogs/about-dialog"; import { AboutDialog } from "../dialogs/about-dialog";
import { CoordinatorDynamicConfigDialog } from '../dialogs/coordinator-dynamic-config'; import { CoordinatorDynamicConfigDialog } from '../dialogs/coordinator-dynamic-config';
import "./header-bar.scss"; import "./header-bar.scss";
@ -90,19 +87,19 @@ export class HeaderBar extends React.Component<HeaderBarProps, HeaderBarState> {
const { aboutDialogOpen, coordinatorDynamicConfigDialogOpen } = this.state; const { aboutDialogOpen, coordinatorDynamicConfigDialogOpen } = this.state;
const legacyMenu = <Menu> const legacyMenu = <Menu>
<MenuItem icon={IconNames.GRAPH} text="Legacy coordinator console" href={LEGACY_COORDINATOR_CONSOLE} target="_blank" /> <MenuItem iconName={IconNames.GRAPH} text="Legacy coordinator console" href={LEGACY_COORDINATOR_CONSOLE} target="_blank" />
<MenuItem icon={IconNames.MAP} text="Legacy overlord console" href={LEGACY_OVERLORD_CONSOLE} target="_blank" /> <MenuItem iconName={IconNames.MAP} text="Legacy overlord console" href={LEGACY_OVERLORD_CONSOLE} target="_blank" />
</Menu>; </Menu>;
const helpMenu = <Menu> const helpMenu = <Menu>
<MenuItem icon={IconNames.GRAPH} text="About" onClick={() => this.setState({ aboutDialogOpen: true })} /> <MenuItem iconName={IconNames.GRAPH} text="About" onClick={() => this.setState({ aboutDialogOpen: true })} />
<MenuItem icon={IconNames.TH} text="Docs" href={DRUID_DOCS} target="_blank" /> <MenuItem iconName={IconNames.TH} text="Docs" href={DRUID_DOCS} target="_blank" />
<MenuItem icon={IconNames.USER} text="User group" href={DRUID_USER_GROUP} target="_blank" /> <MenuItem iconName={IconNames.USER} text="User group" href={DRUID_USER_GROUP} target="_blank" />
<MenuItem icon={IconNames.GIT_BRANCH} text="GitHub" href={DRUID_GITHUB} target="_blank" /> <MenuItem iconName={IconNames.GIT_BRANCH} text="GitHub" href={DRUID_GITHUB} target="_blank" />
</Menu>; </Menu>;
const configMenu = <Menu> const configMenu = <Menu>
<MenuItem icon={IconNames.COG} text="Coordinator dynamic config" onClick={() => this.setState({ coordinatorDynamicConfigDialogOpen: true })}/> <MenuItem iconName={IconNames.COG} text="Coordinator dynamic config" onClick={() => this.setState({ coordinatorDynamicConfigDialogOpen: true })}/>
</Menu>; </Menu>;
return <Navbar className="header-bar"> return <Navbar className="header-bar">
@ -111,32 +108,30 @@ export class HeaderBar extends React.Component<HeaderBarProps, HeaderBarState> {
{this.renderLogo()} {this.renderLogo()}
</a> </a>
<NavbarDivider /> <NavbarDivider />
<AnchorButton className={Classes.MINIMAL} icon={IconNames.MULTI_SELECT} text="Datasources" href="#datasources" active={active === 'datasources'} /> <AnchorButton className={classNames(Classes.MINIMAL, { 'pt-active': active === 'datasources' })} iconName={IconNames.MULTI_SELECT} text="Datasources" href="#datasources" />
<AnchorButton className={Classes.MINIMAL} icon={IconNames.FULL_STACKED_CHART} text="Segments" href="#segments" active={active === 'segments'} /> <AnchorButton className={classNames(Classes.MINIMAL, { 'pt-active': active === 'segments' })} iconName={IconNames.STACKED_CHART} text="Segments" href="#segments" />
<AnchorButton className={Classes.MINIMAL} icon={IconNames.GANTT_CHART} text="Tasks" href="#tasks" active={active === 'tasks'} /> <AnchorButton className={classNames(Classes.MINIMAL, { 'pt-active': active === 'tasks' })} iconName={IconNames.GANTT_CHART} text="Tasks" href="#tasks" />
<AnchorButton className={Classes.MINIMAL} icon={IconNames.DATABASE} text="Data servers" href="#servers" active={active === 'servers'} /> <AnchorButton className={classNames(Classes.MINIMAL, { 'pt-active': active === 'servers' })} iconName={IconNames.DATABASE} text="Data servers" href="#servers" />
<NavbarDivider /> <NavbarDivider />
<AnchorButton className={Classes.MINIMAL} icon={IconNames.CONSOLE} text="SQL" href="#sql" active={active === 'sql'} /> <AnchorButton className={classNames(Classes.MINIMAL, { 'pt-active': active === 'sql' })} iconName={IconNames.APPLICATION} text="SQL" href="#sql" />
<Popover content={configMenu} position={Position.BOTTOM_LEFT}> <Popover className="config-popover" content={configMenu} position={Position.BOTTOM_LEFT} inline>
<Button className={Classes.MINIMAL} icon={IconNames.SETTINGS} text="Config"/> <Button className={Classes.MINIMAL} iconName={IconNames.SETTINGS} text="Config"/>
</Popover> </Popover>
</NavbarGroup> </NavbarGroup>
<NavbarGroup align={Alignment.RIGHT}> <NavbarGroup align={Alignment.RIGHT}>
<Popover content={legacyMenu} position={Position.BOTTOM_LEFT}> <Popover className="legacy-popover" content={legacyMenu} position={Position.BOTTOM_RIGHT} inline>
<Button className={Classes.MINIMAL} icon={IconNames.SHARE} text="Legacy" /> <Button className={Classes.MINIMAL} iconName={IconNames.SHARE} text="Legacy" />
</Popover> </Popover>
<Popover content={helpMenu} position={Position.BOTTOM_LEFT}> <Popover className="help-popover" content={helpMenu} position={Position.BOTTOM_RIGHT} inline>
<Button className={Classes.MINIMAL} icon={IconNames.LIFESAVER} text="Help" /> <Button className={Classes.MINIMAL} iconName={IconNames.HELP} text="Help" />
</Popover> </Popover>
</NavbarGroup> </NavbarGroup>
<AboutDialog { aboutDialogOpen ? <AboutDialog
isOpen={aboutDialogOpen}
onClose={() => this.setState({ aboutDialogOpen: false })} onClose={() => this.setState({ aboutDialogOpen: false })}
/> /> : null }
<CoordinatorDynamicConfigDialog { coordinatorDynamicConfigDialogOpen ? <CoordinatorDynamicConfigDialog
isOpen={coordinatorDynamicConfigDialogOpen}
onClose={() => this.setState({ coordinatorDynamicConfigDialogOpen: false })} onClose={() => this.setState({ coordinatorDynamicConfigDialogOpen: false })}
/> /> : null }
</Navbar>; </Navbar>;
} }
} }

View File

@ -18,16 +18,8 @@
import * as React from 'react'; import * as React from 'react';
import axios from 'axios'; import axios from 'axios';
import { import { Button, InputGroup, Collapse } from "@blueprintjs/core";
FormGroup, import { IconNames, FormGroup, HTMLSelect, Card, ControlGroup, NumericInput, TagInput } from "../components/filler";
Button,
ControlGroup,
Card,
InputGroup,
HTMLSelect,
Collapse, NumericInput, TagInput,
} from "@blueprintjs/core";
import { IconNames } from "@blueprintjs/icons";
import './rule-editor.scss'; import './rule-editor.scss';
export interface Rule { export interface Rule {
@ -156,27 +148,32 @@ export class RuleEditor extends React.Component<RuleEditorProps, RuleEditorState
const ruleTiers = Object.keys(tieredReplicants).sort(); const ruleTiers = Object.keys(tieredReplicants).sort();
return ruleTiers.map(tier => { return ruleTiers.map(tier => {
return <ControlGroup key={tier}> return <ControlGroup key={tier}>
<Button minimal style={{pointerEvents: 'none'}}>Replicants:</Button> <Button className="pt-minimal" style={{pointerEvents: 'none'}}>Replicants:</Button>
<NumericInput <NumericInput
value={tieredReplicants[tier]} value={tieredReplicants[tier]}
onValueChange={v => { onValueChange={(v: number) => {
if (isNaN(v)) return; if (isNaN(v)) return;
onChange(RuleEditor.changeTierReplication(rule, tier, v)); onChange(RuleEditor.changeTierReplication(rule, tier, v));
}} }}
min={1} min={1}
max={256} max={256}
/> />
<Button minimal style={{pointerEvents: 'none'}}>Tier:</Button> <Button className="pt-minimal" style={{pointerEvents: 'none'}}>Tier:</Button>
<HTMLSelect <HTMLSelect
fill={true} fill={true}
value={tier} value={tier}
options={tiers.filter(t => t === tier || !tieredReplicants[t])} onChange={(e: any) => onChange(RuleEditor.changeTier(rule, tier, e.target.value))}
onChange={e => onChange(RuleEditor.changeTier(rule, tier, e.target.value))} >
/> {
tiers.filter(t => t === tier || !tieredReplicants[t]).map((t) => {
return <option key={t} value={t}>{t}</option>;
})
}
</HTMLSelect>
<Button <Button
disabled={ruleTiers.length === 1} disabled={ruleTiers.length === 1}
onClick={() => this.removeTier(tier)} onClick={() => this.removeTier(tier)}
icon={IconNames.TRASH} iconName={IconNames.TRASH}
/> />
</ControlGroup>; </ControlGroup>;
}); });
@ -187,22 +184,20 @@ export class RuleEditor extends React.Component<RuleEditorProps, RuleEditorState
if (Object.keys(rule.tieredReplicants || {}).length >= Object.keys(tiers).length) return null; if (Object.keys(rule.tieredReplicants || {}).length >= Object.keys(tiers).length) return null;
return <FormGroup className="right"> return <FormGroup className="right">
<Button onClick={this.addTier} minimal icon={IconNames.PLUS}>Add a tier</Button> <Button onClick={this.addTier} className="pt-minimal" iconName={IconNames.PLUS}>Add a tier</Button>
</FormGroup>; </FormGroup>;
} }
renderColocatedDataSources() { renderColocatedDataSources() {
const { rule, onChange } = this.props; const { rule, onChange } = this.props;
return <ControlGroup> return <FormGroup label="Colocated datasources:">
<Button minimal style={{pointerEvents: 'none'}}>Colocated datasources:</Button>
<TagInput <TagInput
values={rule.colocatedDataSources || []} values={rule.colocatedDataSources || []}
onChange={(v: any) => onChange(RuleEditor.changeColocatedDataSources(rule, v)) } onChange={(v: any) => onChange(RuleEditor.changeColocatedDataSources(rule, v)) }
addOnBlur={true} fill
fill={true}
/> />
</ControlGroup>; </FormGroup>;
} }
render() { render() {
@ -211,30 +206,18 @@ export class RuleEditor extends React.Component<RuleEditorProps, RuleEditorState
if (!rule) return null; if (!rule) return null;
const ruleLoadTypes: {label: string, value: LoadType}[] = [
{label: 'Load', value: 'load'},
{label: 'Drop', value: 'drop'},
{label: 'Broadcast', value: 'broadcast'}
];
const ruleTimeTypes: {label: string, value: TimeType}[] = [
{label: 'forever', value: 'Forever'},
{label: 'by period', value: 'ByPeriod'},
{label: 'by interval', value: 'ByInterval'}
];
const ruleLoadType = RuleEditor.getLoadType(rule); const ruleLoadType = RuleEditor.getLoadType(rule);
const ruleTimeType = RuleEditor.getTimeType(rule); const ruleTimeType = RuleEditor.getTimeType(rule);
return <div className="rule-editor"> return <div className="rule-editor">
<div className="title"> <div className="title">
<Button className="left" minimal rightIcon={isOpen ? IconNames.CARET_DOWN : IconNames.CARET_RIGHT} onClick={() => this.setState({isOpen: !isOpen})}> <Button className="left pt-minimal" rightIconName={isOpen ? IconNames.CARET_DOWN : IconNames.CARET_RIGHT} onClick={() => this.setState({isOpen: !isOpen})}>
{RuleEditor.ruleToString(rule)} {RuleEditor.ruleToString(rule)}
</Button> </Button>
<div className="spacer"/> <div className="spacer"/>
{moveUp ? <Button minimal icon={IconNames.ARROW_UP} onClick={moveUp}/> : null} {moveUp ? <Button className="pt-minimal" iconName={IconNames.ARROW_UP} onClick={moveUp}/> : null}
{moveDown ? <Button minimal icon={IconNames.ARROW_DOWN} onClick={moveDown}/> : null} {moveDown ? <Button className="pt-minimal" iconName={IconNames.ARROW_DOWN} onClick={moveDown}/> : null}
<Button minimal icon={IconNames.TRASH} onClick={onDelete}/> <Button className="pt-minimal" iconName={IconNames.TRASH} onClick={onDelete}/>
</div> </div>
<Collapse isOpen={isOpen}> <Collapse isOpen={isOpen}>
@ -243,14 +226,20 @@ export class RuleEditor extends React.Component<RuleEditorProps, RuleEditorState
<ControlGroup> <ControlGroup>
<HTMLSelect <HTMLSelect
value={ruleLoadType} value={ruleLoadType}
options={ruleLoadTypes} onChange={(e: any) => onChange(RuleEditor.changeLoadType(rule, e.target.value as any))}
onChange={e => onChange(RuleEditor.changeLoadType(rule, e.target.value as any))} >
/> <option value="load">Load</option>
<option value="drop">Drop</option>
<option value="broadcast">Broadcast</option>
</HTMLSelect>
<HTMLSelect <HTMLSelect
value={ruleTimeType} value={ruleTimeType}
options={ruleTimeTypes} onChange={(e: any) => onChange(RuleEditor.changeTimeType(rule, e.target.value as any))}
onChange={e => onChange(RuleEditor.changeTimeType(rule, e.target.value as any))} >
/> <option value="Forever">forever</option>
<option value="ByPeriod">by period</option>
<option value="ByInterval">by interval</option>
</HTMLSelect>
{ ruleTimeType === 'ByPeriod' && <InputGroup value={rule.period || ''} onChange={(e: any) => onChange(RuleEditor.changePeriod(rule, e.target.value as any))}/>} { ruleTimeType === 'ByPeriod' && <InputGroup value={rule.period || ''} onChange={(e: any) => onChange(RuleEditor.changePeriod(rule, e.target.value as any))}/>}
{ ruleTimeType === 'ByInterval' && <InputGroup value={rule.interval || ''} onChange={(e: any) => onChange(RuleEditor.changeInterval(rule, e.target.value as any))}/>} { ruleTimeType === 'ByInterval' && <InputGroup value={rule.interval || ''} onChange={(e: any) => onChange(RuleEditor.changeInterval(rule, e.target.value as any))}/>}
</ControlGroup> </ControlGroup>

View File

@ -18,11 +18,14 @@
import * as React from 'react'; import * as React from 'react';
import * as classNames from 'classnames'; import * as classNames from 'classnames';
import { import * as ace from 'brace'
TextArea, import AceEditor from "react-ace";
Intent, import 'brace/mode/sql';
Button import 'brace/mode/hjson';
} from "@blueprintjs/core"; import 'brace/theme/solarized_dark';
import 'brace/ext/language_tools';
import { Intent, Button } from "@blueprintjs/core";
import { IconNames } from './filler';
export interface SqlControlProps extends React.Props<any> { export interface SqlControlProps extends React.Props<any> {
initSql: string | null; initSql: string | null;
@ -31,39 +34,56 @@ export interface SqlControlProps extends React.Props<any> {
export interface SqlControlState { export interface SqlControlState {
query: string; query: string;
autoCompleteOn: boolean;
} }
export class SqlControl extends React.Component<SqlControlProps, SqlControlState> { export class SqlControl extends React.Component<SqlControlProps, SqlControlState> {
constructor(props: SqlControlProps, context: any) { constructor(props: SqlControlProps, context: any) {
super(props, context); super(props, context);
this.state = { this.state = {
query: props.initSql || '' query: props.initSql || '',
autoCompleteOn: true
}; };
} }
private handleChange = (e: React.ChangeEvent<HTMLTextAreaElement>) => { private handleChange = (newValue: string): void => {
this.setState({ this.setState({
query: e.target.value query: newValue
}); })
} }
render() { render() {
const { onRun } = this.props; const { onRun } = this.props;
const { query } = this.state; const { query, autoCompleteOn } = this.state;
const isRune = query.trim().startsWith('{'); const isRune = query.trim().startsWith('{');
// Maybe use: https://github.com/securingsincity/react-ace/blob/master/docs/Ace.md // Set the key in the AceEditor to force a rebind and prevent an error that happens otherwise
return <div className="sql-control"> return <div className="sql-control">
<TextArea <AceEditor
className="bp3-fill" key={isRune ? "hjson" : "sql"}
large={true} mode={isRune ? "hjson" : "sql"}
intent={Intent.PRIMARY} theme="solarized_dark"
name="ace-editor"
onChange={this.handleChange} onChange={this.handleChange}
focus={true}
fontSize={14}
width={'100%'}
height={"30vh"}
showPrintMargin={false}
value={query} value={query}
editorProps={{
$blockScrolling: Infinity
}}
setOptions={{
enableBasicAutocompletion: isRune ? false : autoCompleteOn,
enableLiveAutocompletion: isRune ? false : autoCompleteOn,
showLineNumbers: true,
tabSize: 2,
}}
/> />
<div className="buttons"> <div className="buttons">
<Button rightIcon="caret-right" onClick={() => onRun(query)}>{isRune ? 'Rune' : 'Run'}</Button> <Button rightIconName={IconNames.CARET_RIGHT} onClick={() => onRun(query)}>{isRune ? 'Rune' : 'Run'}</Button>
</div> </div>
</div> </div>
} }

View File

@ -19,20 +19,9 @@
@import "./variables"; @import "./variables";
.console-application { .console-application {
position: relative;
height: 100%; height: 100%;
.bp3-navbar {
position: absolute;
top: 0;
left: 0;
right: 0;
}
.bp3-navbar-group > a {
text-decoration: inherit;
color: inherit;
}
.view-container { .view-container {
position: absolute; position: absolute;
top: 50px; top: 50px;
@ -44,6 +33,10 @@
&.scrollable { &.scrollable {
overflow-y: scroll; overflow-y: scroll;
} }
.app-view {
position: relative;
}
} }
.control-separator { .control-separator {
@ -66,9 +59,5 @@
font-size: 35px; font-size: 35px;
line-height: 35px; line-height: 35px;
} }
label.bp3-label {
margin-bottom: 7px;
}
} }
} }

View File

@ -26,10 +26,10 @@ import { HeaderBar, HeaderActiveTab } from './components/header-bar';
import { localStorageGet, localStorageSet } from './utils'; import { localStorageGet, localStorageSet } from './utils';
import { DRUID_DOCS_SQL, LEGACY_COORDINATOR_CONSOLE, LEGACY_OVERLORD_CONSOLE } from './variables'; import { DRUID_DOCS_SQL, LEGACY_COORDINATOR_CONSOLE, LEGACY_OVERLORD_CONSOLE } from './variables';
import { HomeView } from './views/home-view'; import { HomeView } from './views/home-view';
import { ServersView } from './views/servers-view';
import { DatasourcesView } from './views/datasource-view'; import { DatasourcesView } from './views/datasource-view';
import { TasksView } from './views/tasks-view';
import { SegmentsView } from './views/segments-view'; import { SegmentsView } from './views/segments-view';
import { ServersView } from './views/servers-view';
import { TasksView } from './views/tasks-view';
import { SqlView } from './views/sql-view'; import { SqlView } from './views/sql-view';
import "./console-application.scss"; import "./console-application.scss";
@ -59,7 +59,7 @@ export class ConsoleApplication extends React.Component<ConsoleApplicationProps,
// Status works but SQL 405s => the SQL endpoint is disabled // Status works but SQL 405s => the SQL endpoint is disabled
AppToaster.show({ AppToaster.show({
icon: 'error', iconName: 'error',
intent: Intent.DANGER, intent: Intent.DANGER,
timeout: 120000, timeout: 120000,
message: <> message: <>
@ -161,7 +161,7 @@ export class ConsoleApplication extends React.Component<ConsoleApplicationProps,
}} /> }} />
</Switch> </Switch>
</div> </div>
</HashRouter> </HashRouter>;
} }
} }

View File

@ -18,11 +18,10 @@
import * as React from 'react'; import * as React from 'react';
import { Button, Dialog, Classes, AnchorButton, Intent } from "@blueprintjs/core"; import { Button, Dialog, Classes, AnchorButton, Intent } from "@blueprintjs/core";
import { IconNames } from "@blueprintjs/icons"; import { IconNames } from "../components/filler";
import { DRUID_COMMUNITY, DRUID_DEVELOPER_GROUP, DRUID_USER_GROUP, DRUID_WEBSITE } from '../variables'; import { DRUID_COMMUNITY, DRUID_DEVELOPER_GROUP, DRUID_USER_GROUP, DRUID_WEBSITE } from '../variables';
export interface AboutDialogProps extends React.Props<any> { export interface AboutDialogProps extends React.Props<any> {
isOpen: boolean,
onClose: () => void onClose: () => void
} }
@ -36,15 +35,15 @@ export class AboutDialog extends React.Component<AboutDialogProps, AboutDialogSt
} }
render() { render() {
const { isOpen, onClose } = this.props; const { onClose } = this.props;
return <Dialog return <Dialog
icon={IconNames.INFO_SIGN} iconName={IconNames.GRAPH}
onClose={onClose} onClose={onClose}
title="Apache Druid" title="Apache Druid"
isOpen={isOpen} isOpen
usePortal={true} inline
canEscapeKeyClose={true} canEscapeKeyClose
> >
<div className={Classes.DIALOG_BODY}> <div className={Classes.DIALOG_BODY}>
<p> <p>

View File

@ -19,22 +19,14 @@
import classNames from 'classnames'; import classNames from 'classnames';
import * as React from 'react'; import * as React from 'react';
import { import {
FormGroup,
Button, Button,
InputGroup, InputGroup,
Dialog, Dialog,
NumericInput,
Classes, Classes,
Tooltip,
AnchorButton,
TagInput,
Intent, Intent,
ButtonGroup, ProgressBar
ProgressBar,
MaybeElement,
Icon,
IconName
} from "@blueprintjs/core"; } from "@blueprintjs/core";
import { Icon, FormGroup, ButtonGroup, NumericInput, TagInput } from '../components/filler';
import { AppToaster } from '../singletons/toaster'; import { AppToaster } from '../singletons/toaster';
export interface AsyncAlertDialogProps extends React.Props<any> { export interface AsyncAlertDialogProps extends React.Props<any> {
@ -43,7 +35,7 @@ export interface AsyncAlertDialogProps extends React.Props<any> {
confirmButtonText: string; confirmButtonText: string;
cancelButtonText?: string; cancelButtonText?: string;
className?: string, className?: string,
icon?: IconName | MaybeElement; icon?: string;
intent?: Intent; intent?: Intent;
successText: string; successText: string;
failText: string; failText: string;
@ -88,17 +80,19 @@ export class AsyncActionDialog extends React.Component<AsyncAlertDialogProps, As
render() { render() {
const { action, onClose, className, icon, intent, confirmButtonText, cancelButtonText, children } = this.props; const { action, onClose, className, icon, intent, confirmButtonText, cancelButtonText, children } = this.props;
const { working } = this.state; const { working } = this.state;
if (!action) return null;
const handleClose = () => onClose(false); const handleClose = () => onClose(false);
return <Dialog return <Dialog
isOpen={Boolean(action)} isOpen
inline
className={classNames(Classes.ALERT, 'async-alert-dialog', className)} className={classNames(Classes.ALERT, 'async-alert-dialog', className)}
canEscapeKeyClose={!working} canEscapeKeyClose={!working}
onClose={handleClose} onClose={handleClose}
> >
<div className={Classes.ALERT_BODY}> <div className={Classes.ALERT_BODY}>
{ icon && <Icon icon={icon} iconSize={40} intent={intent} /> } { icon && <Icon icon={icon} /> }
{ !working && <div className={Classes.ALERT_CONTENTS}>{children}</div> } { !working && <div className={Classes.ALERT_CONTENTS}>{children}</div> }
</div> </div>
{ {

View File

@ -19,15 +19,14 @@
import { Intent } from '@blueprintjs/core'; import { Intent } from '@blueprintjs/core';
import * as React from 'react'; import * as React from 'react';
import axios from 'axios'; import axios from 'axios';
import { IconNames } from "@blueprintjs/icons";
import { AppToaster } from '../singletons/toaster'; import { AppToaster } from '../singletons/toaster';
import { IconNames } from '../components/filler';
import { AutoForm } from '../components/auto-form'; import { AutoForm } from '../components/auto-form';
import { getDruidErrorMessage } from '../utils'; import { getDruidErrorMessage } from '../utils';
import { SnitchDialog } from './snitch-dialog'; import { SnitchDialog } from './snitch-dialog';
import './coordinator-dynamic-config.scss'; import './coordinator-dynamic-config.scss';
export interface CoordinatorDynamicConfigDialogProps extends React.Props<any> { export interface CoordinatorDynamicConfigDialogProps extends React.Props<any> {
isOpen: boolean,
onClose: () => void onClose: () => void
} }
@ -43,6 +42,10 @@ export class CoordinatorDynamicConfigDialog extends React.Component<CoordinatorD
} }
} }
componentDidMount(): void {
this.getClusterConfig();
}
async getClusterConfig() { async getClusterConfig() {
let config: Record<string, any> | null = null; let config: Record<string, any> | null = null;
try { try {
@ -50,7 +53,7 @@ export class CoordinatorDynamicConfigDialog extends React.Component<CoordinatorD
config = configResp.data config = configResp.data
} catch (e) { } catch (e) {
AppToaster.show({ AppToaster.show({
icon: IconNames.ERROR, iconName: IconNames.ERROR,
intent: Intent.DANGER, intent: Intent.DANGER,
message: `Could not load coordinator dynamic config: ${getDruidErrorMessage(e)}` message: `Could not load coordinator dynamic config: ${getDruidErrorMessage(e)}`
}); });
@ -73,7 +76,7 @@ export class CoordinatorDynamicConfigDialog extends React.Component<CoordinatorD
}); });
} catch (e) { } catch (e) {
AppToaster.show({ AppToaster.show({
icon: IconNames.ERROR, iconName: IconNames.ERROR,
intent: Intent.DANGER, intent: Intent.DANGER,
message: `Could not save coordinator dynamic config: ${getDruidErrorMessage(e)}` message: `Could not save coordinator dynamic config: ${getDruidErrorMessage(e)}`
}); });
@ -87,15 +90,14 @@ export class CoordinatorDynamicConfigDialog extends React.Component<CoordinatorD
} }
render() { render() {
const { isOpen, onClose } = this.props; const { onClose } = this.props;
const { dynamicConfig } = this.state; const { dynamicConfig } = this.state;
return <SnitchDialog return <SnitchDialog
className="coordinator-dynamic-config" className="coordinator-dynamic-config"
isOpen={ isOpen } isOpen
onSave={this.saveClusterConfig} onSave={this.saveClusterConfig}
onOpening={() => {this.getClusterConfig()}} onClose={onClose}
onClose={ onClose }
title="Coordinator dynamic config" title="Coordinator dynamic config"
> >
<p> <p>

View File

@ -16,6 +16,9 @@
* limitations under the License. * limitations under the License.
*/ */
import * as React from 'react';
(React as any).PropTypes = require('prop-types'); // Trick blueprint 1.0.1 into accepting React 16 as React 15.
import { reorderArray } from './retention-dialog'; import { reorderArray } from './retention-dialog';
describe('reorderArray', () => { describe('reorderArray', () => {

View File

@ -18,11 +18,8 @@
import * as React from 'react'; import * as React from 'react';
import axios from 'axios'; import axios from 'axios';
import { import { Button } from "@blueprintjs/core";
FormGroup, import { FormGroup, IconNames } from '../components/filler';
Button,
} from "@blueprintjs/core";
import { IconNames } from "@blueprintjs/icons";
import { RuleEditor, Rule } from '../components/rule-editor'; import { RuleEditor, Rule } from '../components/rule-editor';
import { SnitchDialog } from './snitch-dialog'; import { SnitchDialog } from './snitch-dialog';
@ -148,6 +145,7 @@ export class RetentionDialog extends React.Component<RetentionDialogProps, Reten
saveDisabled={false} saveDisabled={false}
canOutsideClickClose={false} canOutsideClickClose={false}
isOpen isOpen
inline
onClose={onCancel} onClose={onCancel}
title={`Edit retention rules: ${datasource}${datasource === '_default' ? ' (cluster defaults)' : ''}`} title={`Edit retention rules: ${datasource}${datasource === '_default' ? ' (cluster defaults)' : ''}`}
onReset={this.reset} onReset={this.reset}
@ -162,7 +160,7 @@ export class RetentionDialog extends React.Component<RetentionDialogProps, Reten
{(currentRules || []).map(this.renderRule)} {(currentRules || []).map(this.renderRule)}
</FormGroup> </FormGroup>
<FormGroup className="right"> <FormGroup className="right">
<Button icon={IconNames.PLUS} onClick={this.addRule}>New rule</Button> <Button iconName={IconNames.PLUS} onClick={this.addRule}>New rule</Button>
</FormGroup> </FormGroup>
{ {
(!currentRules.length && datasource !== '_default') && (!currentRules.length && datasource !== '_default') &&

View File

@ -18,7 +18,6 @@
import * as React from 'react'; import * as React from 'react';
import { import {
FormGroup,
Button, Button,
InputGroup, InputGroup,
Dialog, Dialog,
@ -26,8 +25,8 @@ import {
Classes, Classes,
Intent, Intent,
} from "@blueprintjs/core"; } from "@blueprintjs/core";
import { IconNames, FormGroup } from '../components/filler';
import { IconNames } from "@blueprintjs/icons";
export interface SnitchDialogProps extends IDialogProps { export interface SnitchDialogProps extends IDialogProps {
onSave: (author: string, comment: string) => void; onSave: (author: string, comment: string) => void;
@ -107,12 +106,12 @@ export class SnitchDialog extends React.Component<SnitchDialogProps, SnitchDialo
<FormGroup label={"Who is making this change?"}> <FormGroup label={"Who is making this change?"}>
<InputGroup value={author} onChange={(e: any) => this.changeAuthor(e.target.value)}/> <InputGroup value={author} onChange={(e: any) => this.changeAuthor(e.target.value)}/>
</FormGroup> </FormGroup>
<FormGroup className={"comment"}> <FormGroup label={"Why are you making this change?"} className={"comment"}>
<InputGroup <InputGroup
className="pt-large"
value={comment} value={comment}
placeholder={"Why are you making this change?"} placeholder={"Enter description here"}
onChange={(e: any) => this.changeComment(e.target.value)} onChange={(e: any) => this.changeComment(e.target.value)}
large={true}
/> />
</FormGroup> </FormGroup>
</div> </div>
@ -128,38 +127,25 @@ export class SnitchDialog extends React.Component<SnitchDialogProps, SnitchDialo
const { showFinalStep } = this.state; const { showFinalStep } = this.state;
return <div className={Classes.DIALOG_FOOTER_ACTIONS}> return <div className={Classes.DIALOG_FOOTER_ACTIONS}>
<FormGroup> { showFinalStep
{ showFinalStep ? <Button onClick={this.back} iconName={IconNames.ARROW_LEFT}>Back</Button>
? <Button onClick={this.back} icon={IconNames.ARROW_LEFT}>Back</Button> : onReset ? <Button onClick={this.reset} intent={"none" as any}>Reset</Button> : null
: onReset ? <Button onClick={this.reset} intent={"none"}>Reset</Button> : null }
}
{ showFinalStep { showFinalStep
? <Button disabled={saveDisabled} text="Save" onClick={this.save} intent={Intent.PRIMARY} rightIcon={IconNames.TICK}/> ? <Button disabled={saveDisabled} text="Save" onClick={this.save} intent={Intent.PRIMARY as any} rightIconName={IconNames.TICK}/>
: <Button disabled={saveDisabled} text="Next" onClick={this.goToFinalStep} intent={Intent.PRIMARY} rightIcon={IconNames.ARROW_RIGHT}/> : <Button disabled={saveDisabled} text="Next" onClick={this.goToFinalStep} intent={Intent.PRIMARY as any} rightIconName={IconNames.ARROW_RIGHT}/>
} }
</FormGroup>
</div> </div>
} }
onOpening = (node: HTMLElement) => {
const { onOpening } = this.props;
this.setState({
author: '',
comment: ''
});
onOpening && onOpening(node);
}
render() { render() {
const { isOpen, onClose, className, children, saveDisabled } = this.props; const { onClose, className, children, saveDisabled } = this.props;
const { showFinalStep } = this.state; const { showFinalStep } = this.state;
if (showFinalStep) return this.renderFinalStep(); if (showFinalStep) return this.renderFinalStep();
return <Dialog {...this.props} onOpening={this.onOpening}> return <Dialog isOpen inline {...this.props}>
<div className={Classes.DIALOG_BODY}> <div className={Classes.DIALOG_BODY}>
{children} {children}
</div> </div>

View File

@ -18,6 +18,11 @@
.post-spec-dialog { .post-spec-dialog {
.post-spec-dialog-textarea { .post-spec-dialog-textarea {
margin: 20px; background-color: #232C35;
margin-bottom: 10px;
.ace_gutter {
background-color: #232C35;
}
} }
} }

View File

@ -17,13 +17,13 @@
*/ */
import * as React from "react"; import * as React from "react";
import axios from 'axios'; import { Button, Classes, Dialog, Intent } from "@blueprintjs/core";
import {Button, Classes, Dialog, Intent, EditableText} from "@blueprintjs/core";
import "./spec-dialog.scss" import "./spec-dialog.scss"
import {QueryManager} from "../utils"; import AceEditor from "react-ace";
import "brace/theme/solarized_dark";
import "brace/mode/json"
export interface SpecDialogProps extends React.Props<any> { export interface SpecDialogProps extends React.Props<any> {
isOpen: boolean;
onSubmit: (spec: JSON) => void; onSubmit: (spec: JSON) => void;
onClose: () => void; onClose: () => void;
title: string; title: string;
@ -59,22 +59,33 @@ export class SpecDialog extends React.Component<SpecDialogProps, SpecDialogState
} }
render() { render() {
const { isOpen, onClose, title } = this.props; const { onClose, title } = this.props;
const { spec } = this.state; const { spec } = this.state;
return <Dialog return <Dialog
className={"post-spec-dialog"} className={"post-spec-dialog"}
isOpen={isOpen} isOpen
onClose={onClose} onClose={onClose}
title={title} title={title}
> >
<EditableText <AceEditor
mode="json"
theme="solarized_dark"
className={"post-spec-dialog-textarea"} className={"post-spec-dialog-textarea"}
multiline={true} onChange={ (e) => {this.setState({ spec: e })} }
minLines={30} fontSize={12}
maxLines={30} showPrintMargin={false}
placeholder={"Enter the spec JSON to post"} showGutter={true}
onChange={ (e) => {this.setState({ spec: e })}} highlightActiveLine={true}
value={spec}
width={"100%"}
setOptions={{
enableBasicAutocompletion: true,
enableLiveAutocompletion: true,
showLineNumbers: true,
enableSnippets: true,
tabSize: 2,
}}
/> />
<div className={Classes.DIALOG_FOOTER}> <div className={Classes.DIALOG_FOOTER}>
<div className={Classes.DIALOG_FOOTER_ACTIONS}> <div className={Classes.DIALOG_FOOTER_ACTIONS}>

View File

@ -17,13 +17,9 @@
*/ */
@import '../node_modules/normalize.css/normalize'; @import '../node_modules/normalize.css/normalize';
@import '../node_modules/@blueprintjs/core/lib/css/blueprint'; @import '../lib/blueprint';
@import '../lib/react-table'; @import '../lib/react-table';
* {
position: relative;
}
html, html,
body { body {
//font-family: 'Open Sans', Helvetica, Arial, sans-serif; //font-family: 'Open Sans', Helvetica, Arial, sans-serif;
@ -33,7 +29,7 @@ body {
} }
body { body {
&.bp3-dark { &.pt-dark {
background: rgb(41, 55, 66); background: rgb(41, 55, 66);
} }

View File

@ -19,6 +19,7 @@
import 'es6-shim/es6-shim'; import 'es6-shim/es6-shim';
import 'es7-shim'; // Webpack with automatically pick browser.js which does the shim() import 'es7-shim'; // Webpack with automatically pick browser.js which does the shim()
import * as React from 'react'; import * as React from 'react';
(React as any).PropTypes = require('prop-types'); // Trick blueprint 1.0.1 into accepting React 16 as React 15.
import * as ReactDOM from 'react-dom'; import * as ReactDOM from 'react-dom';
import "./singletons/react-table-defaults"; import "./singletons/react-table-defaults";
import "./entry.scss"; import "./entry.scss";
@ -34,7 +35,7 @@ ReactDOM.render(
{ {
version: '0.0.1' version: '0.0.1'
} }
), ) as any,
container container
); );

View File

@ -25,7 +25,7 @@ import { countBy, makeTextFilter } from '../utils';
class FullButton extends React.Component { class FullButton extends React.Component {
render() { render() {
return <Button fill={true} {...this.props}/>; return <Button className="pt-fill" {...this.props}/>;
} }
} }

View File

@ -16,8 +16,8 @@
* limitations under the License. * limitations under the License.
*/ */
import { Button, InputGroup, Intent, HTMLSelect } from '@blueprintjs/core'; import { Button, InputGroup, Intent } from '@blueprintjs/core';
import { IconNames } from "@blueprintjs/icons"; import { IconNames, HTMLSelect } from "../components/filler";
import * as numeral from "numeral"; import * as numeral from "numeral";
import * as React from 'react'; import * as React from 'react';
import { Filter, FilterRender } from 'react-table'; import { Filter, FilterRender } from 'react-table';
@ -43,7 +43,7 @@ export function makeTextFilter(placeholder: string = ''): FilterRender {
key={key} key={key}
onChange={(e: any) => onChange(e.target.value)} onChange={(e: any) => onChange(e.target.value)}
value={filterValue} value={filterValue}
rightElement={filterValue ? <Button icon={IconNames.CROSS} intent={Intent.NONE} minimal={true} onClick={() => onChange('')} /> : undefined} rightElement={filterValue ? <Button iconName={IconNames.CROSS} className="pt-minimal" onClick={() => onChange('')} /> : undefined}
placeholder={placeholder} placeholder={placeholder}
/> />
} }
@ -55,7 +55,7 @@ export function makeBooleanFilter(): FilterRender {
return <HTMLSelect return <HTMLSelect
key={key} key={key}
style={{ width: '100%' }} style={{ width: '100%' }}
onChange={event => onChange(event.target.value)} onChange={(event: any) => onChange(event.target.value)}
value={filterValue || "all"} value={filterValue || "all"}
fill={true} fill={true}
> >

View File

@ -22,7 +22,7 @@
.ReactTable { .ReactTable {
position: absolute; position: absolute;
top: 60px; top: 50px;
bottom: 0; bottom: 0;
width: 100%; width: 100%;
} }

View File

@ -22,10 +22,11 @@ import * as classNames from 'classnames';
import ReactTable from "react-table"; import ReactTable from "react-table";
import { Filter } from "react-table"; import { Filter } from "react-table";
import { Button, Intent, Switch } from "@blueprintjs/core"; import { Button, Intent, Switch } from "@blueprintjs/core";
import { IconNames } from "@blueprintjs/icons"; import { IconNames } from "../components/filler";
import { AppToaster } from '../singletons/toaster'; import { AppToaster } from '../singletons/toaster';
import { RuleEditor } from '../components/rule-editor'; import { RuleEditor } from '../components/rule-editor';
import { AsyncActionDialog } from '../dialogs/async-action-dialog'; import { AsyncActionDialog } from '../dialogs/async-action-dialog';
import { RetentionDialog } from '../dialogs/retention-dialog';
import { import {
addFilter, addFilter,
formatNumber, formatNumber,
@ -35,7 +36,6 @@ import {
QueryManager, QueryManager,
pluralIfNeeded, queryDruidSql, getDruidErrorMessage pluralIfNeeded, queryDruidSql, getDruidErrorMessage
} from "../utils"; } from "../utils";
import { RetentionDialog } from '../dialogs/retention-dialog';
import "./datasource-view.scss"; import "./datasource-view.scss";
@ -439,12 +439,12 @@ GROUP BY 1`);
<div className="control-bar"> <div className="control-bar">
<div className="control-label">Datasources</div> <div className="control-label">Datasources</div>
<Button <Button
icon={IconNames.REFRESH} iconName={IconNames.REFRESH}
text="Refresh" text="Refresh"
onClick={() => this.datasourceQueryManager.rerunLastQuery()} onClick={() => this.datasourceQueryManager.rerunLastQuery()}
/> />
<Button <Button
icon={IconNames.CONSOLE} iconName={IconNames.APPLICATION}
text="Go to SQL" text="Go to SQL"
onClick={() => goToSql(this.datasourceQueryManager.getLastQuery())} onClick={() => goToSql(this.datasourceQueryManager.getLastQuery())}
/> />

View File

@ -28,7 +28,7 @@
color: inherit; color: inherit;
} }
.bp3-card { .pt-card {
height: 160px; height: 160px;
} }
} }

View File

@ -19,14 +19,13 @@
import axios from 'axios'; import axios from 'axios';
import * as React from 'react'; import * as React from 'react';
import * as classNames from 'classnames'; import * as classNames from 'classnames';
import { H5, Card, Icon } from "@blueprintjs/core"; import { H5, Card, Icon, IconNames } from "../components/filler";
import { IconName, IconNames } from "@blueprintjs/icons";
import { QueryManager, pluralIfNeeded, queryDruidSql, getHeadProp } from '../utils'; import { QueryManager, pluralIfNeeded, queryDruidSql, getHeadProp } from '../utils';
import './home-view.scss'; import './home-view.scss';
export interface CardOptions { export interface CardOptions {
href: string; href: string;
icon: IconName; icon: string;
title: string; title: string;
loading?: boolean; loading?: boolean;
content: JSX.Element | string; content: JSX.Element | string;
@ -268,7 +267,7 @@ GROUP BY 1`);
return <div className="home-view app-view"> return <div className="home-view app-view">
{this.renderCard({ {this.renderCard({
href: "/status", href: "/status",
icon: IconNames.INFO_SIGN, icon: IconNames.GRAPH,
title: "Status", title: "Status",
loading: state.statusLoading, loading: state.statusLoading,
content: state.status ? `Apache Druid is running version ${state.status.version}` : '', content: state.status ? `Apache Druid is running version ${state.status.version}` : '',
@ -286,7 +285,7 @@ GROUP BY 1`);
{this.renderCard({ {this.renderCard({
href: "#segments", href: "#segments",
icon: IconNames.FULL_STACKED_CHART, icon: IconNames.STACKED_CHART,
title: "Segments", title: "Segments",
loading: state.segmentCountLoading, loading: state.segmentCountLoading,
content: pluralIfNeeded(state.segmentCount, 'segment'), content: pluralIfNeeded(state.segmentCount, 'segment'),

View File

@ -22,7 +22,7 @@
.ReactTable { .ReactTable {
position: absolute; position: absolute;
top: 60px; top: 50px;
bottom: 0; bottom: 0;
width: 100%; width: 100%;

View File

@ -21,8 +21,8 @@ import * as React from 'react';
import * as classNames from 'classnames'; import * as classNames from 'classnames';
import ReactTable from "react-table"; import ReactTable from "react-table";
import { Filter } from "react-table"; import { Filter } from "react-table";
import { H5, Button } from "@blueprintjs/core"; import { Button } from "@blueprintjs/core";
import { IconNames } from "@blueprintjs/icons"; import { H5, IconNames } from "../components/filler";
import { import {
addFilter, addFilter,
makeBooleanFilter, makeBooleanFilter,
@ -261,12 +261,12 @@ export class SegmentsView extends React.Component<SegmentsViewProps, SegmentsVie
<div className="control-bar"> <div className="control-bar">
<div className="control-label">Segments</div> <div className="control-label">Segments</div>
<Button <Button
icon={IconNames.REFRESH} iconName={IconNames.REFRESH}
text="Refresh" text="Refresh"
onClick={() => this.segmentsQueryManager.rerunLastQuery()} onClick={() => this.segmentsQueryManager.rerunLastQuery()}
/> />
<Button <Button
icon={IconNames.CONSOLE} iconName={IconNames.APPLICATION}
text="Go to SQL" text="Go to SQL"
onClick={() => goToSql(this.segmentsQueryManager.getLastQuery().query)} onClick={() => goToSql(this.segmentsQueryManager.getLastQuery().query)}
/> />

View File

@ -31,6 +31,7 @@
} }
.fill-indicator { .fill-indicator {
position: relative;
width: 100%; width: 100%;
height: 100%; height: 100%;
background-color: #dadada; background-color: #dadada;

View File

@ -22,8 +22,8 @@ import * as classNames from 'classnames';
import ReactTable from "react-table"; import ReactTable from "react-table";
import { Filter } from "react-table"; import { Filter } from "react-table";
import { sum } from "d3-array"; import { sum } from "d3-array";
import { Button, H1, Switch } from "@blueprintjs/core"; import { Button, Switch } from "@blueprintjs/core";
import { IconNames } from "@blueprintjs/icons"; import { IconNames } from '../components/filler';
import { addFilter, formatBytes, formatBytesCompact, QueryManager, queryDruidSql } from "../utils"; import { addFilter, formatBytes, formatBytesCompact, QueryManager, queryDruidSql } from "../utils";
import "./servers-view.scss"; import "./servers-view.scss";
@ -334,12 +334,12 @@ WHERE "server_type" = 'historical'`);
<div className="control-bar"> <div className="control-bar">
<div className="control-label">Historicals</div> <div className="control-label">Historicals</div>
<Button <Button
icon={IconNames.REFRESH} iconName={IconNames.REFRESH}
text="Refresh" text="Refresh"
onClick={() => this.serverQueryManager.rerunLastQuery()} onClick={() => this.serverQueryManager.rerunLastQuery()}
/> />
<Button <Button
icon={IconNames.CONSOLE} iconName={IconNames.APPLICATION}
text="Go to SQL" text="Go to SQL"
onClick={() => goToSql(this.serverQueryManager.getLastQuery())} onClick={() => goToSql(this.serverQueryManager.getLastQuery())}
/> />
@ -356,7 +356,7 @@ WHERE "server_type" = 'historical'`);
<div className="control-bar"> <div className="control-bar">
<div className="control-label">MiddleManagers</div> <div className="control-label">MiddleManagers</div>
<Button <Button
icon={IconNames.REFRESH} iconName={IconNames.REFRESH}
text="Refresh" text="Refresh"
onClick={() => this.middleManagerQueryManager.rerunLastQuery()} onClick={() => this.middleManagerQueryManager.rerunLastQuery()}
/> />

View File

@ -21,8 +21,8 @@ import * as React from 'react';
import * as classNames from 'classnames'; import * as classNames from 'classnames';
import ReactTable from "react-table"; import ReactTable from "react-table";
import { Filter } from "react-table"; import { Filter } from "react-table";
import { Button, H1, ButtonGroup, Intent, Label, Alert } from "@blueprintjs/core"; import { Button, Intent, Alert } from "@blueprintjs/core";
import { IconNames } from "@blueprintjs/icons"; import { ButtonGroup, Label, IconNames } from "../components/filler";
import { addFilter, QueryManager, getDruidErrorMessage, countBy, formatDuration, queryDruidSql } from "../utils"; import { addFilter, QueryManager, getDruidErrorMessage, countBy, formatDuration, queryDruidSql } from "../utils";
import { AsyncActionDialog } from "../dialogs/async-action-dialog"; import { AsyncActionDialog } from "../dialogs/async-action-dialog";
import { SpecDialog } from "../dialogs/spec-dialog"; import { SpecDialog } from "../dialogs/spec-dialog";
@ -524,12 +524,12 @@ ORDER BY "rank" DESC, "created_time" DESC`);
<div className="control-bar"> <div className="control-bar">
<div className="control-label">Supervisors</div> <div className="control-label">Supervisors</div>
<Button <Button
icon={IconNames.REFRESH} iconName={IconNames.REFRESH}
text="Refresh" text="Refresh"
onClick={() => this.supervisorQueryManager.rerunLastQuery()} onClick={() => this.supervisorQueryManager.rerunLastQuery()}
/> />
<Button <Button
icon={IconNames.PLUS} iconName={IconNames.PLUS}
text="Submit supervisor" text="Submit supervisor"
onClick={() => this.setState({ supervisorSpecDialogOpen: true })} onClick={() => this.setState({ supervisorSpecDialogOpen: true })}
/> />
@ -542,43 +542,40 @@ ORDER BY "rank" DESC, "created_time" DESC`);
<div className="control-label">Tasks</div> <div className="control-label">Tasks</div>
<Label>Group by</Label> <Label>Group by</Label>
<ButtonGroup> <ButtonGroup>
<Button active={groupTasksBy === null} onClick={() => this.setState({ groupTasksBy: null })}>None</Button> <Button className={classNames({ 'pt-active': groupTasksBy === null })} onClick={() => this.setState({ groupTasksBy: null })}>None</Button>
<Button active={groupTasksBy === 'type'} onClick={() => this.setState({ groupTasksBy: 'type' })}>Type</Button> <Button className={classNames({ 'pt-active': groupTasksBy === 'type' })} onClick={() => this.setState({ groupTasksBy: 'type' })}>Type</Button>
<Button active={groupTasksBy === 'datasource'} onClick={() => this.setState({ groupTasksBy: 'datasource' })}>Datasource</Button> <Button className={classNames({ 'pt-active': groupTasksBy === 'datasource' })} onClick={() => this.setState({ groupTasksBy: 'datasource' })}>Datasource</Button>
<Button active={groupTasksBy === 'status'} onClick={() => this.setState({ groupTasksBy: 'status' })}>Status</Button> <Button className={classNames({ 'pt-active': groupTasksBy === 'status' })} onClick={() => this.setState({ groupTasksBy: 'status' })}>Status</Button>
</ButtonGroup> </ButtonGroup>
<Button <Button
icon={IconNames.REFRESH} iconName={IconNames.REFRESH}
text="Refresh" text="Refresh"
onClick={() => this.taskQueryManager.rerunLastQuery()} onClick={() => this.taskQueryManager.rerunLastQuery()}
/> />
<Button <Button
icon={IconNames.CONSOLE} iconName={IconNames.APPLICATION}
text="Go to SQL" text="Go to SQL"
onClick={() => goToSql(this.taskQueryManager.getLastQuery())} onClick={() => goToSql(this.taskQueryManager.getLastQuery())}
/> />
<Button <Button
icon={IconNames.PLUS} iconName={IconNames.PLUS}
text="Submit task" text="Submit task"
onClick={() => this.setState({ taskSpecDialogOpen: true })} onClick={() => this.setState({ taskSpecDialogOpen: true })}
/> />
</div> </div>
{this.renderTaskTable()} {this.renderTaskTable()}
{ supervisorSpecDialogOpen ? <SpecDialog
<SpecDialog
isOpen={ supervisorSpecDialogOpen }
onClose={() => this.setState({ supervisorSpecDialogOpen: false })} onClose={() => this.setState({ supervisorSpecDialogOpen: false })}
onSubmit={this.submitSupervisor} onSubmit={this.submitSupervisor}
title="Submit supervisor" title="Submit supervisor"
/> /> : null }
<SpecDialog { taskSpecDialogOpen ? <SpecDialog
isOpen={ taskSpecDialogOpen }
onClose={() => this.setState({ taskSpecDialogOpen: false })} onClose={() => this.setState({ taskSpecDialogOpen: false })}
onSubmit={this.submitTask} onSubmit={this.submitTask}
title="Submit task" title="Submit task"
/> /> : null }
<Alert <Alert
icon={IconNames.ERROR} iconName={IconNames.ERROR}
intent={Intent.PRIMARY} intent={Intent.PRIMARY}
isOpen={Boolean(alertErrorMsg)} isOpen={Boolean(alertErrorMsg)}
confirmButtonText="OK" confirmButtonText="OK"

View File

@ -21,8 +21,8 @@
}, },
"include": [ "include": [
"**/*.ts", "src/**/*.ts",
"**/*.tsx" "src/**/*.tsx"
], ],
"exclude": [ "exclude": [
"**/*.test.ts" "**/*.test.ts"

View File

@ -24,7 +24,7 @@
<meta name="description" content="Apache Druid web console"> <meta name="description" content="Apache Druid web console">
<link rel="shortcut icon" href="/favicon.png"> <link rel="shortcut icon" href="/favicon.png">
</head> </head>
<body class="bp3-dark mouse-mode"> <body class="pt-dark mouse-mode">
<div class="app-container"></div> <div class="app-container"></div>
<script src="public/web-console-0.14.0.js"></script> <script src="public/web-console-0.14.0.js"></script>
</body> </body>