mirror of https://github.com/apache/druid.git
Merge branch 'master' into 6088-Time-Ordering-On-Scans-N-Way-Merge
This commit is contained in:
commit
5bd0e1a32c
|
@ -291,5 +291,10 @@ codestyle/checkstyle.xml. "/>
|
|||
<property name="illegalPattern" value="true"/>
|
||||
<property name="message" value="Duplicate line"/>
|
||||
</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>
|
||||
|
|
|
@ -107,7 +107,17 @@ public final class DateTimes
|
|||
|
||||
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(
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
}
|
||||
}
|
|
@ -37,4 +37,23 @@ public class DateTimesTest
|
|||
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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -274,7 +274,7 @@ public class HllSketchSqlAggregatorTest extends CalciteTestBase
|
|||
null,
|
||||
null
|
||||
),
|
||||
BaseCalciteQueryTest.NOT(BaseCalciteQueryTest.SELECTOR("dim2", "", null))
|
||||
BaseCalciteQueryTest.not(BaseCalciteQueryTest.selector("dim2", "", null))
|
||||
),
|
||||
new HllSketchBuildAggregatorFactory(
|
||||
"a3",
|
||||
|
|
|
@ -342,7 +342,7 @@ public class DoublesSketchAggregatorTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void QueryingDataWithFieldNameValueAsFloatInsteadOfSketch() throws Exception
|
||||
public void queryingDataWithFieldNameValueAsFloatInsteadOfSketch() throws Exception
|
||||
{
|
||||
Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
|
||||
new File(this.getClass().getClassLoader().getResource("quantiles/doubles_build_data.tsv").getFile()),
|
||||
|
@ -418,7 +418,7 @@ public class DoublesSketchAggregatorTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void TimeSeriesQueryInputAsFloat() throws Exception
|
||||
public void timeSeriesQueryInputAsFloat() throws Exception
|
||||
{
|
||||
Sequence<Row> seq = timeSeriesHelper.createIndexAndRunQueryOnSegment(
|
||||
new File(this.getClass().getClassLoader().getResource("quantiles/doubles_build_data.tsv").getFile()),
|
||||
|
|
|
@ -278,7 +278,7 @@ public class ThetaSketchSqlAggregatorTest extends CalciteTestBase
|
|||
null,
|
||||
null
|
||||
),
|
||||
BaseCalciteQueryTest.NOT(BaseCalciteQueryTest.SELECTOR("dim2", "", null))
|
||||
BaseCalciteQueryTest.not(BaseCalciteQueryTest.selector("dim2", "", null))
|
||||
),
|
||||
new SketchMergeAggregatorFactory(
|
||||
"a3",
|
||||
|
|
|
@ -114,12 +114,12 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
|
|||
ImmutableList.of(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(QSS(Filtration.eternity()))
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.granularity(Granularities.ALL)
|
||||
.filters(
|
||||
new BloomDimFilter("dim1", BloomKFilterHolder.fromBloomKFilter(filter), null)
|
||||
)
|
||||
.aggregators(AGGS(new CountAggregatorFactory("a0")))
|
||||
.aggregators(aggregators(new CountAggregatorFactory("a0")))
|
||||
.context(TIMESERIES_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
|
@ -146,7 +146,7 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
|
|||
ImmutableList.of(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(QSS(Filtration.eternity()))
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.granularity(Granularities.ALL)
|
||||
.virtualColumns()
|
||||
.filters(
|
||||
|
@ -155,7 +155,7 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
|
|||
createExprMacroTable()
|
||||
)
|
||||
)
|
||||
.aggregators(AGGS(new CountAggregatorFactory("a0")))
|
||||
.aggregators(aggregators(new CountAggregatorFactory("a0")))
|
||||
.context(TIMESERIES_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
|
@ -178,7 +178,7 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
|
|||
ImmutableList.of(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(QSS(Filtration.eternity()))
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.granularity(Granularities.ALL)
|
||||
.virtualColumns()
|
||||
.filters(
|
||||
|
@ -187,7 +187,7 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
|
|||
createExprMacroTable()
|
||||
)
|
||||
)
|
||||
.aggregators(AGGS(new CountAggregatorFactory("a0")))
|
||||
.aggregators(aggregators(new CountAggregatorFactory("a0")))
|
||||
.context(TIMESERIES_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
|
@ -214,7 +214,7 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
|
|||
ImmutableList.of(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(QSS(Filtration.eternity()))
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.granularity(Granularities.ALL)
|
||||
.filters(
|
||||
new OrDimFilter(
|
||||
|
@ -222,7 +222,7 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
|
|||
new BloomDimFilter("dim2", BloomKFilterHolder.fromBloomKFilter(filter2), null)
|
||||
)
|
||||
)
|
||||
.aggregators(AGGS(new CountAggregatorFactory("a0")))
|
||||
.aggregators(aggregators(new CountAggregatorFactory("a0")))
|
||||
.context(TIMESERIES_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
|
|
|
@ -28,10 +28,10 @@ import java.util.Map;
|
|||
|
||||
public class KafkaDataSourceMetadataTest
|
||||
{
|
||||
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 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 KM0 = km("foo", ImmutableMap.of());
|
||||
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 KM3 = km("foo", ImmutableMap.of(0, 2L, 2, 5L));
|
||||
|
||||
@Test
|
||||
public void testMatches()
|
||||
|
@ -70,27 +70,27 @@ public class KafkaDataSourceMetadataTest
|
|||
public void testPlus()
|
||||
{
|
||||
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)
|
||||
);
|
||||
|
||||
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)
|
||||
);
|
||||
|
||||
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)
|
||||
);
|
||||
|
||||
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)
|
||||
);
|
||||
|
||||
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)
|
||||
);
|
||||
}
|
||||
|
@ -99,32 +99,32 @@ public class KafkaDataSourceMetadataTest
|
|||
public void testMinus()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
KM("foo", ImmutableMap.of(1, 3L)),
|
||||
km("foo", ImmutableMap.of(1, 3L)),
|
||||
KM1.minus(KM3)
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
KM("foo", ImmutableMap.of()),
|
||||
km("foo", ImmutableMap.of()),
|
||||
KM0.minus(KM2)
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
KM("foo", ImmutableMap.of()),
|
||||
km("foo", ImmutableMap.of()),
|
||||
KM1.minus(KM2)
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
KM("foo", ImmutableMap.of(2, 5L)),
|
||||
km("foo", ImmutableMap.of(2, 5L)),
|
||||
KM2.minus(KM1)
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
KM("foo", ImmutableMap.of()),
|
||||
km("foo", ImmutableMap.of()),
|
||||
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));
|
||||
}
|
||||
|
|
|
@ -139,7 +139,6 @@ import org.apache.druid.segment.loading.DataSegmentPusher;
|
|||
import org.apache.druid.segment.loading.LocalDataSegmentPusher;
|
||||
import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig;
|
||||
import org.apache.druid.segment.loading.SegmentLoaderConfig;
|
||||
import org.apache.druid.segment.loading.SegmentLoaderLocalCacheManager;
|
||||
import org.apache.druid.segment.loading.StorageLocationConfig;
|
||||
import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl;
|
||||
import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier;
|
||||
|
@ -278,21 +277,21 @@ public class KafkaIndexTaskTest
|
|||
private static List<ProducerRecord<byte[], byte[]>> generateRecords(String topic)
|
||||
{
|
||||
return ImmutableList.of(
|
||||
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("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", "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("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("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", "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, StringUtils.toUtf8("unparseable")),
|
||||
new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable2")),
|
||||
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("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", "20.0", "notanumber")),
|
||||
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, 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", "10", "notanumber", "1.0")),
|
||||
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("2011", "h", "y", "10", "20.0", "1.0"))
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -412,8 +411,8 @@ public class KafkaIndexTaskTest
|
|||
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
|
||||
|
@ -462,8 +461,8 @@ public class KafkaIndexTaskTest
|
|||
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
|
||||
|
@ -571,13 +570,13 @@ public class KafkaIndexTaskTest
|
|||
Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2008/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2009/P1D", 0);
|
||||
SegmentDescriptor desc3 = SD(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc4 = SD(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc5 = SD(task, "2011/P1D", 1);
|
||||
SegmentDescriptor desc6 = SD(task, "2012/P1D", 0);
|
||||
SegmentDescriptor desc7 = SD(task, "2013/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
|
||||
SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc4 = sd(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc5 = sd(task, "2011/P1D", 1);
|
||||
SegmentDescriptor desc6 = sd(task, "2012/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))),
|
||||
|
@ -697,6 +696,7 @@ public class KafkaIndexTaskTest
|
|||
}
|
||||
final Map<Integer, Long> nextOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets());
|
||||
|
||||
|
||||
Assert.assertTrue(checkpoint2.getPartitionSequenceNumberMap().equals(nextOffsets));
|
||||
task.getRunner().setEndOffsets(nextOffsets, false);
|
||||
|
||||
|
@ -728,15 +728,20 @@ public class KafkaIndexTaskTest
|
|||
Assert.assertEquals(8, task.getRunner().getRowIngestionMeters().getProcessed());
|
||||
Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable());
|
||||
Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2008/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2009/P1D", 0);
|
||||
SegmentDescriptor desc3 = SD(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc4 = SD(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc5 = SD(task, "2011/P1D", 1);
|
||||
SegmentDescriptor desc6 = SD(task, "2012/P1D", 0);
|
||||
SegmentDescriptor desc7 = SD(task, "2013/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
|
||||
SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc4 = sd(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc5 = sd(task, "2011/P1D", 1);
|
||||
SegmentDescriptor desc6 = sd(task, "2012/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(
|
||||
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());
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2008/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2009/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
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(
|
||||
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("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("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("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("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("2012", "e", "y", "10", "20.0", "1.0")),
|
||||
new ProducerRecord<>(topic, 0, null, jb("2009", "B", "y", "10", "20.0", "1.0"))
|
||||
);
|
||||
|
||||
final String baseSequenceName = "sequence0";
|
||||
|
@ -974,8 +979,8 @@ public class KafkaIndexTaskTest
|
|||
Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
|
||||
|
@ -1024,9 +1029,9 @@ public class KafkaIndexTaskTest
|
|||
Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2008/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2009/P1D", 0);
|
||||
SegmentDescriptor desc3 = SD(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
|
||||
SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
|
||||
|
@ -1084,7 +1089,7 @@ public class KafkaIndexTaskTest
|
|||
Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// 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(
|
||||
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
|
||||
|
@ -1165,8 +1170,8 @@ public class KafkaIndexTaskTest
|
|||
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
|
||||
|
@ -1213,8 +1218,8 @@ public class KafkaIndexTaskTest
|
|||
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
|
||||
|
@ -1308,10 +1313,10 @@ public class KafkaIndexTaskTest
|
|||
Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc3 = SD(task, "2013/P1D", 0);
|
||||
SegmentDescriptor desc4 = SD(task, "2049/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc3 = sd(task, "2013/P1D", 0);
|
||||
SegmentDescriptor desc4 = sd(task, "2049/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 13L))),
|
||||
|
@ -1464,8 +1469,8 @@ public class KafkaIndexTaskTest
|
|||
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published segments & metadata
|
||||
SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
|
||||
|
@ -1529,8 +1534,8 @@ public class KafkaIndexTaskTest
|
|||
Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published segments & metadata, should all be from the first task
|
||||
SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
|
||||
|
@ -1582,8 +1587,8 @@ public class KafkaIndexTaskTest
|
|||
Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode());
|
||||
|
||||
// Check published segments & metadata
|
||||
SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
|
||||
Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
|
||||
|
||||
|
@ -1600,8 +1605,8 @@ public class KafkaIndexTaskTest
|
|||
Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published segments & metadata
|
||||
SegmentDescriptor desc3 = SD(task2, "2011/P1D", 1);
|
||||
SegmentDescriptor desc4 = SD(task2, "2013/P1D", 0);
|
||||
SegmentDescriptor desc3 = sd(task2, "2011/P1D", 1);
|
||||
SegmentDescriptor desc4 = sd(task2, "2013/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
|
||||
Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
|
||||
|
||||
|
@ -1644,11 +1649,11 @@ public class KafkaIndexTaskTest
|
|||
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published segments & metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2010/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
|
||||
SegmentDescriptor desc3 = SD(task, "2011/P1D", 1);
|
||||
SegmentDescriptor desc4 = SD(task, "2012/P1D", 0);
|
||||
SegmentDescriptor desc3 = sd(task, "2011/P1D", 1);
|
||||
SegmentDescriptor desc4 = sd(task, "2012/P1D", 0);
|
||||
Assert.assertEquals(isIncrementalHandoffSupported
|
||||
? ImmutableSet.of(desc1, desc2, desc4)
|
||||
: ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
|
||||
|
@ -1723,9 +1728,9 @@ public class KafkaIndexTaskTest
|
|||
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published segments & metadata
|
||||
SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
|
||||
SegmentDescriptor desc3 = SD(task2, "2012/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
|
||||
SegmentDescriptor desc3 = sd(task2, "2012/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
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());
|
||||
|
||||
// Check published segments & metadata
|
||||
SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 6L))),
|
||||
|
@ -1910,8 +1915,8 @@ public class KafkaIndexTaskTest
|
|||
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 6L))),
|
||||
|
@ -2038,8 +2043,8 @@ public class KafkaIndexTaskTest
|
|||
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
|
||||
|
@ -2179,10 +2184,10 @@ public class KafkaIndexTaskTest
|
|||
Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2008/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2009/P1D", 0);
|
||||
SegmentDescriptor desc3 = SD(task, "2013/P1D", 0);
|
||||
SegmentDescriptor desc4 = SD(task, "2049/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
|
||||
SegmentDescriptor desc3 = sd(task, "2013/P1D", 0);
|
||||
SegmentDescriptor desc4 = sd(task, "2049/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 13L))),
|
||||
|
@ -2560,9 +2565,7 @@ public class KafkaIndexTaskTest
|
|||
this::makeTimeseriesAndScanConglomerate,
|
||||
Execs.directExecutor(), // queryExecutorService
|
||||
EasyMock.createMock(MonitorScheduler.class),
|
||||
new SegmentLoaderFactory(
|
||||
new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper())
|
||||
),
|
||||
new SegmentLoaderFactory(null, testUtils.getTestObjectMapper()),
|
||||
testUtils.getTestObjectMapper(),
|
||||
testUtils.getTestIndexIO(),
|
||||
MapCache.create(1024),
|
||||
|
@ -2673,7 +2676,7 @@ public class KafkaIndexTaskTest
|
|||
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 {
|
||||
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);
|
||||
return new SegmentDescriptor(interval, getLock(task, interval).getVersion(), partitionNum);
|
||||
|
|
|
@ -66,25 +66,25 @@ public class KafkaRecordSupplierTest
|
|||
private static List<ProducerRecord<byte[], byte[]>> generateRecords(String topic)
|
||||
{
|
||||
return ImmutableList.of(
|
||||
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("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", "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("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("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", "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, StringUtils.toUtf8("unparseable")),
|
||||
new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable2")),
|
||||
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("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", "20.0", "notanumber")),
|
||||
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, 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, 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("2012", "g", "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 {
|
||||
return new ObjectMapper().writeValueAsBytes(
|
||||
|
|
|
@ -29,10 +29,10 @@ import java.util.Map;
|
|||
|
||||
public class KinesisDataSourceMetadataTest
|
||||
{
|
||||
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 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 KM0 = km("foo", ImmutableMap.of());
|
||||
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 KM3 = km("foo", ImmutableMap.of("0", "2L", "2", "5L"));
|
||||
|
||||
@Test
|
||||
public void testMatches()
|
||||
|
@ -71,27 +71,27 @@ public class KinesisDataSourceMetadataTest
|
|||
public void testPlus()
|
||||
{
|
||||
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)
|
||||
);
|
||||
|
||||
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)
|
||||
);
|
||||
|
||||
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)
|
||||
);
|
||||
|
||||
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)
|
||||
);
|
||||
|
||||
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)
|
||||
);
|
||||
}
|
||||
|
@ -100,32 +100,32 @@ public class KinesisDataSourceMetadataTest
|
|||
public void testMinus()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
KM("foo", ImmutableMap.of("1", "3L")),
|
||||
km("foo", ImmutableMap.of("1", "3L")),
|
||||
KM1.minus(KM3)
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
KM("foo", ImmutableMap.of()),
|
||||
km("foo", ImmutableMap.of()),
|
||||
KM0.minus(KM2)
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
KM("foo", ImmutableMap.of()),
|
||||
km("foo", ImmutableMap.of()),
|
||||
KM1.minus(KM2)
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
KM("foo", ImmutableMap.of("2", "5L")),
|
||||
km("foo", ImmutableMap.of("2", "5L")),
|
||||
KM2.minus(KM1)
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
KM("foo", ImmutableMap.of()),
|
||||
km("foo", ImmutableMap.of()),
|
||||
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));
|
||||
}
|
||||
|
|
|
@ -135,7 +135,6 @@ import org.apache.druid.segment.loading.DataSegmentPusher;
|
|||
import org.apache.druid.segment.loading.LocalDataSegmentPusher;
|
||||
import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig;
|
||||
import org.apache.druid.segment.loading.SegmentLoaderConfig;
|
||||
import org.apache.druid.segment.loading.SegmentLoaderLocalCacheManager;
|
||||
import org.apache.druid.segment.loading.StorageLocationConfig;
|
||||
import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl;
|
||||
import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
|
||||
|
@ -199,26 +198,26 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
|||
private static String shardId0 = "0";
|
||||
private static KinesisRecordSupplier recordSupplier;
|
||||
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", "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", "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", "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", "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", "4", jb("2011", "e", "y", "10", "20.0", "1.0")),
|
||||
new OrderedPartitionableRecord<>(
|
||||
stream,
|
||||
"1",
|
||||
"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", "7", Collections.singletonList(StringUtils.toUtf8("unparseable2"))),
|
||||
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", "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", "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", "1", JB("2011", "h", "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", "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, "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"))
|
||||
);
|
||||
|
||||
private static ServiceEmitter emitter;
|
||||
|
@ -405,8 +404,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
|||
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(
|
||||
|
@ -485,8 +484,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
|||
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2012/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2012/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KinesisDataSourceMetadata(
|
||||
|
@ -610,13 +609,13 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
|||
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2008/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2009/P1D", 0);
|
||||
SegmentDescriptor desc3 = SD(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc4 = SD(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc5 = SD(task, "2011/P1D", 1);
|
||||
SegmentDescriptor desc6 = SD(task, "2012/P1D", 0);
|
||||
SegmentDescriptor desc7 = SD(task, "2013/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
|
||||
SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc4 = sd(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc5 = sd(task, "2011/P1D", 1);
|
||||
SegmentDescriptor desc6 = sd(task, "2012/P1D", 0);
|
||||
SegmentDescriptor desc7 = sd(task, "2013/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
|
||||
|
@ -771,12 +770,12 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
|||
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2008/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2009/P1D", 0);
|
||||
SegmentDescriptor desc3 = SD(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc4 = SD(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc5 = SD(task, "2049/P1D", 0);
|
||||
SegmentDescriptor desc7 = SD(task, "2013/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
|
||||
SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc4 = sd(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc5 = sd(task, "2049/P1D", 0);
|
||||
SegmentDescriptor desc7 = sd(task, "2013/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc7), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
|
||||
|
@ -858,8 +857,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
|||
Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KinesisDataSourceMetadata(
|
||||
|
@ -941,9 +940,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
|||
Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2008/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2009/P1D", 0);
|
||||
SegmentDescriptor desc3 = SD(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
|
||||
SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KinesisDataSourceMetadata(
|
||||
|
@ -1034,7 +1033,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
|||
Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// 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(
|
||||
new KinesisDataSourceMetadata(
|
||||
|
@ -1172,8 +1171,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
|||
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
|
||||
|
@ -1249,8 +1248,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
|||
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
|
||||
|
@ -1396,10 +1395,10 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
|||
Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getUnparseable());
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc3 = SD(task, "2013/P1D", 0);
|
||||
SegmentDescriptor desc4 = SD(task, "2049/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc3 = sd(task, "2013/P1D", 0);
|
||||
SegmentDescriptor desc4 = sd(task, "2049/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KinesisDataSourceMetadata(
|
||||
|
@ -1621,8 +1620,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
|||
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published segments & metadata
|
||||
SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KinesisDataSourceMetadata(
|
||||
|
@ -1732,8 +1731,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
|||
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published segments & metadata, should all be from the first task
|
||||
SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KinesisDataSourceMetadata(
|
||||
|
@ -1830,8 +1829,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
|||
Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode());
|
||||
|
||||
// Check published segments & metadata
|
||||
SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
|
||||
Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
|
||||
|
||||
|
@ -1850,8 +1849,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
|||
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published segments & metadata
|
||||
SegmentDescriptor desc3 = SD(task2, "2011/P1D", 1);
|
||||
SegmentDescriptor desc4 = SD(task2, "2013/P1D", 0);
|
||||
SegmentDescriptor desc3 = sd(task2, "2011/P1D", 1);
|
||||
SegmentDescriptor desc4 = sd(task2, "2013/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
|
||||
Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
|
||||
|
||||
|
@ -1929,9 +1928,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
|||
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published segments & metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc4 = SD(task, "2012/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc4 = sd(task, "2012/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc4), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
|
||||
|
@ -2044,10 +2043,10 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
|||
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published segments & metadata
|
||||
SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
|
||||
SegmentDescriptor desc3 = SD(task2, "2011/P1D", 1);
|
||||
SegmentDescriptor desc4 = SD(task2, "2012/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
|
||||
SegmentDescriptor desc3 = sd(task2, "2011/P1D", 1);
|
||||
SegmentDescriptor desc4 = sd(task2, "2012/P1D", 0);
|
||||
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
|
@ -2197,8 +2196,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
|||
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published segments & metadata
|
||||
SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KinesisDataSourceMetadata(
|
||||
|
@ -2318,8 +2317,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
|||
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(
|
||||
|
@ -2408,8 +2407,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
|||
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
|
||||
|
||||
// Check published metadata
|
||||
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
|
||||
SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
|
||||
SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
|
||||
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
|
||||
Assert.assertEquals(
|
||||
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
|
||||
|
@ -2765,9 +2764,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
|||
this::makeTimeseriesOnlyConglomerate,
|
||||
Execs.directExecutor(), // queryExecutorService
|
||||
EasyMock.createMock(MonitorScheduler.class),
|
||||
new SegmentLoaderFactory(
|
||||
new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper())
|
||||
),
|
||||
new SegmentLoaderFactory(null, testUtils.getTestObjectMapper()),
|
||||
testUtils.getTestObjectMapper(),
|
||||
testUtils.getTestIndexIO(),
|
||||
MapCache.create(1024),
|
||||
|
@ -2880,7 +2877,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
|||
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 dim1,
|
||||
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);
|
||||
return new SegmentDescriptor(interval, getLock(task, interval).getVersion(), partitionNum);
|
||||
|
|
|
@ -76,20 +76,20 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
|
|||
private static Shard shard1;
|
||||
private static KinesisRecordSupplier recordSupplier;
|
||||
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", "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("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("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("unparseable2"))).withSequenceNumber("4"),
|
||||
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("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("2011", "h", "y", "10", "20.0", "1.0")).withSequenceNumber("9")
|
||||
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("2012", "g", "y", "10", "20.0", "1.0")).withSequenceNumber("8"),
|
||||
new Record().withData(jb("2011", "h", "y", "10", "20.0", "1.0")).withSequenceNumber("9")
|
||||
);
|
||||
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("2009", "b", "y", "10", "20.0", "1.0")).withSequenceNumber("1")
|
||||
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")
|
||||
);
|
||||
private static List<Object> allRecords = ImmutableList.builder()
|
||||
.addAll(shard0Records.stream()
|
||||
|
@ -120,7 +120,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
|
|||
.toList()))
|
||||
.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 {
|
||||
return ByteBuffer.wrap(new ObjectMapper().writeValueAsBytes(
|
||||
|
|
|
@ -3560,7 +3560,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
);
|
||||
}
|
||||
|
||||
private static List<byte[]> JB(
|
||||
private static List<byte[]> jb(
|
||||
String timestamp,
|
||||
String dim1,
|
||||
String dim2,
|
||||
|
|
|
@ -43,6 +43,7 @@ import org.apache.druid.guice.JsonConfigProvider;
|
|||
import org.apache.druid.guice.LazySingleton;
|
||||
import org.apache.druid.initialization.DruidModule;
|
||||
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 javax.annotation.Nullable;
|
||||
|
@ -184,7 +185,8 @@ public class S3StorageDruidModule implements DruidModule
|
|||
final Protocol protocolFromClientConfig = parseProtocol(clientConfig.getProtocol());
|
||||
final String endpointUrl = endpointConfig.getUrl();
|
||||
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());
|
||||
if (protocol != null && (protocol != protocolFromClientConfig)) {
|
||||
log.warn("[%s] protocol will be used for endpoint [%s]", protocol, endpointUrl);
|
||||
|
|
|
@ -19,7 +19,10 @@
|
|||
|
||||
package org.apache.druid.indexing.common;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
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.SegmentLoaderConfig;
|
||||
import org.apache.druid.segment.loading.SegmentLoaderLocalCacheManager;
|
||||
|
@ -29,23 +32,30 @@ import java.io.File;
|
|||
import java.util.Collections;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class SegmentLoaderFactory
|
||||
{
|
||||
private final SegmentLoaderLocalCacheManager loader;
|
||||
private final IndexIO indexIO;
|
||||
private final ObjectMapper jsonMapper;
|
||||
|
||||
@Inject
|
||||
public SegmentLoaderFactory(
|
||||
SegmentLoaderLocalCacheManager loader
|
||||
IndexIO indexIO,
|
||||
@Json ObjectMapper mapper
|
||||
)
|
||||
{
|
||||
this.loader = loader;
|
||||
this.indexIO = indexIO;
|
||||
this.jsonMapper = mapper;
|
||||
}
|
||||
|
||||
public SegmentLoader manufacturate(File storageDir)
|
||||
{
|
||||
return loader.withConfig(
|
||||
new SegmentLoaderConfig().withLocations(Collections.singletonList(new StorageLocationConfig().setPath(storageDir)))
|
||||
return new SegmentLoaderLocalCacheManager(
|
||||
indexIO,
|
||||
new SegmentLoaderConfig().withLocations(
|
||||
Collections.singletonList(new StorageLocationConfig().setPath(storageDir))),
|
||||
jsonMapper
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -29,6 +29,7 @@ import com.google.common.base.Preconditions;
|
|||
import com.google.common.collect.BiMap;
|
||||
import com.google.common.collect.HashBiMap;
|
||||
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.MultiValueHandling;
|
||||
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.TimeAndDimsParseSpec;
|
||||
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.actions.SegmentListUsedAction;
|
||||
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
||||
|
@ -132,6 +135,15 @@ public class CompactionTask extends AbstractTask
|
|||
@JsonIgnore
|
||||
private final RowIngestionMetersFactory rowIngestionMetersFactory;
|
||||
|
||||
@JsonIgnore
|
||||
private final CoordinatorClient coordinatorClient;
|
||||
|
||||
@JsonIgnore
|
||||
private final SegmentLoaderFactory segmentLoaderFactory;
|
||||
|
||||
@JsonIgnore
|
||||
private final RetryPolicyFactory retryPolicyFactory;
|
||||
|
||||
@JsonIgnore
|
||||
private List<IndexTask> indexTaskSpecs;
|
||||
|
||||
|
@ -153,7 +165,10 @@ public class CompactionTask extends AbstractTask
|
|||
@JacksonInject ObjectMapper jsonMapper,
|
||||
@JacksonInject AuthorizerMapper authorizerMapper,
|
||||
@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);
|
||||
|
@ -186,6 +201,9 @@ public class CompactionTask extends AbstractTask
|
|||
this.authorizerMapper = authorizerMapper;
|
||||
this.chatHandlerProvider = chatHandlerProvider;
|
||||
this.rowIngestionMetersFactory = rowIngestionMetersFactory;
|
||||
this.coordinatorClient = coordinatorClient;
|
||||
this.segmentLoaderFactory = segmentLoaderFactory;
|
||||
this.retryPolicyFactory = retryPolicyFactory;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -278,20 +296,23 @@ public class CompactionTask extends AbstractTask
|
|||
metricsSpec,
|
||||
keepSegmentGranularity,
|
||||
segmentGranularity,
|
||||
jsonMapper
|
||||
jsonMapper,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
).stream()
|
||||
.map(spec -> new IndexTask(
|
||||
getId(),
|
||||
getGroupId(),
|
||||
getTaskResource(),
|
||||
getDataSource(),
|
||||
spec,
|
||||
getContext(),
|
||||
authorizerMapper,
|
||||
chatHandlerProvider,
|
||||
rowIngestionMetersFactory
|
||||
))
|
||||
.collect(Collectors.toList());
|
||||
.map(spec -> new IndexTask(
|
||||
getId(),
|
||||
getGroupId(),
|
||||
getTaskResource(),
|
||||
getDataSource(),
|
||||
spec,
|
||||
getContext(),
|
||||
authorizerMapper,
|
||||
chatHandlerProvider,
|
||||
rowIngestionMetersFactory
|
||||
))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
if (indexTaskSpecs.isEmpty()) {
|
||||
|
@ -338,7 +359,10 @@ public class CompactionTask extends AbstractTask
|
|||
@Nullable final AggregatorFactory[] metricsSpec,
|
||||
@Nullable final Boolean keepSegmentGranularity,
|
||||
@Nullable final Granularity segmentGranularity,
|
||||
final ObjectMapper jsonMapper
|
||||
final ObjectMapper jsonMapper,
|
||||
final CoordinatorClient coordinatorClient,
|
||||
final SegmentLoaderFactory segmentLoaderFactory,
|
||||
final RetryPolicyFactory retryPolicyFactory
|
||||
) throws IOException, SegmentLoadingException
|
||||
{
|
||||
Pair<Map<DataSegment, File>, List<TimelineObjectHolder<String, DataSegment>>> pair = prepareSegments(
|
||||
|
@ -379,7 +403,14 @@ public class CompactionTask extends AbstractTask
|
|||
return Collections.singletonList(
|
||||
new IndexIngestionSpec(
|
||||
dataSchema,
|
||||
createIoConfig(toolbox, dataSchema, segmentProvider.interval),
|
||||
createIoConfig(
|
||||
toolbox,
|
||||
dataSchema,
|
||||
segmentProvider.interval,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
),
|
||||
compactionTuningConfig
|
||||
)
|
||||
);
|
||||
|
@ -411,7 +442,14 @@ public class CompactionTask extends AbstractTask
|
|||
specs.add(
|
||||
new IndexIngestionSpec(
|
||||
dataSchema,
|
||||
createIoConfig(toolbox, dataSchema, interval),
|
||||
createIoConfig(
|
||||
toolbox,
|
||||
dataSchema,
|
||||
interval,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
),
|
||||
compactionTuningConfig
|
||||
)
|
||||
);
|
||||
|
@ -438,7 +476,14 @@ public class CompactionTask extends AbstractTask
|
|||
return Collections.singletonList(
|
||||
new IndexIngestionSpec(
|
||||
dataSchema,
|
||||
createIoConfig(toolbox, dataSchema, segmentProvider.interval),
|
||||
createIoConfig(
|
||||
toolbox,
|
||||
dataSchema,
|
||||
segmentProvider.interval,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
),
|
||||
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(
|
||||
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
|
||||
dataSchema.getParser().getParseSpec().getDimensionsSpec().getDimensionNames(),
|
||||
Arrays.stream(dataSchema.getAggregators()).map(AggregatorFactory::getName).collect(Collectors.toList()),
|
||||
toolbox.getIndexIO()
|
||||
toolbox.getIndexIO(),
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
),
|
||||
false
|
||||
);
|
||||
|
@ -811,7 +866,7 @@ public class CompactionTask extends AbstractTask
|
|||
* targetCompactionSizeBytes cannot be used with {@link IndexTuningConfig#maxRowsPerSegment},
|
||||
* {@link IndexTuningConfig#maxTotalRows}, or {@link IndexTuningConfig#numShards} together.
|
||||
* {@link #hasPartitionConfig} checks one of those configs is set.
|
||||
*
|
||||
* <p>
|
||||
* This throws an {@link IllegalArgumentException} if targetCompactionSizeBytes is set and hasPartitionConfig
|
||||
* returns true. If targetCompactionSizeBytes is not set, this returns null or
|
||||
* {@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 ChatHandlerProvider chatHandlerProvider;
|
||||
private final RowIngestionMetersFactory rowIngestionMetersFactory;
|
||||
private final CoordinatorClient coordinatorClient;
|
||||
private final SegmentLoaderFactory segmentLoaderFactory;
|
||||
private final RetryPolicyFactory retryPolicyFactory;
|
||||
|
||||
@Nullable
|
||||
private Interval interval;
|
||||
|
@ -885,7 +943,10 @@ public class CompactionTask extends AbstractTask
|
|||
ObjectMapper jsonMapper,
|
||||
AuthorizerMapper authorizerMapper,
|
||||
ChatHandlerProvider chatHandlerProvider,
|
||||
RowIngestionMetersFactory rowIngestionMetersFactory
|
||||
RowIngestionMetersFactory rowIngestionMetersFactory,
|
||||
CoordinatorClient coordinatorClient,
|
||||
SegmentLoaderFactory segmentLoaderFactory,
|
||||
RetryPolicyFactory retryPolicyFactory
|
||||
)
|
||||
{
|
||||
this.dataSource = dataSource;
|
||||
|
@ -893,6 +954,9 @@ public class CompactionTask extends AbstractTask
|
|||
this.authorizerMapper = authorizerMapper;
|
||||
this.chatHandlerProvider = chatHandlerProvider;
|
||||
this.rowIngestionMetersFactory = rowIngestionMetersFactory;
|
||||
this.coordinatorClient = coordinatorClient;
|
||||
this.segmentLoaderFactory = segmentLoaderFactory;
|
||||
this.retryPolicyFactory = retryPolicyFactory;
|
||||
}
|
||||
|
||||
public Builder interval(Interval interval)
|
||||
|
@ -968,7 +1032,10 @@ public class CompactionTask extends AbstractTask
|
|||
jsonMapper,
|
||||
authorizerMapper,
|
||||
chatHandlerProvider,
|
||||
rowIngestionMetersFactory
|
||||
rowIngestionMetersFactory,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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.RowIngestionMetersFactory;
|
||||
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.Intervals;
|
||||
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.firehose.ChatHandler;
|
||||
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.server.security.Action;
|
||||
import org.apache.druid.server.security.AuthorizerMapper;
|
||||
|
@ -419,8 +417,6 @@ public class IndexTask extends AbstractTask implements ChatHandler
|
|||
|
||||
final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory();
|
||||
|
||||
setFirehoseFactoryToolbox(firehoseFactory, toolbox);
|
||||
|
||||
final File firehoseTempDir = toolbox.getFirehoseTemporaryDir();
|
||||
// Firehose temporary directory is automatically removed when this IndexTask completes.
|
||||
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()
|
||||
{
|
||||
return TaskReport.buildTaskReports(
|
||||
|
|
|
@ -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.TaskResource;
|
||||
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.Intervals;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
@ -185,11 +184,6 @@ public class ParallelIndexSubTask extends AbstractTask
|
|||
{
|
||||
final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory();
|
||||
|
||||
if (firehoseFactory instanceof IngestSegmentFirehoseFactory) {
|
||||
// pass toolbox to Firehose
|
||||
((IngestSegmentFirehoseFactory) firehoseFactory).setTaskToolbox(toolbox);
|
||||
}
|
||||
|
||||
final File firehoseTempDir = toolbox.getFirehoseTemporaryDir();
|
||||
// Firehose temporary directory is automatically removed when this IndexTask completes.
|
||||
FileUtils.forceMkdir(firehoseTempDir);
|
||||
|
|
|
@ -30,16 +30,20 @@ import com.google.common.collect.BiMap;
|
|||
import com.google.common.collect.HashBiMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
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.FirehoseFactory;
|
||||
import org.apache.druid.data.input.impl.InputRowParser;
|
||||
import org.apache.druid.indexing.common.TaskToolbox;
|
||||
import org.apache.druid.indexing.common.actions.SegmentListUsedAction;
|
||||
import org.apache.druid.indexing.common.RetryPolicy;
|
||||
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.emitter.EmittingLogger;
|
||||
import org.apache.druid.query.filter.DimFilter;
|
||||
import org.apache.druid.segment.IndexIO;
|
||||
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.realtime.firehose.IngestSegmentFirehose;
|
||||
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.VersionedIntervalTimeline;
|
||||
import org.apache.druid.timeline.partition.PartitionChunk;
|
||||
import org.joda.time.Duration;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
|
@ -68,7 +75,9 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
|
|||
private final List<String> dimensions;
|
||||
private final List<String> metrics;
|
||||
private final IndexIO indexIO;
|
||||
private TaskToolbox taskToolbox;
|
||||
private final CoordinatorClient coordinatorClient;
|
||||
private final SegmentLoaderFactory segmentLoaderFactory;
|
||||
private final RetryPolicyFactory retryPolicyFactory;
|
||||
|
||||
@JsonCreator
|
||||
public IngestSegmentFirehoseFactory(
|
||||
|
@ -77,7 +86,10 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
|
|||
@JsonProperty("filter") DimFilter dimFilter,
|
||||
@JsonProperty("dimensions") List<String> dimensions,
|
||||
@JsonProperty("metrics") List<String> metrics,
|
||||
@JacksonInject IndexIO indexIO
|
||||
@JacksonInject IndexIO indexIO,
|
||||
@JacksonInject CoordinatorClient coordinatorClient,
|
||||
@JacksonInject SegmentLoaderFactory segmentLoaderFactory,
|
||||
@JacksonInject RetryPolicyFactory retryPolicyFactory
|
||||
)
|
||||
{
|
||||
Preconditions.checkNotNull(dataSource, "dataSource");
|
||||
|
@ -88,6 +100,9 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
|
|||
this.dimensions = dimensions;
|
||||
this.metrics = metrics;
|
||||
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
|
||||
|
@ -120,23 +135,46 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
|
|||
return metrics;
|
||||
}
|
||||
|
||||
public void setTaskToolbox(TaskToolbox taskToolbox)
|
||||
{
|
||||
this.taskToolbox = taskToolbox;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Firehose connect(InputRowParser inputRowParser, File temporaryDirectory) throws ParseException
|
||||
{
|
||||
log.info("Connecting firehose: dataSource[%s], interval[%s]", dataSource, interval);
|
||||
|
||||
Preconditions.checkNotNull(taskToolbox, "taskToolbox is not set");
|
||||
|
||||
try {
|
||||
final List<DataSegment> usedSegments = taskToolbox
|
||||
.getTaskActionClient()
|
||||
.submit(new SegmentListUsedAction(dataSource, interval, null));
|
||||
final Map<DataSegment, File> segmentFileMap = taskToolbox.fetchSegments(usedSegments);
|
||||
// This call used to use the TaskActionClient, so for compatibility we use the same retry configuration
|
||||
// as TaskActionClient.
|
||||
final RetryPolicy retryPolicy = retryPolicyFactory.makeRetryPolicy();
|
||||
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
|
||||
.forSegments(usedSegments)
|
||||
.lookup(interval);
|
||||
|
@ -201,11 +239,18 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
|
|||
final TransformSpec transformSpec = TransformSpec.fromInputRowParser(inputRowParser);
|
||||
return new IngestSegmentFirehose(adapters, transformSpec, dims, metricsList, dimFilter);
|
||||
}
|
||||
catch (IOException | SegmentLoadingException e) {
|
||||
catch (SegmentLoadingException 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
|
||||
static List<String> getUniqueDimensions(
|
||||
List<TimelineObjectHolder<String, DataSegment>> timelineSegments,
|
||||
|
@ -260,7 +305,7 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
|
|||
|
||||
final BiMap<Integer, String> orderedMetrics = uniqueMetrics.inverse();
|
||||
return IntStream.range(0, orderedMetrics.size())
|
||||
.mapToObj(orderedMetrics::get)
|
||||
.collect(Collectors.toList());
|
||||
.mapToObj(orderedMetrics::get)
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -76,6 +76,7 @@ public class TaskToolboxTest
|
|||
private MonitorScheduler mockMonitorScheduler = EasyMock.createMock(MonitorScheduler.class);
|
||||
private ExecutorService mockQueryExecutorService = EasyMock.createMock(ExecutorService.class);
|
||||
private ObjectMapper ObjectMapper = new ObjectMapper();
|
||||
private SegmentLoaderFactory mockSegmentLoaderFactory = EasyMock.createMock(SegmentLoaderFactory.class);
|
||||
private SegmentLoaderLocalCacheManager mockSegmentLoaderLocalCacheManager = EasyMock.createMock(SegmentLoaderLocalCacheManager.class);
|
||||
private Task task = EasyMock.createMock(Task.class);
|
||||
private IndexMergerV9 mockIndexMergerV9 = EasyMock.createMock(IndexMergerV9.class);
|
||||
|
@ -107,7 +108,7 @@ public class TaskToolboxTest
|
|||
() -> mockQueryRunnerFactoryConglomerate,
|
||||
mockQueryExecutorService,
|
||||
mockMonitorScheduler,
|
||||
new SegmentLoaderFactory(mockSegmentLoaderLocalCacheManager),
|
||||
mockSegmentLoaderFactory,
|
||||
ObjectMapper,
|
||||
mockIndexIO,
|
||||
mockCache,
|
||||
|
@ -162,13 +163,13 @@ public class TaskToolboxTest
|
|||
public void testFetchSegments() throws SegmentLoadingException, IOException
|
||||
{
|
||||
File expectedFile = temporaryFolder.newFile();
|
||||
EasyMock
|
||||
.expect(mockSegmentLoaderFactory.manufacturate(EasyMock.anyObject()))
|
||||
.andReturn(mockSegmentLoaderLocalCacheManager).anyTimes();
|
||||
EasyMock
|
||||
.expect(mockSegmentLoaderLocalCacheManager.getSegmentFiles(EasyMock.anyObject()))
|
||||
.andReturn(expectedFile).anyTimes();
|
||||
EasyMock
|
||||
.expect(mockSegmentLoaderLocalCacheManager.withConfig(EasyMock.anyObject()))
|
||||
.andReturn(mockSegmentLoaderLocalCacheManager).anyTimes();
|
||||
EasyMock.replay(mockSegmentLoaderLocalCacheManager);
|
||||
EasyMock.replay(mockSegmentLoaderFactory, mockSegmentLoaderLocalCacheManager);
|
||||
DataSegment dataSegment = DataSegment.builder().dataSource("source").interval(Intervals.of("2012-01-01/P1D")).version("1").size(1).build();
|
||||
List<DataSegment> segments = ImmutableList.of
|
||||
(
|
||||
|
|
|
@ -119,7 +119,6 @@ import org.apache.druid.segment.indexing.DataSchema;
|
|||
import org.apache.druid.segment.indexing.RealtimeIOConfig;
|
||||
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||
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.SegmentHandoffNotifier;
|
||||
import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory;
|
||||
|
@ -1607,9 +1606,7 @@ public class AppenderatorDriverRealtimeIndexTaskTest
|
|||
() -> conglomerate,
|
||||
Execs.directExecutor(), // queryExecutorService
|
||||
EasyMock.createMock(MonitorScheduler.class),
|
||||
new SegmentLoaderFactory(
|
||||
new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper())
|
||||
),
|
||||
new SegmentLoaderFactory(null, testUtils.getTestObjectMapper()),
|
||||
testUtils.getTestObjectMapper(),
|
||||
testUtils.getTestIndexIO(),
|
||||
MapCache.create(1024),
|
||||
|
|
|
@ -23,11 +23,15 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
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.DimensionsSpec;
|
||||
import org.apache.druid.data.input.impl.ParseSpec;
|
||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||
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.TestUtils;
|
||||
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.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.partition.NumberedShardSpec;
|
||||
import org.joda.time.Interval;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
|
@ -96,12 +101,24 @@ public class CompactionTaskRunTest extends IngestionTestBase
|
|||
);
|
||||
|
||||
private RowIngestionMetersFactory rowIngestionMetersFactory;
|
||||
private CoordinatorClient coordinatorClient;
|
||||
private SegmentLoaderFactory segmentLoaderFactory;
|
||||
private ExecutorService exec;
|
||||
private static RetryPolicyFactory retryPolicyFactory = new RetryPolicyFactory(new RetryPolicyConfig());
|
||||
|
||||
public CompactionTaskRunTest()
|
||||
{
|
||||
TestUtils testUtils = new TestUtils();
|
||||
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
|
||||
|
@ -126,7 +143,10 @@ public class CompactionTaskRunTest extends IngestionTestBase
|
|||
getObjectMapper(),
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
null,
|
||||
rowIngestionMetersFactory
|
||||
rowIngestionMetersFactory,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
);
|
||||
|
||||
final CompactionTask compactionTask = builder
|
||||
|
@ -156,7 +176,10 @@ public class CompactionTaskRunTest extends IngestionTestBase
|
|||
getObjectMapper(),
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
null,
|
||||
rowIngestionMetersFactory
|
||||
rowIngestionMetersFactory,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
);
|
||||
|
||||
final CompactionTask compactionTask1 = builder
|
||||
|
@ -200,7 +223,10 @@ public class CompactionTaskRunTest extends IngestionTestBase
|
|||
getObjectMapper(),
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
null,
|
||||
rowIngestionMetersFactory
|
||||
rowIngestionMetersFactory,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
);
|
||||
|
||||
final CompactionTask compactionTask1 = builder
|
||||
|
@ -248,7 +274,10 @@ public class CompactionTaskRunTest extends IngestionTestBase
|
|||
getObjectMapper(),
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
null,
|
||||
rowIngestionMetersFactory
|
||||
rowIngestionMetersFactory,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
);
|
||||
|
||||
// day segmentGranularity
|
||||
|
|
|
@ -28,6 +28,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
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.impl.DimensionSchema;
|
||||
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.GuiceInjectableValues;
|
||||
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.TestUtils;
|
||||
import org.apache.druid.indexing.common.actions.SegmentListUsedAction;
|
||||
|
@ -153,12 +157,15 @@ public class CompactionTaskTest
|
|||
private static List<AggregatorFactory> AGGREGATORS;
|
||||
private static List<DataSegment> SEGMENTS;
|
||||
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 Map<DataSegment, File> segmentMap;
|
||||
private static RetryPolicyFactory retryPolicyFactory = new RetryPolicyFactory(new RetryPolicyConfig());
|
||||
|
||||
private final boolean keepSegmentGranularity;
|
||||
|
||||
private TaskToolbox toolbox;
|
||||
private SegmentLoaderFactory segmentLoaderFactory;
|
||||
|
||||
@BeforeClass
|
||||
public static void setupClass()
|
||||
|
@ -202,7 +209,6 @@ public class CompactionTaskTest
|
|||
AGGREGATORS.add(new FloatFirstAggregatorFactory("agg_3", "float_dim_3"));
|
||||
AGGREGATORS.add(new DoubleLastAggregatorFactory("agg_4", "double_dim_4"));
|
||||
|
||||
segmentMap = new HashMap<>(SEGMENT_INTERVALS.size());
|
||||
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)));
|
||||
segmentMap.put(
|
||||
|
@ -243,6 +249,8 @@ public class CompactionTaskTest
|
|||
binder.bind(AuthorizerMapper.class).toInstance(AuthTestUtils.TEST_AUTHORIZER_MAPPER);
|
||||
binder.bind(ChatHandlerProvider.class).toInstance(new NoopChatHandlerProvider());
|
||||
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
|
||||
public void setup()
|
||||
{
|
||||
final IndexIO testIndexIO = new TestIndexIO(objectMapper, segmentMap);
|
||||
toolbox = new TestTaskToolbox(
|
||||
new TestTaskActionClient(new ArrayList<>(segmentMap.keySet())),
|
||||
new TestIndexIO(objectMapper, segmentMap),
|
||||
testIndexIO,
|
||||
segmentMap
|
||||
);
|
||||
segmentLoaderFactory = new SegmentLoaderFactory(testIndexIO, objectMapper);
|
||||
}
|
||||
|
||||
@Parameters(name = "keepSegmentGranularity={0}")
|
||||
public static Collection<Object[]> parameters()
|
||||
{
|
||||
return ImmutableList.of(
|
||||
new Object[] {false},
|
||||
new Object[] {true}
|
||||
new Object[]{false},
|
||||
new Object[]{true}
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -336,7 +346,10 @@ public class CompactionTaskTest
|
|||
objectMapper,
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
null,
|
||||
rowIngestionMetersFactory
|
||||
rowIngestionMetersFactory,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
);
|
||||
final CompactionTask task = builder
|
||||
.interval(COMPACTION_INTERVAL)
|
||||
|
@ -357,7 +370,10 @@ public class CompactionTaskTest
|
|||
objectMapper,
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
null,
|
||||
rowIngestionMetersFactory
|
||||
rowIngestionMetersFactory,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
);
|
||||
final CompactionTask task = builder
|
||||
.segments(SEGMENTS)
|
||||
|
@ -378,7 +394,10 @@ public class CompactionTaskTest
|
|||
objectMapper,
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
null,
|
||||
rowIngestionMetersFactory
|
||||
rowIngestionMetersFactory,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
);
|
||||
|
||||
final CompactionTask task = builder
|
||||
|
@ -426,7 +445,10 @@ public class CompactionTaskTest
|
|||
null,
|
||||
keepSegmentGranularity,
|
||||
null,
|
||||
objectMapper
|
||||
objectMapper,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
);
|
||||
final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(
|
||||
keepSegmentGranularity
|
||||
|
@ -440,7 +462,13 @@ public class CompactionTaskTest
|
|||
)
|
||||
);
|
||||
Assert.assertEquals(6, ingestionSpecs.size());
|
||||
assertIngestionSchema(ingestionSpecs, expectedDimensionsSpec, AGGREGATORS, SEGMENT_INTERVALS, Granularities.MONTH);
|
||||
assertIngestionSchema(
|
||||
ingestionSpecs,
|
||||
expectedDimensionsSpec,
|
||||
AGGREGATORS,
|
||||
SEGMENT_INTERVALS,
|
||||
Granularities.MONTH
|
||||
);
|
||||
} else {
|
||||
Assert.assertEquals(1, ingestionSpecs.size());
|
||||
assertIngestionSchema(
|
||||
|
@ -491,7 +519,10 @@ public class CompactionTaskTest
|
|||
null,
|
||||
keepSegmentGranularity,
|
||||
null,
|
||||
objectMapper
|
||||
objectMapper,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
);
|
||||
final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(
|
||||
keepSegmentGranularity
|
||||
|
@ -564,7 +595,10 @@ public class CompactionTaskTest
|
|||
null,
|
||||
keepSegmentGranularity,
|
||||
null,
|
||||
objectMapper
|
||||
objectMapper,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
);
|
||||
final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(
|
||||
keepSegmentGranularity
|
||||
|
@ -637,7 +671,10 @@ public class CompactionTaskTest
|
|||
null,
|
||||
keepSegmentGranularity,
|
||||
null,
|
||||
objectMapper
|
||||
objectMapper,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
);
|
||||
final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(
|
||||
keepSegmentGranularity
|
||||
|
@ -710,7 +747,10 @@ public class CompactionTaskTest
|
|||
null,
|
||||
keepSegmentGranularity,
|
||||
null,
|
||||
objectMapper
|
||||
objectMapper,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
);
|
||||
|
||||
if (keepSegmentGranularity) {
|
||||
|
@ -760,7 +800,10 @@ public class CompactionTaskTest
|
|||
customMetricsSpec,
|
||||
keepSegmentGranularity,
|
||||
null,
|
||||
objectMapper
|
||||
objectMapper,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
);
|
||||
|
||||
final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(
|
||||
|
@ -805,7 +848,10 @@ public class CompactionTaskTest
|
|||
null,
|
||||
keepSegmentGranularity,
|
||||
null,
|
||||
objectMapper
|
||||
objectMapper,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
);
|
||||
final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(
|
||||
keepSegmentGranularity
|
||||
|
@ -819,7 +865,13 @@ public class CompactionTaskTest
|
|||
)
|
||||
);
|
||||
Assert.assertEquals(6, ingestionSpecs.size());
|
||||
assertIngestionSchema(ingestionSpecs, expectedDimensionsSpec, AGGREGATORS, SEGMENT_INTERVALS, Granularities.MONTH);
|
||||
assertIngestionSchema(
|
||||
ingestionSpecs,
|
||||
expectedDimensionsSpec,
|
||||
AGGREGATORS,
|
||||
SEGMENT_INTERVALS,
|
||||
Granularities.MONTH
|
||||
);
|
||||
} else {
|
||||
Assert.assertEquals(1, ingestionSpecs.size());
|
||||
assertIngestionSchema(
|
||||
|
@ -850,7 +902,10 @@ public class CompactionTaskTest
|
|||
null,
|
||||
keepSegmentGranularity,
|
||||
null,
|
||||
objectMapper
|
||||
objectMapper,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -871,7 +926,10 @@ public class CompactionTaskTest
|
|||
null,
|
||||
keepSegmentGranularity,
|
||||
null,
|
||||
objectMapper
|
||||
objectMapper,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -886,7 +944,10 @@ public class CompactionTaskTest
|
|||
objectMapper,
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
null,
|
||||
rowIngestionMetersFactory
|
||||
rowIngestionMetersFactory,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
);
|
||||
|
||||
final CompactionTask task = builder
|
||||
|
@ -934,7 +995,10 @@ public class CompactionTaskTest
|
|||
null,
|
||||
keepSegmentGranularity,
|
||||
null,
|
||||
objectMapper
|
||||
objectMapper,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -949,7 +1013,10 @@ public class CompactionTaskTest
|
|||
null,
|
||||
null,
|
||||
new PeriodGranularity(Period.months(3), null, null),
|
||||
objectMapper
|
||||
objectMapper,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
);
|
||||
final List<DimensionsSpec> expectedDimensionsSpec = ImmutableList.of(
|
||||
new DimensionsSpec(getDimensionSchema(new DoubleDimensionSchema("string_to_double")))
|
||||
|
@ -982,7 +1049,10 @@ public class CompactionTaskTest
|
|||
null,
|
||||
false,
|
||||
new PeriodGranularity(Period.months(3), null, null),
|
||||
objectMapper
|
||||
objectMapper,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
);
|
||||
final List<DimensionsSpec> expectedDimensionsSpec = ImmutableList.of(
|
||||
new DimensionsSpec(getDimensionSchema(new DoubleDimensionSchema("string_to_double")))
|
||||
|
@ -1015,7 +1085,10 @@ public class CompactionTaskTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
objectMapper
|
||||
objectMapper,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
);
|
||||
final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(
|
||||
true
|
||||
|
@ -1048,7 +1121,10 @@ public class CompactionTaskTest
|
|||
objectMapper,
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
null,
|
||||
rowIngestionMetersFactory
|
||||
rowIngestionMetersFactory,
|
||||
coordinatorClient,
|
||||
segmentLoaderFactory,
|
||||
retryPolicyFactory
|
||||
);
|
||||
final CompactionTask task = builder
|
||||
.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 final Map<DataSegment, File> segmentFileMap;
|
||||
|
|
|
@ -107,7 +107,6 @@ import org.apache.druid.segment.indexing.RealtimeIOConfig;
|
|||
import org.apache.druid.segment.indexing.RealtimeTuningConfig;
|
||||
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||
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.FireDepartment;
|
||||
import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier;
|
||||
|
@ -1076,9 +1075,7 @@ public class RealtimeIndexTaskTest
|
|||
() -> conglomerate,
|
||||
Execs.directExecutor(), // queryExecutorService
|
||||
EasyMock.createMock(MonitorScheduler.class),
|
||||
new SegmentLoaderFactory(
|
||||
new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper())
|
||||
),
|
||||
new SegmentLoaderFactory(null, testUtils.getTestObjectMapper()),
|
||||
testUtils.getTestObjectMapper(),
|
||||
testUtils.getTestIndexIO(),
|
||||
MapCache.create(1024),
|
||||
|
|
|
@ -29,6 +29,9 @@ import com.google.common.collect.ImmutableSet;
|
|||
import com.google.common.io.Files;
|
||||
import com.google.inject.Binder;
|
||||
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.impl.DimensionsSpec;
|
||||
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.GuiceInjectableValues;
|
||||
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.TaskToolboxFactory;
|
||||
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.task.NoopTask;
|
||||
import org.apache.druid.indexing.common.task.NoopTestTaskFileWriter;
|
||||
import org.apache.druid.indexing.common.task.Task;
|
||||
import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage;
|
||||
import org.apache.druid.indexing.overlord.TaskLockbox;
|
||||
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.Intervals;
|
||||
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.emitter.service.ServiceEmitter;
|
||||
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.LongSumAggregatorFactory;
|
||||
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.incremental.IncrementalIndex;
|
||||
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.LocalLoadSpec;
|
||||
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.firehose.IngestSegmentFirehose;
|
||||
import org.apache.druid.segment.realtime.firehose.CombiningFirehoseFactory;
|
||||
import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory;
|
||||
import org.apache.druid.segment.transform.ExpressionTransform;
|
||||
import org.apache.druid.segment.transform.TransformSpec;
|
||||
|
@ -103,15 +93,12 @@ import org.junit.runners.Parameterized;
|
|||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
|
@ -170,157 +157,21 @@ public class IngestSegmentFirehoseFactoryTest
|
|||
}
|
||||
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
|
||||
public List<DataSegment> getUsedSegmentsForInterval(String dataSource, Interval interval)
|
||||
public List<DataSegment> getDatabaseSegmentDataSourceSegments(String dataSource, List<Interval> intervals)
|
||||
{
|
||||
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);
|
||||
EasyMock.replay(notifierFactory);
|
||||
|
||||
SegmentLoaderConfig segmentLoaderConfig = new SegmentLoaderConfig()
|
||||
{
|
||||
@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();
|
||||
}
|
||||
final SegmentLoaderFactory slf = new SegmentLoaderFactory(null, MAPPER);
|
||||
final RetryPolicyFactory retryPolicyFactory = new RetryPolicyFactory(new RetryPolicyConfig());
|
||||
|
||||
@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<>();
|
||||
for (InputRowParser parser : Arrays.<InputRowParser>asList(
|
||||
ROW_PARSER,
|
||||
|
@ -342,27 +193,35 @@ public class IngestSegmentFirehoseFactoryTest
|
|||
null,
|
||||
ImmutableList.of(METRIC_LONG_NAME, METRIC_FLOAT_NAME)
|
||||
)) {
|
||||
final IngestSegmentFirehoseFactory factory = new IngestSegmentFirehoseFactory(
|
||||
TASK.getDataSource(),
|
||||
Intervals.ETERNITY,
|
||||
new SelectorDimFilter(DIM_NAME, DIM_VALUE, null),
|
||||
dim_names,
|
||||
metric_names,
|
||||
INDEX_IO
|
||||
);
|
||||
factory.setTaskToolbox(taskToolboxFactory.build(TASK));
|
||||
values.add(
|
||||
new Object[]{
|
||||
StringUtils.format(
|
||||
"DimNames[%s]MetricNames[%s]ParserDimNames[%s]",
|
||||
dim_names == null ? "null" : "dims",
|
||||
metric_names == null ? "null" : "metrics",
|
||||
parser == ROW_PARSER ? "dims" : "null"
|
||||
),
|
||||
factory,
|
||||
parser
|
||||
}
|
||||
);
|
||||
for (Boolean wrapInCombining : Arrays.asList(false, true)) {
|
||||
final IngestSegmentFirehoseFactory isfFactory = new IngestSegmentFirehoseFactory(
|
||||
TASK.getDataSource(),
|
||||
Intervals.ETERNITY,
|
||||
new SelectorDimFilter(DIM_NAME, DIM_VALUE, null),
|
||||
dim_names,
|
||||
metric_names,
|
||||
INDEX_IO,
|
||||
cc,
|
||||
slf,
|
||||
retryPolicyFactory
|
||||
);
|
||||
final FirehoseFactory factory = wrapInCombining
|
||||
? new CombiningFirehoseFactory(ImmutableList.of(isfFactory))
|
||||
: isfFactory;
|
||||
values.add(
|
||||
new Object[]{
|
||||
StringUtils.format(
|
||||
"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(
|
||||
String testName,
|
||||
IngestSegmentFirehoseFactory factory,
|
||||
FirehoseFactory factory,
|
||||
InputRowParser rowParser
|
||||
)
|
||||
{
|
||||
|
@ -436,7 +295,7 @@ public class IngestSegmentFirehoseFactoryTest
|
|||
private static final File persistDir = Paths.get(tmpDir.getAbsolutePath(), "indexTestMerger").toFile();
|
||||
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 static final InputRowParser<Map<String, Object>> ROW_PARSER = new MapInputRowParser(
|
||||
|
@ -518,15 +377,20 @@ public class IngestSegmentFirehoseFactoryTest
|
|||
@Test
|
||||
public void sanityTest()
|
||||
{
|
||||
Assert.assertEquals(TASK.getDataSource(), factory.getDataSource());
|
||||
if (factory.getDimensions() != null) {
|
||||
Assert.assertArrayEquals(new String[]{DIM_NAME}, factory.getDimensions().toArray());
|
||||
if (factory instanceof CombiningFirehoseFactory) {
|
||||
// This method tests IngestSegmentFirehoseFactory-specific methods.
|
||||
return;
|
||||
}
|
||||
Assert.assertEquals(Intervals.ETERNITY, factory.getInterval());
|
||||
if (factory.getMetrics() != null) {
|
||||
final IngestSegmentFirehoseFactory isfFactory = (IngestSegmentFirehoseFactory) factory;
|
||||
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(
|
||||
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());
|
||||
Integer rowcount = 0;
|
||||
try (final IngestSegmentFirehose firehose =
|
||||
(IngestSegmentFirehose)
|
||||
factory.connect(rowParser, null)) {
|
||||
try (final Firehose firehose = factory.connect(rowParser, null)) {
|
||||
while (firehose.hasMore()) {
|
||||
InputRow row = firehose.nextRow();
|
||||
Assert.assertArrayEquals(new String[]{DIM_NAME}, row.getDimensions().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_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;
|
||||
}
|
||||
}
|
||||
|
@ -563,9 +429,8 @@ public class IngestSegmentFirehoseFactoryTest
|
|||
)
|
||||
);
|
||||
int skipped = 0;
|
||||
try (final IngestSegmentFirehose firehose =
|
||||
(IngestSegmentFirehose)
|
||||
factory.connect(transformSpec.decorate(rowParser), null)) {
|
||||
try (final Firehose firehose =
|
||||
factory.connect(transformSpec.decorate(rowParser), null)) {
|
||||
while (firehose.hasMore()) {
|
||||
InputRow row = firehose.nextRow();
|
||||
if (row == null) {
|
||||
|
|
|
@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableMap;
|
|||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.io.Files;
|
||||
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.InputRow;
|
||||
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.MapInputRowParser;
|
||||
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.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.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.Intervals;
|
||||
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.IncrementalIndexSchema;
|
||||
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.transform.TransformSpec;
|
||||
import org.apache.druid.server.metrics.NoopServiceEmitter;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.SegmentId;
|
||||
import org.apache.druid.timeline.partition.LinearShardSpec;
|
||||
|
@ -165,7 +152,7 @@ public class IngestSegmentFirehoseFactoryTimelineTest
|
|||
FileUtils.deleteDirectory(tmpDir);
|
||||
}
|
||||
|
||||
private static TestCase TC(
|
||||
private static TestCase tc(
|
||||
String intervalString,
|
||||
int expectedCount,
|
||||
long expectedSum,
|
||||
|
@ -187,7 +174,7 @@ public class IngestSegmentFirehoseFactoryTimelineTest
|
|||
);
|
||||
}
|
||||
|
||||
private static DataSegmentMaker DS(
|
||||
private static DataSegmentMaker ds(
|
||||
String intervalString,
|
||||
String version,
|
||||
int partitionNum,
|
||||
|
@ -197,7 +184,7 @@ public class IngestSegmentFirehoseFactoryTimelineTest
|
|||
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(
|
||||
DateTimes.of(timeString).getMillis(),
|
||||
|
@ -249,115 +236,68 @@ public class IngestSegmentFirehoseFactoryTimelineTest
|
|||
public static Collection<Object[]> constructorFeeder()
|
||||
{
|
||||
final List<TestCase> testCases = ImmutableList.of(
|
||||
TC(
|
||||
tc(
|
||||
"2000/2000T02", 3, 7,
|
||||
DS("2000/2000T01", "v1", 0, IR("2000", 1), IR("2000T00:01", 2)),
|
||||
DS("2000T01/2000T02", "v1", 0, IR("2000T01", 4))
|
||||
ds("2000/2000T01", "v1", 0, ir("2000", 1), ir("2000T00:01", 2)),
|
||||
ds("2000T01/2000T02", "v1", 0, ir("2000T01", 4))
|
||||
) /* Adjacent segments */,
|
||||
TC(
|
||||
tc(
|
||||
"2000/2000T02", 3, 7,
|
||||
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("2000/2000T02", "v1", 0, ir("2000", 1), ir("2000T00:01", 2), ir("2000T01", 8)),
|
||||
ds("2000T01/2000T02", "v2", 0, ir("2000T01:01", 4))
|
||||
) /* 1H segment overlaid on top of 2H segment */,
|
||||
TC(
|
||||
tc(
|
||||
"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("2000T01/2000T02", "v2", 0, IR("2000T01:01", 4))
|
||||
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))
|
||||
) /* 1H segment overlaid on top of 1D segment */,
|
||||
TC(
|
||||
tc(
|
||||
"2000/2000T02", 4, 15,
|
||||
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", 0, ir("2000", 1), ir("2000T00:01", 2), ir("2000T01", 8)),
|
||||
ds("2000/2000T02", "v1", 1, ir("2000T01:01", 4))
|
||||
) /* Segment set with two segments for the same interval */,
|
||||
TC(
|
||||
tc(
|
||||
"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 */,
|
||||
TC(
|
||||
tc(
|
||||
"2000T02/2000T04", 2, 12,
|
||||
DS("2000/2000T03", "v1", 0, IR("2000", 1), IR("2000T01", 2), IR("2000T02", 4)),
|
||||
DS("2000T03/2000T04", "v1", 0, IR("2000T03", 8))
|
||||
ds("2000/2000T03", "v1", 0, ir("2000", 1), ir("2000T01", 2), ir("2000T02", 4)),
|
||||
ds("2000T03/2000T04", "v1", 0, ir("2000T03", 8))
|
||||
) /* Segment intersecting desired interval */
|
||||
);
|
||||
|
||||
final List<Object[]> constructors = new ArrayList<>();
|
||||
|
||||
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
|
||||
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
|
||||
final SegmentListUsedAction action = (SegmentListUsedAction) taskAction;
|
||||
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);
|
||||
// Expect the interval we asked for
|
||||
if (intervals.equals(ImmutableList.of(testCase.interval))) {
|
||||
return ImmutableList.copyOf(testCase.segments);
|
||||
} 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(
|
||||
DATA_SOURCE,
|
||||
testCase.interval,
|
||||
new TrueDimFilter(),
|
||||
Arrays.asList(DIMENSIONS),
|
||||
Arrays.asList(METRICS),
|
||||
INDEX_IO
|
||||
INDEX_IO,
|
||||
cc,
|
||||
slf,
|
||||
retryPolicyFactory
|
||||
);
|
||||
factory.setTaskToolbox(taskToolboxFactory.build(NoopTask.create(DATA_SOURCE)));
|
||||
|
||||
constructors.add(
|
||||
new Object[]{
|
||||
|
|
|
@ -37,7 +37,6 @@ import org.apache.druid.segment.loading.NoopDataSegmentArchiver;
|
|||
import org.apache.druid.segment.loading.NoopDataSegmentKiller;
|
||||
import org.apache.druid.segment.loading.NoopDataSegmentMover;
|
||||
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.coordination.NoopDataSegmentAnnouncer;
|
||||
import org.apache.druid.server.initialization.ServerConfig;
|
||||
|
@ -94,7 +93,7 @@ public class SingleTaskBackgroundRunnerTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
new SegmentLoaderFactory(EasyMock.createMock(SegmentLoaderLocalCacheManager.class)),
|
||||
new SegmentLoaderFactory(null, utils.getTestObjectMapper()),
|
||||
utils.getTestObjectMapper(),
|
||||
utils.getTestIndexIO(),
|
||||
null,
|
||||
|
|
|
@ -106,7 +106,6 @@ import org.apache.druid.segment.loading.DataSegmentPusher;
|
|||
import org.apache.druid.segment.loading.LocalDataSegmentKiller;
|
||||
import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig;
|
||||
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.FireDepartment;
|
||||
import org.apache.druid.segment.realtime.FireDepartmentTest;
|
||||
|
@ -195,16 +194,16 @@ public class TaskLifecycleTest
|
|||
private static DateTime now = DateTimes.nowUtc();
|
||||
|
||||
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.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.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.TWO)).toString("YYYY-MM-dd'T'HH:mm:ss"), "test_dim1", "test_dim2", 3.0f)
|
||||
);
|
||||
|
||||
private static final Iterable<InputRow> IdxTaskInputRows = ImmutableList.of(
|
||||
IR("2010-01-01T01", "x", "y", 1),
|
||||
IR("2010-01-01T01", "x", "z", 1),
|
||||
IR("2010-01-02T01", "a", "b", 2),
|
||||
IR("2010-01-02T01", "a", "c", 1)
|
||||
ir("2010-01-01T01", "x", "y", 1),
|
||||
ir("2010-01-01T01", "x", "z", 1),
|
||||
ir("2010-01-02T01", "a", "b", 2),
|
||||
ir("2010-01-02T01", "a", "c", 1)
|
||||
);
|
||||
|
||||
@Rule
|
||||
|
@ -241,7 +240,7 @@ public class TaskLifecycleTest
|
|||
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(
|
||||
DateTimes.of(dt).getMillis(),
|
||||
|
@ -611,9 +610,7 @@ public class TaskLifecycleTest
|
|||
() -> queryRunnerFactoryConglomerate, // query runner factory conglomerate corporation unionized collective
|
||||
Execs.directExecutor(), // query executor service
|
||||
monitorScheduler, // monitor scheduler
|
||||
new SegmentLoaderFactory(
|
||||
new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, new DefaultObjectMapper())
|
||||
),
|
||||
new SegmentLoaderFactory(null, new DefaultObjectMapper()),
|
||||
MAPPER,
|
||||
INDEX_IO,
|
||||
MapCache.create(0),
|
||||
|
|
|
@ -41,7 +41,6 @@ import org.apache.druid.indexing.overlord.TestTaskRunner;
|
|||
import org.apache.druid.segment.IndexIO;
|
||||
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.server.coordination.ChangeRequestHistory;
|
||||
|
@ -120,7 +119,7 @@ public class WorkerTaskManagerTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
new SegmentLoaderFactory(new SegmentLoaderLocalCacheManager(null, loaderConfig, jsonMapper)),
|
||||
new SegmentLoaderFactory(null, jsonMapper),
|
||||
jsonMapper,
|
||||
indexIO,
|
||||
null,
|
||||
|
|
|
@ -46,9 +46,6 @@ import org.apache.druid.indexing.overlord.TestRemoteTaskRunnerConfig;
|
|||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.segment.IndexIO;
|
||||
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.server.DruidNode;
|
||||
import org.apache.druid.server.initialization.IndexerZkConfig;
|
||||
|
@ -62,10 +59,10 @@ import org.junit.Assert;
|
|||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class WorkerTaskMonitorTest
|
||||
{
|
||||
|
@ -169,20 +166,8 @@ public class WorkerTaskMonitorTest
|
|||
new TaskToolboxFactory(
|
||||
taskConfig,
|
||||
taskActionClientFactory,
|
||||
null, null, null, null, null, null, null, notifierFactory, null, null, null, new SegmentLoaderFactory(
|
||||
new SegmentLoaderLocalCacheManager(
|
||||
null,
|
||||
new SegmentLoaderConfig()
|
||||
{
|
||||
@Override
|
||||
public List<StorageLocationConfig> getLocations()
|
||||
{
|
||||
return new ArrayList<>();
|
||||
}
|
||||
},
|
||||
jsonMapper
|
||||
)
|
||||
),
|
||||
null, null, null, null, null, null, null, notifierFactory, null, null, null,
|
||||
new SegmentLoaderFactory(null, jsonMapper),
|
||||
jsonMapper,
|
||||
indexIO,
|
||||
null,
|
||||
|
|
|
@ -42,6 +42,7 @@ import org.apache.druid.segment.column.ColumnCapabilities;
|
|||
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
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.data.IndexedInts;
|
||||
import org.apache.druid.segment.serde.ComplexMetricSerde;
|
||||
|
@ -194,30 +195,38 @@ public class SegmentAnalyzer
|
|||
final ColumnHolder columnHolder
|
||||
)
|
||||
{
|
||||
long size = 0;
|
||||
|
||||
Comparable min = null;
|
||||
Comparable max = null;
|
||||
long size = 0;
|
||||
final int cardinality;
|
||||
if (capabilities.hasBitmapIndexes()) {
|
||||
final BitmapIndex bitmapIndex = columnHolder.getBitmapIndex();
|
||||
cardinality = bitmapIndex.getCardinality();
|
||||
|
||||
if (!capabilities.hasBitmapIndexes()) {
|
||||
return ColumnAnalysis.error("string_no_bitmap");
|
||||
}
|
||||
|
||||
final BitmapIndex bitmapIndex = columnHolder.getBitmapIndex();
|
||||
final int cardinality = bitmapIndex.getCardinality();
|
||||
|
||||
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 (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) {
|
||||
min = NullHandling.nullToEmptyIfNeeded(bitmapIndex.getValue(0));
|
||||
max = NullHandling.nullToEmptyIfNeeded(bitmapIndex.getValue(cardinality - 1));
|
||||
if (analyzingMinMax() && cardinality > 0) {
|
||||
min = NullHandling.nullToEmptyIfNeeded(bitmapIndex.getValue(0));
|
||||
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(
|
||||
|
|
|
@ -394,7 +394,7 @@ public class GroupByLimitPushDownInsufficientBufferTest
|
|||
strategySelector,
|
||||
new GroupByQueryQueryToolChest(
|
||||
strategySelector,
|
||||
NoopIntervalChunkingQueryRunnerDecorator()
|
||||
noopIntervalChunkingQueryRunnerDecorator()
|
||||
)
|
||||
);
|
||||
|
||||
|
@ -402,7 +402,7 @@ public class GroupByLimitPushDownInsufficientBufferTest
|
|||
tooSmallStrategySelector,
|
||||
new GroupByQueryQueryToolChest(
|
||||
tooSmallStrategySelector,
|
||||
NoopIntervalChunkingQueryRunnerDecorator()
|
||||
noopIntervalChunkingQueryRunnerDecorator()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
@ -679,7 +679,7 @@ public class GroupByLimitPushDownInsufficientBufferTest
|
|||
}
|
||||
};
|
||||
|
||||
public static IntervalChunkingQueryRunnerDecorator NoopIntervalChunkingQueryRunnerDecorator()
|
||||
public static IntervalChunkingQueryRunnerDecorator noopIntervalChunkingQueryRunnerDecorator()
|
||||
{
|
||||
return new IntervalChunkingQueryRunnerDecorator(null, null, null)
|
||||
{
|
||||
|
|
|
@ -426,7 +426,7 @@ public class GroupByLimitPushDownMultiNodeMergeTest
|
|||
strategySelector,
|
||||
new GroupByQueryQueryToolChest(
|
||||
strategySelector,
|
||||
NoopIntervalChunkingQueryRunnerDecorator()
|
||||
noopIntervalChunkingQueryRunnerDecorator()
|
||||
)
|
||||
);
|
||||
|
||||
|
@ -434,7 +434,7 @@ public class GroupByLimitPushDownMultiNodeMergeTest
|
|||
strategySelector2,
|
||||
new GroupByQueryQueryToolChest(
|
||||
strategySelector2,
|
||||
NoopIntervalChunkingQueryRunnerDecorator()
|
||||
noopIntervalChunkingQueryRunnerDecorator()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
@ -780,7 +780,7 @@ public class GroupByLimitPushDownMultiNodeMergeTest
|
|||
}
|
||||
};
|
||||
|
||||
public static IntervalChunkingQueryRunnerDecorator NoopIntervalChunkingQueryRunnerDecorator()
|
||||
public static IntervalChunkingQueryRunnerDecorator noopIntervalChunkingQueryRunnerDecorator()
|
||||
{
|
||||
return new IntervalChunkingQueryRunnerDecorator(null, null, null)
|
||||
{
|
||||
|
|
|
@ -287,7 +287,7 @@ public class GroupByMultiSegmentTest
|
|||
strategySelector,
|
||||
new GroupByQueryQueryToolChest(
|
||||
strategySelector,
|
||||
NoopIntervalChunkingQueryRunnerDecorator()
|
||||
noopIntervalChunkingQueryRunnerDecorator()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
@ -419,7 +419,7 @@ public class GroupByMultiSegmentTest
|
|||
}
|
||||
};
|
||||
|
||||
public static IntervalChunkingQueryRunnerDecorator NoopIntervalChunkingQueryRunnerDecorator()
|
||||
public static IntervalChunkingQueryRunnerDecorator noopIntervalChunkingQueryRunnerDecorator()
|
||||
{
|
||||
return new IntervalChunkingQueryRunnerDecorator(null, null, null) {
|
||||
@Override
|
||||
|
|
|
@ -36,7 +36,7 @@ public class LookupConfigTest
|
|||
public TemporaryFolder temporaryFolder = new TemporaryFolder();
|
||||
|
||||
@Test
|
||||
public void TestSerDesr() throws IOException
|
||||
public void testSerDesr() throws IOException
|
||||
{
|
||||
LookupConfig lookupConfig = new LookupConfig(temporaryFolder.newFile().getAbsolutePath());
|
||||
Assert.assertEquals(
|
||||
|
|
|
@ -83,10 +83,16 @@ public class SegmentMetadataQueryTest
|
|||
public static QueryRunner makeMMappedQueryRunner(
|
||||
SegmentId segmentId,
|
||||
boolean rollup,
|
||||
boolean bitmaps,
|
||||
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(
|
||||
factory,
|
||||
segmentId,
|
||||
|
@ -99,10 +105,16 @@ public class SegmentMetadataQueryTest
|
|||
public static QueryRunner makeIncrementalIndexQueryRunner(
|
||||
SegmentId segmentId,
|
||||
boolean rollup,
|
||||
boolean bitmaps,
|
||||
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(
|
||||
factory,
|
||||
segmentId,
|
||||
|
@ -121,17 +133,19 @@ public class SegmentMetadataQueryTest
|
|||
private final SegmentMetadataQuery testQuery;
|
||||
private final SegmentAnalysis expectedSegmentAnalysis1;
|
||||
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()
|
||||
{
|
||||
return ImmutableList.of(
|
||||
new Object[]{true, true, true, true, false},
|
||||
new Object[]{true, false, true, false, false},
|
||||
new Object[]{false, true, true, false, false},
|
||||
new Object[]{false, false, false, false, false},
|
||||
new Object[]{false, false, true, true, false},
|
||||
new Object[]{false, false, false, true, true}
|
||||
new Object[]{true, true, true, true, false, true},
|
||||
new Object[]{true, false, true, false, false, true},
|
||||
new Object[]{false, true, true, false, false, true},
|
||||
new Object[]{false, false, false, false, false, true},
|
||||
new Object[]{false, false, true, true, false, 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 rollup1,
|
||||
boolean rollup2,
|
||||
boolean differentIds
|
||||
boolean differentIds,
|
||||
boolean bitmaps
|
||||
)
|
||||
{
|
||||
final SegmentId id1 = SegmentId.dummy(differentIds ? "testSegment1" : "testSegment");
|
||||
final SegmentId id2 = SegmentId.dummy(differentIds ? "testSegment2" : "testSegment");
|
||||
this.runner1 = mmap1
|
||||
? makeMMappedQueryRunner(id1, rollup1, FACTORY)
|
||||
: makeIncrementalIndexQueryRunner(id1, rollup1, FACTORY);
|
||||
? makeMMappedQueryRunner(id1, rollup1, bitmaps, FACTORY)
|
||||
: makeIncrementalIndexQueryRunner(id1, rollup1, bitmaps, FACTORY);
|
||||
this.runner2 = mmap2
|
||||
? makeMMappedQueryRunner(id2, rollup2, FACTORY)
|
||||
: makeIncrementalIndexQueryRunner(id2, rollup2, FACTORY);
|
||||
? makeMMappedQueryRunner(id2, rollup2, bitmaps, FACTORY)
|
||||
: makeIncrementalIndexQueryRunner(id2, rollup2, bitmaps, FACTORY);
|
||||
this.mmap1 = mmap1;
|
||||
this.mmap2 = mmap2;
|
||||
this.rollup1 = rollup1;
|
||||
this.rollup2 = rollup2;
|
||||
this.differentIds = differentIds;
|
||||
this.bitmaps = bitmaps;
|
||||
testQuery = Druids.newSegmentMetadataQueryBuilder()
|
||||
.dataSource("testing")
|
||||
.intervals("2013/2014")
|
||||
|
@ -169,6 +185,16 @@ public class SegmentMetadataQueryTest
|
|||
.merge(true)
|
||||
.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(
|
||||
id1.toString(),
|
||||
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(
|
||||
ValueType.STRING.toString(),
|
||||
false,
|
||||
mmap1 ? 10881 : 10764,
|
||||
preferedSize1,
|
||||
1,
|
||||
"preferred",
|
||||
"preferred",
|
||||
|
@ -203,7 +229,7 @@ public class SegmentMetadataQueryTest
|
|||
null,
|
||||
null
|
||||
)
|
||||
), mmap1 ? 167493 : 168188,
|
||||
), overallSize1,
|
||||
1209,
|
||||
null,
|
||||
null,
|
||||
|
@ -228,7 +254,7 @@ public class SegmentMetadataQueryTest
|
|||
new ColumnAnalysis(
|
||||
ValueType.STRING.toString(),
|
||||
false,
|
||||
mmap2 ? 10881 : 0,
|
||||
placementSize2,
|
||||
1,
|
||||
null,
|
||||
null,
|
||||
|
@ -245,7 +271,7 @@ public class SegmentMetadataQueryTest
|
|||
null
|
||||
)
|
||||
// null_column will be included only for incremental index, which makes a little bigger result than expected
|
||||
), mmap2 ? 167493 : 168188,
|
||||
), overallSize2,
|
||||
1209,
|
||||
null,
|
||||
null,
|
||||
|
@ -470,10 +496,16 @@ public class SegmentMetadataQueryTest
|
|||
@Test
|
||||
public void testSegmentMetadataQueryWithDefaultAnalysisMerge()
|
||||
{
|
||||
int size1 = 0;
|
||||
int size2 = 0;
|
||||
if (bitmaps) {
|
||||
size1 = mmap1 ? 10881 : 10764;
|
||||
size2 = mmap2 ? 10881 : 10764;
|
||||
}
|
||||
ColumnAnalysis analysis = new ColumnAnalysis(
|
||||
ValueType.STRING.toString(),
|
||||
false,
|
||||
(mmap1 ? 10881 : 10764) + (mmap2 ? 10881 : 10764),
|
||||
size1 + size2,
|
||||
1,
|
||||
"preferred",
|
||||
"preferred",
|
||||
|
@ -485,10 +517,16 @@ public class SegmentMetadataQueryTest
|
|||
@Test
|
||||
public void testSegmentMetadataQueryWithDefaultAnalysisMerge2()
|
||||
{
|
||||
int size1 = 0;
|
||||
int size2 = 0;
|
||||
if (bitmaps) {
|
||||
size1 = mmap1 ? 6882 : 6808;
|
||||
size2 = mmap2 ? 6882 : 6808;
|
||||
}
|
||||
ColumnAnalysis analysis = new ColumnAnalysis(
|
||||
ValueType.STRING.toString(),
|
||||
false,
|
||||
(mmap1 ? 6882 : 6808) + (mmap2 ? 6882 : 6808),
|
||||
size1 + size2,
|
||||
3,
|
||||
"spot",
|
||||
"upfront",
|
||||
|
@ -500,10 +538,16 @@ public class SegmentMetadataQueryTest
|
|||
@Test
|
||||
public void testSegmentMetadataQueryWithDefaultAnalysisMerge3()
|
||||
{
|
||||
int size1 = 0;
|
||||
int size2 = 0;
|
||||
if (bitmaps) {
|
||||
size1 = mmap1 ? 9765 : 9660;
|
||||
size2 = mmap2 ? 9765 : 9660;
|
||||
}
|
||||
ColumnAnalysis analysis = new ColumnAnalysis(
|
||||
ValueType.STRING.toString(),
|
||||
false,
|
||||
(mmap1 ? 9765 : 9660) + (mmap2 ? 9765 : 9660),
|
||||
size1 + size2,
|
||||
9,
|
||||
"automotive",
|
||||
"travel",
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.segment;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Suppliers;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.io.CharSource;
|
||||
import com.google.common.io.LineProcessor;
|
||||
|
@ -111,12 +112,31 @@ public class TestIndex
|
|||
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(
|
||||
DIMENSION_SCHEMAS,
|
||||
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[] FLOAT_METRICS = new String[]{"indexFloat", "indexMinFloat", "indexMaxFloat"};
|
||||
private static final Logger log = new Logger(TestIndex.class);
|
||||
|
@ -147,107 +167,95 @@ public class TestIndex
|
|||
}
|
||||
}
|
||||
|
||||
private static IncrementalIndex realtimeIndex = null;
|
||||
private static IncrementalIndex noRollupRealtimeIndex = null;
|
||||
private static QueryableIndex mmappedIndex = null;
|
||||
private static QueryableIndex noRollupMmappedIndex = null;
|
||||
private static QueryableIndex mergedRealtime = null;
|
||||
private static Supplier<IncrementalIndex> realtimeIndex = Suppliers.memoize(
|
||||
() -> makeRealtimeIndex("druid.sample.numeric.tsv")
|
||||
);
|
||||
private static Supplier<IncrementalIndex> noRollupRealtimeIndex = Suppliers.memoize(
|
||||
() -> 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()
|
||||
{
|
||||
synchronized (log) {
|
||||
if (realtimeIndex != null) {
|
||||
return realtimeIndex;
|
||||
}
|
||||
}
|
||||
|
||||
return realtimeIndex = makeRealtimeIndex("druid.sample.numeric.tsv");
|
||||
return realtimeIndex.get();
|
||||
}
|
||||
|
||||
public static IncrementalIndex getNoRollupIncrementalTestIndex()
|
||||
{
|
||||
synchronized (log) {
|
||||
if (noRollupRealtimeIndex != null) {
|
||||
return noRollupRealtimeIndex;
|
||||
}
|
||||
}
|
||||
return noRollupRealtimeIndex.get();
|
||||
}
|
||||
|
||||
return noRollupRealtimeIndex = makeRealtimeIndex("druid.sample.numeric.tsv", false);
|
||||
public static IncrementalIndex getNoBitmapIncrementalTestIndex()
|
||||
{
|
||||
return noBitmapRealtimeIndex.get();
|
||||
}
|
||||
|
||||
public static QueryableIndex getMMappedTestIndex()
|
||||
{
|
||||
synchronized (log) {
|
||||
if (mmappedIndex != null) {
|
||||
return mmappedIndex;
|
||||
}
|
||||
}
|
||||
|
||||
IncrementalIndex incrementalIndex = getIncrementalTestIndex();
|
||||
mmappedIndex = persistRealtimeAndLoadMMapped(incrementalIndex);
|
||||
|
||||
return mmappedIndex;
|
||||
return mmappedIndex.get();
|
||||
}
|
||||
|
||||
public static QueryableIndex getNoRollupMMappedTestIndex()
|
||||
{
|
||||
synchronized (log) {
|
||||
if (noRollupMmappedIndex != null) {
|
||||
return noRollupMmappedIndex;
|
||||
}
|
||||
}
|
||||
return noRollupMmappedIndex.get();
|
||||
}
|
||||
|
||||
IncrementalIndex incrementalIndex = getNoRollupIncrementalTestIndex();
|
||||
noRollupMmappedIndex = persistRealtimeAndLoadMMapped(incrementalIndex);
|
||||
|
||||
return noRollupMmappedIndex;
|
||||
public static QueryableIndex getNoBitmapMMappedTestIndex()
|
||||
{
|
||||
return noBitmapMmappedIndex.get();
|
||||
}
|
||||
|
||||
public static QueryableIndex mergedRealtimeIndex()
|
||||
{
|
||||
synchronized (log) {
|
||||
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);
|
||||
}
|
||||
}
|
||||
return mergedRealtime.get();
|
||||
}
|
||||
|
||||
public static IncrementalIndex makeRealtimeIndex(final String resourceFilename)
|
||||
|
@ -256,6 +264,11 @@ public class TestIndex
|
|||
}
|
||||
|
||||
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);
|
||||
if (resource == null) {
|
||||
|
@ -263,20 +276,20 @@ public class TestIndex
|
|||
}
|
||||
log.info("Realtime loading index file[%s]", resource);
|
||||
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)
|
||||
{
|
||||
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()
|
||||
.withMinTimestamp(DateTimes.of("2011-01-12T00:00:00.000Z").getMillis())
|
||||
.withTimestampSpec(new TimestampSpec("ds", "auto", null))
|
||||
.withDimensionsSpec(DIMENSIONS_SPEC)
|
||||
.withDimensionsSpec(bitmap ? DIMENSIONS_SPEC : DIMENSIONS_SPEC_NO_BITMAPS)
|
||||
.withVirtualColumns(VIRTUAL_COLUMNS)
|
||||
.withMetrics(METRIC_AGGS)
|
||||
.withRollup(rollup)
|
||||
|
|
|
@ -117,22 +117,22 @@ public class ExpressionFilterTest extends BaseFilterTest
|
|||
@Test
|
||||
public void testOneSingleValuedStringColumn()
|
||||
{
|
||||
assertFilterMatches(EDF("dim3 == ''"), ImmutableList.of("0"));
|
||||
assertFilterMatches(EDF("dim3 == '1'"), ImmutableList.of("3", "4", "6"));
|
||||
assertFilterMatches(EDF("dim3 == 'a'"), ImmutableList.of("7"));
|
||||
assertFilterMatches(EDF("dim3 == 1"), 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 < '2'"), ImmutableList.of("0", "1", "3", "4", "6", "9"));
|
||||
assertFilterMatches(edf("dim3 == ''"), ImmutableList.of("0"));
|
||||
assertFilterMatches(edf("dim3 == '1'"), ImmutableList.of("3", "4", "6"));
|
||||
assertFilterMatches(edf("dim3 == 'a'"), ImmutableList.of("7"));
|
||||
assertFilterMatches(edf("dim3 == 1"), 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 < '2'"), ImmutableList.of("0", "1", "3", "4", "6", "9"));
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
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"), ImmutableList.of("0", "3", "4", "6", "7", "9"));
|
||||
assertFilterMatches(edf("dim3 < 2.0"), ImmutableList.of("0", "3", "4", "6", "7", "9"));
|
||||
} else {
|
||||
// Empty String and "a" will not match
|
||||
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"), 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
|
||||
|
@ -141,124 +141,124 @@ public class ExpressionFilterTest extends BaseFilterTest
|
|||
// Expressions currently treat multi-valued arrays as nulls.
|
||||
// This test is just documenting the current behavior, not necessarily saying it makes sense.
|
||||
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 {
|
||||
assertFilterMatches(EDF("dim4 == ''"), ImmutableList.of("2"));
|
||||
assertFilterMatches(edf("dim4 == ''"), ImmutableList.of("2"));
|
||||
// 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 == '3'"), ImmutableList.of("3"));
|
||||
assertFilterMatches(edf("dim4 == '1'"), ImmutableList.of());
|
||||
assertFilterMatches(edf("dim4 == '3'"), ImmutableList.of("3"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOneLongColumn()
|
||||
{
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
assertFilterMatches(EDF("dim1 == ''"), ImmutableList.of("0"));
|
||||
assertFilterMatches(edf("dim1 == ''"), ImmutableList.of("0"));
|
||||
} else {
|
||||
// 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 == 2"), ImmutableList.of("2"));
|
||||
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("like(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("0", "1"));
|
||||
assertFilterMatches(edf("dim1 < 2"), ImmutableList.of("0", "1"));
|
||||
assertFilterMatches(edf("dim1 < 2.0"), ImmutableList.of("0", "1"));
|
||||
assertFilterMatches(edf("like(dim1, '1%')"), ImmutableList.of("1"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOneFloatColumn()
|
||||
{
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
assertFilterMatches(EDF("dim2 == ''"), ImmutableList.of("0"));
|
||||
assertFilterMatches(edf("dim2 == ''"), ImmutableList.of("0"));
|
||||
} else {
|
||||
// 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 == 2"), ImmutableList.of("2"));
|
||||
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("like(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("0", "1"));
|
||||
assertFilterMatches(edf("dim2 < 2"), ImmutableList.of("0", "1"));
|
||||
assertFilterMatches(edf("dim2 < 2.0"), ImmutableList.of("0", "1"));
|
||||
assertFilterMatches(edf("like(dim2, '1%')"), ImmutableList.of("1"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConstantExpression()
|
||||
{
|
||||
assertFilterMatches(EDF("1 + 1"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
|
||||
assertFilterMatches(EDF("0 + 0"), ImmutableList.of());
|
||||
assertFilterMatches(edf("1 + 1"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
|
||||
assertFilterMatches(edf("0 + 0"), ImmutableList.of());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompareColumns()
|
||||
{
|
||||
// String vs string
|
||||
assertFilterMatches(EDF("dim0 == dim3"), ImmutableList.of("2", "5", "8"));
|
||||
assertFilterMatches(edf("dim0 == dim3"), ImmutableList.of("2", "5", "8"));
|
||||
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
// 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
|
||||
assertFilterMatches(EDF("dim2 == dim3"), ImmutableList.of("0", "2", "5", "8"));
|
||||
assertFilterMatches(edf("dim2 == dim3"), ImmutableList.of("0", "2", "5", "8"));
|
||||
} else {
|
||||
// String vs long
|
||||
assertFilterMatches(EDF("dim1 == dim3"), ImmutableList.of("2", "5", "8"));
|
||||
assertFilterMatches(edf("dim1 == dim3"), ImmutableList.of("2", "5", "8"));
|
||||
|
||||
// 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
|
||||
// Expressions currently treat multi-valued arrays as nulls.
|
||||
// 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
|
||||
public void testMissingColumn()
|
||||
{
|
||||
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 {
|
||||
// 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 == 2"), ImmutableList.of());
|
||||
assertFilterMatches(edf("missing == '1'"), ImmutableList.of());
|
||||
assertFilterMatches(edf("missing == 2"), ImmutableList.of());
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
// 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.0"), 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"));
|
||||
} else {
|
||||
// missing equivalent to null
|
||||
assertFilterMatches(EDF("missing < '2'"), ImmutableList.of());
|
||||
assertFilterMatches(EDF("missing < 2"), 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.0"), ImmutableList.of());
|
||||
}
|
||||
assertFilterMatches(EDF("missing > '2'"), ImmutableList.of());
|
||||
assertFilterMatches(EDF("missing > 2"), ImmutableList.of());
|
||||
assertFilterMatches(EDF("missing > 2.0"), ImmutableList.of());
|
||||
assertFilterMatches(EDF("like(missing, '1%')"), ImmutableList.of());
|
||||
assertFilterMatches(edf("missing > '2'"), ImmutableList.of());
|
||||
assertFilterMatches(edf("missing > 2"), ImmutableList.of());
|
||||
assertFilterMatches(edf("missing > 2.0"), ImmutableList.of());
|
||||
assertFilterMatches(edf("like(missing, '1%')"), ImmutableList.of());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetRequiredColumn()
|
||||
{
|
||||
Assert.assertEquals(EDF("like(dim1, '1%')").getRequiredColumns(), Sets.newHashSet("dim1"));
|
||||
Assert.assertEquals(EDF("dim2 == '1'").getRequiredColumns(), Sets.newHashSet("dim2"));
|
||||
Assert.assertEquals(EDF("dim3 < '2'").getRequiredColumns(), Sets.newHashSet("dim3"));
|
||||
Assert.assertEquals(EDF("dim4 == ''").getRequiredColumns(), Sets.newHashSet("dim4"));
|
||||
Assert.assertEquals(EDF("1 + 1").getRequiredColumns(), new HashSet<>());
|
||||
Assert.assertEquals(EDF("dim0 == dim3").getRequiredColumns(), Sets.newHashSet("dim0", "dim3"));
|
||||
Assert.assertEquals(EDF("missing == ''").getRequiredColumns(), Sets.newHashSet("missing"));
|
||||
Assert.assertEquals(edf("like(dim1, '1%')").getRequiredColumns(), Sets.newHashSet("dim1"));
|
||||
Assert.assertEquals(edf("dim2 == '1'").getRequiredColumns(), Sets.newHashSet("dim2"));
|
||||
Assert.assertEquals(edf("dim3 < '2'").getRequiredColumns(), Sets.newHashSet("dim3"));
|
||||
Assert.assertEquals(edf("dim4 == ''").getRequiredColumns(), Sets.newHashSet("dim4"));
|
||||
Assert.assertEquals(edf("1 + 1").getRequiredColumns(), new HashSet<>());
|
||||
Assert.assertEquals(edf("dim0 == dim3").getRequiredColumns(), Sets.newHashSet("dim0", "dim3"));
|
||||
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);
|
||||
}
|
||||
|
|
|
@ -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.http.client.response.FullResponseHolder;
|
||||
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.HttpResponseStatus;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import java.util.List;
|
||||
|
||||
public class CoordinatorClient
|
||||
|
@ -95,13 +97,15 @@ public class CoordinatorClient
|
|||
{
|
||||
try {
|
||||
FullResponseHolder response = druidLeaderClient.go(
|
||||
druidLeaderClient.makeRequest(HttpMethod.GET,
|
||||
StringUtils.format(
|
||||
"/druid/coordinator/v1/datasources/%s/intervals/%s/serverview?partial=%s",
|
||||
dataSource,
|
||||
interval.toString().replace('/', '_'),
|
||||
incompleteOk
|
||||
))
|
||||
druidLeaderClient.makeRequest(
|
||||
HttpMethod.GET,
|
||||
StringUtils.format(
|
||||
"/druid/coordinator/v1/datasources/%s/intervals/%s/serverview?partial=%s",
|
||||
StringUtils.urlEncode(dataSource),
|
||||
interval.toString().replace('/', '_'),
|
||||
incompleteOk
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
if (!response.getStatus().equals(HttpResponseStatus.OK)) {
|
||||
|
@ -121,4 +125,35 @@ public class CoordinatorClient
|
|||
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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
public SegmentLoaderLocalCacheManager(
|
||||
IndexIO indexIO,
|
||||
|
@ -79,11 +82,6 @@ public class SegmentLoaderLocalCacheManager implements SegmentLoader
|
|||
}
|
||||
}
|
||||
|
||||
public SegmentLoaderLocalCacheManager withConfig(SegmentLoaderConfig config)
|
||||
{
|
||||
return new SegmentLoaderLocalCacheManager(indexIO, config, jsonMapper);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isSegmentLoaded(final DataSegment segment)
|
||||
{
|
||||
|
|
|
@ -100,8 +100,8 @@ public class AppenderatorPlumberTest
|
|||
// getDataSource
|
||||
Assert.assertEquals(AppenderatorTester.DATASOURCE, appenderator.getDataSource());
|
||||
|
||||
InputRow[] rows = new InputRow[] {AppenderatorTest.IR("2000", "foo", 1),
|
||||
AppenderatorTest.IR("2000", "bar", 2), AppenderatorTest.IR("2000", "qux", 4)};
|
||||
InputRow[] rows = new InputRow[] {AppenderatorTest.ir("2000", "foo", 1),
|
||||
AppenderatorTest.ir("2000", "bar", 2), AppenderatorTest.ir("2000", "qux", 4)};
|
||||
// add
|
||||
Assert.assertEquals(1, plumber.add(rows[0], null).getRowCount());
|
||||
|
||||
|
|
|
@ -58,9 +58,9 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
public class AppenderatorTest
|
||||
{
|
||||
private static final List<SegmentIdWithShardSpec> IDENTIFIERS = ImmutableList.of(
|
||||
SI("2000/2001", "A", 0),
|
||||
SI("2000/2001", "A", 1),
|
||||
SI("2001/2002", "A", 0)
|
||||
si("2000/2001", "A", 0),
|
||||
si("2000/2001", "A", 1),
|
||||
si("2001/2002", "A", 0)
|
||||
);
|
||||
|
||||
@Test
|
||||
|
@ -83,21 +83,21 @@ public class AppenderatorTest
|
|||
commitMetadata.put("x", "1");
|
||||
Assert.assertEquals(
|
||||
1,
|
||||
appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier)
|
||||
appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier)
|
||||
.getNumRowsInSegment()
|
||||
);
|
||||
|
||||
commitMetadata.put("x", "2");
|
||||
Assert.assertEquals(
|
||||
2,
|
||||
appenderator.add(IDENTIFIERS.get(0), IR("2000", "bar", 2), committerSupplier)
|
||||
appenderator.add(IDENTIFIERS.get(0), ir("2000", "bar", 2), committerSupplier)
|
||||
.getNumRowsInSegment()
|
||||
);
|
||||
|
||||
commitMetadata.put("x", "3");
|
||||
Assert.assertEquals(
|
||||
1,
|
||||
appenderator.add(IDENTIFIERS.get(1), IR("2000", "qux", 4), committerSupplier)
|
||||
appenderator.add(IDENTIFIERS.get(1), ir("2000", "qux", 4), committerSupplier)
|
||||
.getNumRowsInSegment()
|
||||
);
|
||||
|
||||
|
@ -173,14 +173,14 @@ public class AppenderatorTest
|
|||
};
|
||||
|
||||
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
|
||||
int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0;
|
||||
Assert.assertEquals(
|
||||
138 + nullHandlingOverhead,
|
||||
((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(
|
||||
138 + nullHandlingOverhead,
|
||||
((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(1))
|
||||
|
@ -216,11 +216,11 @@ public class AppenderatorTest
|
|||
};
|
||||
|
||||
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
|
||||
int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0;
|
||||
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(
|
||||
276 + 2 * nullHandlingOverhead,
|
||||
((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory()
|
||||
|
@ -258,7 +258,7 @@ public class AppenderatorTest
|
|||
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
|
||||
appenderator.startJob();
|
||||
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
|
||||
int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0;
|
||||
Assert.assertEquals(
|
||||
|
@ -266,7 +266,7 @@ public class AppenderatorTest
|
|||
((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(0))
|
||||
);
|
||||
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(
|
||||
276 + 2 * nullHandlingOverhead,
|
||||
((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory()
|
||||
|
@ -310,17 +310,17 @@ public class AppenderatorTest
|
|||
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
|
||||
appenderator.startJob();
|
||||
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());
|
||||
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());
|
||||
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());
|
||||
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());
|
||||
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());
|
||||
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());
|
||||
appenderator.persistAll(committerSupplier.get());
|
||||
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
|
||||
|
@ -356,17 +356,17 @@ public class AppenderatorTest
|
|||
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
|
||||
appenderator.startJob();
|
||||
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());
|
||||
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());
|
||||
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());
|
||||
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());
|
||||
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());
|
||||
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());
|
||||
appenderator.persistAll(committerSupplier.get());
|
||||
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
|
||||
|
@ -409,15 +409,15 @@ public class AppenderatorTest
|
|||
|
||||
appenderator.startJob();
|
||||
eventCount.incrementAndGet();
|
||||
appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier);
|
||||
appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier);
|
||||
eventCount.incrementAndGet();
|
||||
appenderator.add(IDENTIFIERS.get(0), IR("2000", "bar", 2), committerSupplier);
|
||||
appenderator.add(IDENTIFIERS.get(0), ir("2000", "bar", 2), committerSupplier);
|
||||
eventCount.incrementAndGet();
|
||||
appenderator.add(IDENTIFIERS.get(0), IR("2000", "baz", 3), committerSupplier);
|
||||
appenderator.add(IDENTIFIERS.get(0), ir("2000", "baz", 3), committerSupplier);
|
||||
eventCount.incrementAndGet();
|
||||
appenderator.add(IDENTIFIERS.get(0), IR("2000", "qux", 4), committerSupplier);
|
||||
appenderator.add(IDENTIFIERS.get(0), ir("2000", "qux", 4), committerSupplier);
|
||||
eventCount.incrementAndGet();
|
||||
appenderator.add(IDENTIFIERS.get(0), IR("2000", "bob", 5), committerSupplier);
|
||||
appenderator.add(IDENTIFIERS.get(0), ir("2000", "bob", 5), committerSupplier);
|
||||
appenderator.close();
|
||||
|
||||
try (final AppenderatorTester tester2 = new AppenderatorTester(
|
||||
|
@ -445,9 +445,9 @@ public class AppenderatorTest
|
|||
Assert.assertEquals(0, appenderator.getTotalRowCount());
|
||||
appenderator.startJob();
|
||||
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());
|
||||
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());
|
||||
|
||||
appenderator.persistAll(committerSupplier.get()).get();
|
||||
|
@ -457,13 +457,13 @@ public class AppenderatorTest
|
|||
appenderator.drop(IDENTIFIERS.get(1)).get();
|
||||
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());
|
||||
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());
|
||||
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());
|
||||
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());
|
||||
|
||||
appenderator.persistAll(committerSupplier.get()).get();
|
||||
|
@ -483,13 +483,13 @@ public class AppenderatorTest
|
|||
final Appenderator appenderator = tester.getAppenderator();
|
||||
|
||||
appenderator.startJob();
|
||||
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(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("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("2001T03", "foo", 64), 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(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("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("2001T03", "foo", 64), Suppliers.ofInstance(Committers.nil()));
|
||||
|
||||
// Query1: 2000/2001
|
||||
final TimeseriesQuery query1 = Druids.newTimeseriesQueryBuilder()
|
||||
|
@ -619,13 +619,13 @@ public class AppenderatorTest
|
|||
final Appenderator appenderator = tester.getAppenderator();
|
||||
|
||||
appenderator.startJob();
|
||||
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(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("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("2001T03", "foo", 64), 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(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("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("2001T03", "foo", 64), Suppliers.ofInstance(Committers.nil()));
|
||||
|
||||
// Query1: segment #2
|
||||
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(
|
||||
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(
|
||||
DateTimes.of(ts).getMillis(),
|
||||
|
|
|
@ -167,9 +167,9 @@ public class DefaultOfflineAppenderatorFactoryTest
|
|||
new LinearShardSpec(0)
|
||||
);
|
||||
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());
|
||||
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());
|
||||
appenderator.close();
|
||||
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
|
||||
|
|
|
@ -91,7 +91,6 @@ import org.apache.druid.segment.loading.DataSegmentMover;
|
|||
import org.apache.druid.segment.loading.OmniDataSegmentArchiver;
|
||||
import org.apache.druid.segment.loading.OmniDataSegmentKiller;
|
||||
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.NoopChatHandlerProvider;
|
||||
import org.apache.druid.segment.realtime.firehose.ServiceAnnouncingChatHandlerProvider;
|
||||
|
@ -109,7 +108,6 @@ import org.eclipse.jetty.server.Server;
|
|||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
|
@ -255,12 +253,6 @@ public class CliPeon extends GuiceRunnable
|
|||
.to(CoordinatorBasedSegmentHandoffNotifierFactory.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(JettyServerInitializer.class).to(QueryJettyServerInitializer.class);
|
||||
|
|
|
@ -66,6 +66,7 @@ public class RouterJettyServerInitializer implements JettyServerInitializer
|
|||
"/",
|
||||
"/coordinator-console/*",
|
||||
"/public/*",
|
||||
"/assets/*",
|
||||
"/old-console/*",
|
||||
"/pages/*",
|
||||
"/unified-console.html",
|
||||
|
|
|
@ -362,7 +362,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
final DatabaseMetaData metaData = client.getMetaData();
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
ROW(Pair.of("TABLE_CAT", "druid"))
|
||||
row(Pair.of("TABLE_CAT", "druid"))
|
||||
),
|
||||
getRows(metaData.getCatalogs())
|
||||
);
|
||||
|
@ -374,7 +374,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
final DatabaseMetaData metaData = client.getMetaData();
|
||||
Assert.assertEquals(
|
||||
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"))
|
||||
);
|
||||
|
@ -386,19 +386,19 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
final DatabaseMetaData metaData = client.getMetaData();
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
ROW(
|
||||
row(
|
||||
Pair.of("TABLE_CAT", "druid"),
|
||||
Pair.of("TABLE_NAME", CalciteTests.DATASOURCE1),
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_TYPE", "TABLE")
|
||||
),
|
||||
ROW(
|
||||
row(
|
||||
Pair.of("TABLE_CAT", "druid"),
|
||||
Pair.of("TABLE_NAME", CalciteTests.DATASOURCE2),
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_TYPE", "TABLE")
|
||||
),
|
||||
ROW(
|
||||
row(
|
||||
Pair.of("TABLE_CAT", "druid"),
|
||||
Pair.of("TABLE_NAME", CalciteTests.DATASOURCE3),
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
|
@ -418,25 +418,25 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
final DatabaseMetaData metaData = superuserClient.getMetaData();
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
ROW(
|
||||
row(
|
||||
Pair.of("TABLE_CAT", "druid"),
|
||||
Pair.of("TABLE_NAME", CalciteTests.DATASOURCE1),
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_TYPE", "TABLE")
|
||||
),
|
||||
ROW(
|
||||
row(
|
||||
Pair.of("TABLE_CAT", "druid"),
|
||||
Pair.of("TABLE_NAME", CalciteTests.DATASOURCE2),
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_TYPE", "TABLE")
|
||||
),
|
||||
ROW(
|
||||
row(
|
||||
Pair.of("TABLE_CAT", "druid"),
|
||||
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_TYPE", "TABLE")
|
||||
),
|
||||
ROW(
|
||||
row(
|
||||
Pair.of("TABLE_CAT", "druid"),
|
||||
Pair.of("TABLE_NAME", CalciteTests.DATASOURCE3),
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
|
@ -456,7 +456,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
final DatabaseMetaData metaData = client.getMetaData();
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
ROW(
|
||||
row(
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_NAME", "foo"),
|
||||
Pair.of("COLUMN_NAME", "__time"),
|
||||
|
@ -464,7 +464,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
Pair.of("TYPE_NAME", "TIMESTAMP"),
|
||||
Pair.of("IS_NULLABLE", "NO")
|
||||
),
|
||||
ROW(
|
||||
row(
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_NAME", "foo"),
|
||||
Pair.of("COLUMN_NAME", "cnt"),
|
||||
|
@ -472,7 +472,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
Pair.of("TYPE_NAME", "BIGINT"),
|
||||
Pair.of("IS_NULLABLE", "NO")
|
||||
),
|
||||
ROW(
|
||||
row(
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_NAME", "foo"),
|
||||
Pair.of("COLUMN_NAME", "dim1"),
|
||||
|
@ -480,7 +480,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
Pair.of("TYPE_NAME", "VARCHAR"),
|
||||
Pair.of("IS_NULLABLE", "YES")
|
||||
),
|
||||
ROW(
|
||||
row(
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_NAME", "foo"),
|
||||
Pair.of("COLUMN_NAME", "dim2"),
|
||||
|
@ -488,7 +488,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
Pair.of("TYPE_NAME", "VARCHAR"),
|
||||
Pair.of("IS_NULLABLE", "YES")
|
||||
),
|
||||
ROW(
|
||||
row(
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_NAME", "foo"),
|
||||
Pair.of("COLUMN_NAME", "dim3"),
|
||||
|
@ -496,7 +496,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
Pair.of("TYPE_NAME", "VARCHAR"),
|
||||
Pair.of("IS_NULLABLE", "YES")
|
||||
),
|
||||
ROW(
|
||||
row(
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_NAME", "foo"),
|
||||
Pair.of("COLUMN_NAME", "m1"),
|
||||
|
@ -504,7 +504,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
Pair.of("TYPE_NAME", "FLOAT"),
|
||||
Pair.of("IS_NULLABLE", "NO")
|
||||
),
|
||||
ROW(
|
||||
row(
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_NAME", "foo"),
|
||||
Pair.of("COLUMN_NAME", "m2"),
|
||||
|
@ -512,7 +512,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
Pair.of("TYPE_NAME", "DOUBLE"),
|
||||
Pair.of("IS_NULLABLE", "NO")
|
||||
),
|
||||
ROW(
|
||||
row(
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_NAME", "foo"),
|
||||
Pair.of("COLUMN_NAME", "unique_dim1"),
|
||||
|
@ -547,7 +547,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
final DatabaseMetaData metaData = superuserClient.getMetaData();
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
ROW(
|
||||
row(
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
|
||||
Pair.of("COLUMN_NAME", "__time"),
|
||||
|
@ -555,7 +555,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
Pair.of("TYPE_NAME", "TIMESTAMP"),
|
||||
Pair.of("IS_NULLABLE", "NO")
|
||||
),
|
||||
ROW(
|
||||
row(
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
|
||||
Pair.of("COLUMN_NAME", "cnt"),
|
||||
|
@ -563,7 +563,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
Pair.of("TYPE_NAME", "BIGINT"),
|
||||
Pair.of("IS_NULLABLE", "NO")
|
||||
),
|
||||
ROW(
|
||||
row(
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
|
||||
Pair.of("COLUMN_NAME", "dim1"),
|
||||
|
@ -571,7 +571,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
Pair.of("TYPE_NAME", "VARCHAR"),
|
||||
Pair.of("IS_NULLABLE", "YES")
|
||||
),
|
||||
ROW(
|
||||
row(
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
|
||||
Pair.of("COLUMN_NAME", "dim2"),
|
||||
|
@ -579,7 +579,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
Pair.of("TYPE_NAME", "VARCHAR"),
|
||||
Pair.of("IS_NULLABLE", "YES")
|
||||
),
|
||||
ROW(
|
||||
row(
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
|
||||
Pair.of("COLUMN_NAME", "m1"),
|
||||
|
@ -587,7 +587,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
Pair.of("TYPE_NAME", "FLOAT"),
|
||||
Pair.of("IS_NULLABLE", "NO")
|
||||
),
|
||||
ROW(
|
||||
row(
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
|
||||
Pair.of("COLUMN_NAME", "m2"),
|
||||
|
@ -595,7 +595,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
Pair.of("TYPE_NAME", "DOUBLE"),
|
||||
Pair.of("IS_NULLABLE", "NO")
|
||||
),
|
||||
ROW(
|
||||
row(
|
||||
Pair.of("TABLE_SCHEM", "druid"),
|
||||
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
|
||||
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<>();
|
||||
for (Pair<String, ?> entry : entries) {
|
||||
|
|
|
@ -243,70 +243,70 @@ public class BaseCalciteQueryTest extends CalciteTestBase
|
|||
}
|
||||
|
||||
// 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);
|
||||
}
|
||||
|
||||
// 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);
|
||||
return Calcites.jodaToCalciteTimestamp(new DateTime(timeString, timeZone), timeZone);
|
||||
}
|
||||
|
||||
// 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));
|
||||
}
|
||||
|
||||
public static AndDimFilter AND(DimFilter... filters)
|
||||
public static AndDimFilter and(DimFilter... 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));
|
||||
}
|
||||
|
||||
public static NotDimFilter NOT(DimFilter filter)
|
||||
public static NotDimFilter not(DimFilter 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);
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
public static ExpressionDimFilter EXPRESSION_FILTER(final String expression)
|
||||
public static ExpressionDimFilter expressionFilter(final String expression)
|
||||
{
|
||||
return new ExpressionDimFilter(expression, CalciteTests.createExprMacroTable());
|
||||
}
|
||||
|
||||
public static DimFilter NUMERIC_SELECTOR(
|
||||
public static DimFilter numeric_Selector(
|
||||
final String fieldName,
|
||||
final String value,
|
||||
final ExtractionFn extractionFn
|
||||
)
|
||||
{
|
||||
// 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 lower,
|
||||
final String upper,
|
||||
|
@ -319,7 +319,7 @@ public class BaseCalciteQueryTest extends CalciteTestBase
|
|||
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());
|
||||
return new BoundDimFilter(
|
||||
|
@ -329,32 +329,32 @@ public class BaseCalciteQueryTest extends CalciteTestBase
|
|||
false,
|
||||
true,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
StringComparators.NUMERIC
|
||||
);
|
||||
}
|
||||
|
||||
public static CascadeExtractionFn CASCADE(final ExtractionFn... fns)
|
||||
public static CascadeExtractionFn cascade(final ExtractionFn... 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);
|
||||
}
|
||||
|
||||
public static List<AggregatorFactory> AGGS(final AggregatorFactory... aggregators)
|
||||
public static List<AggregatorFactory> aggregators(final AggregatorFactory... aggregators)
|
||||
{
|
||||
return Arrays.asList(aggregators);
|
||||
}
|
||||
|
||||
public static DimFilterHavingSpec HAVING(final DimFilter filter)
|
||||
public static DimFilterHavingSpec having(final DimFilter filter)
|
||||
{
|
||||
return new DimFilterHavingSpec(filter, true);
|
||||
}
|
||||
|
||||
public static ExpressionVirtualColumn EXPRESSION_VIRTUAL_COLUMN(
|
||||
public static ExpressionVirtualColumn expression_Virtual_Column(
|
||||
final String name,
|
||||
final String expression,
|
||||
final ValueType outputType
|
||||
|
@ -363,7 +363,7 @@ public class BaseCalciteQueryTest extends CalciteTestBase
|
|||
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());
|
||||
}
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -2,6 +2,7 @@ node/
|
|||
node_modules/
|
||||
resources/
|
||||
public/
|
||||
assets/
|
||||
lib/*.css
|
||||
|
||||
coordinator-console/
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -17,9 +17,9 @@
|
|||
"start": "webpack-dev-server --hot --open"
|
||||
},
|
||||
"dependencies": {
|
||||
"@blueprintjs/core": "^3.12.0",
|
||||
"@types/hjson": "^2.4.0",
|
||||
"@blueprintjs/core": "1.0.1",
|
||||
"axios": "^0.18.0",
|
||||
"brace": "^0.11.1",
|
||||
"classnames": "^2.2.6",
|
||||
"d3-array": "^2.0.3",
|
||||
"druid-console": "^0.0.2",
|
||||
|
@ -28,16 +28,20 @@
|
|||
"hjson": "^3.1.2",
|
||||
"lodash.debounce": "^4.0.8",
|
||||
"numeral": "^2.0.6",
|
||||
"react": "^16.7.0",
|
||||
"react-dom": "^16.7.0",
|
||||
"prop-types": "^15.7.2",
|
||||
"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-dom": "^4.3.1",
|
||||
"react-table": "^6.8.6",
|
||||
"react-table": "^6.9.2",
|
||||
"tslib": "^1.9.3"
|
||||
},
|
||||
"devDependencies": {
|
||||
"@types/classnames": "^2.2.7",
|
||||
"@types/d3-array": "^1.2.4",
|
||||
"@types/hjson": "^2.4.0",
|
||||
"@types/jest": "^23.3.13",
|
||||
"@types/lodash.debounce": "^4.0.4",
|
||||
"@types/mocha": "^5.2.5",
|
||||
|
|
|
@ -23,6 +23,10 @@ cp -r ./node_modules/druid-console/coordinator-console .
|
|||
cp -r ./node_modules/druid-console/pages .
|
||||
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..."
|
||||
PATH="./target/node:$PATH" ./node_modules/.bin/stylus lib/react-table.styl -o lib/react-table.css
|
||||
|
||||
|
|
|
@ -17,9 +17,10 @@
|
|||
# limitations under the License.
|
||||
|
||||
rm -rf \
|
||||
lib/react-table.css \
|
||||
lib/*.css \
|
||||
node_modules \
|
||||
coordinator-console \
|
||||
pages \
|
||||
public \
|
||||
assets \
|
||||
index.html
|
||||
|
|
|
@ -24,3 +24,4 @@ cp -r coordinator-console "$1"
|
|||
cp -r old-console "$1"
|
||||
cp -r pages "$1"
|
||||
cp -r public "$1"
|
||||
cp -r assets "$1"
|
||||
|
|
|
@ -19,20 +19,8 @@
|
|||
import { resolveSrv } from 'dns';
|
||||
import * as React from 'react';
|
||||
import axios from 'axios';
|
||||
import {
|
||||
FormGroup,
|
||||
Button,
|
||||
InputGroup,
|
||||
Dialog,
|
||||
NumericInput,
|
||||
Classes,
|
||||
Tooltip,
|
||||
AnchorButton,
|
||||
TagInput,
|
||||
Intent,
|
||||
ButtonGroup,
|
||||
HTMLSelect
|
||||
} from "@blueprintjs/core";
|
||||
import { InputGroup } from "@blueprintjs/core";
|
||||
import { HTMLSelect, FormGroup, NumericInput, TagInput } from "../components/filler";
|
||||
|
||||
interface Field {
|
||||
name: string;
|
||||
|
@ -67,7 +55,7 @@ export class AutoForm<T> extends React.Component<AutoFormProps<T>, AutoFormState
|
|||
const { model, onChange } = this.props;
|
||||
return <NumericInput
|
||||
value={(model as any)[field.name]}
|
||||
onValueChange={v => {
|
||||
onValueChange={(v: any) => {
|
||||
if (isNaN(v)) return;
|
||||
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;
|
||||
return <NumericInput
|
||||
value={(model as any)[field.name]}
|
||||
onValueChange={v => {
|
||||
onValueChange={(v: number) => {
|
||||
if (isNaN(v)) return;
|
||||
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 {
|
||||
const { model, onChange } = this.props;
|
||||
return <HTMLSelect
|
||||
options={["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" }));
|
||||
}}
|
||||
/>
|
||||
>
|
||||
<option value="True">True</option>
|
||||
<option value="False">False</option>
|
||||
</HTMLSelect>
|
||||
}
|
||||
|
||||
private renderStringArrayInput(field: Field): JSX.Element {
|
||||
|
@ -118,7 +108,7 @@ export class AutoForm<T> extends React.Component<AutoFormProps<T>, AutoFormState
|
|||
onChange={(v: any) => {
|
||||
onChange(Object.assign({}, model, { [field.name]: v }));
|
||||
}}
|
||||
addOnBlur={true}
|
||||
fill
|
||||
/>;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
|
@ -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}
|
||||
/>;
|
||||
}
|
||||
}
|
|
@ -17,9 +17,10 @@
|
|||
*/
|
||||
|
||||
.header-bar {
|
||||
overflow: hidden;
|
||||
z-index: 10;
|
||||
|
||||
.logo {
|
||||
position: relative;
|
||||
width: 100px;
|
||||
height: 50px;
|
||||
|
||||
|
@ -31,4 +32,13 @@
|
|||
height: 75px;
|
||||
}
|
||||
}
|
||||
|
||||
.config-popover .pt-popover-content,
|
||||
.legacy-popover .pt-popover-content {
|
||||
width: 240px;
|
||||
}
|
||||
|
||||
.help-popover .pt-popover-content {
|
||||
width: 180px;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,12 +17,9 @@
|
|||
*/
|
||||
|
||||
import * as React from 'react';
|
||||
import {
|
||||
Button,
|
||||
Classes,
|
||||
AnchorButton, NavbarGroup, Alignment, NavbarHeading, NavbarDivider, Popover, Position, Navbar, Menu, MenuItem
|
||||
} from "@blueprintjs/core";
|
||||
import { IconNames } from "@blueprintjs/icons";
|
||||
import classNames from 'classnames';
|
||||
import { Button, Classes, AnchorButton, Popover, Position, Menu, MenuItem } from "@blueprintjs/core";
|
||||
import { IconNames, NavbarGroup, Alignment, NavbarDivider, Navbar } from "../components/filler";
|
||||
import { AboutDialog } from "../dialogs/about-dialog";
|
||||
import { CoordinatorDynamicConfigDialog } from '../dialogs/coordinator-dynamic-config';
|
||||
import "./header-bar.scss";
|
||||
|
@ -90,19 +87,19 @@ export class HeaderBar extends React.Component<HeaderBarProps, HeaderBarState> {
|
|||
const { aboutDialogOpen, coordinatorDynamicConfigDialogOpen } = this.state;
|
||||
|
||||
const legacyMenu = <Menu>
|
||||
<MenuItem icon={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.GRAPH} text="Legacy coordinator console" href={LEGACY_COORDINATOR_CONSOLE} target="_blank" />
|
||||
<MenuItem iconName={IconNames.MAP} text="Legacy overlord console" href={LEGACY_OVERLORD_CONSOLE} target="_blank" />
|
||||
</Menu>;
|
||||
|
||||
const helpMenu = <Menu>
|
||||
<MenuItem icon={IconNames.GRAPH} text="About" onClick={() => this.setState({ aboutDialogOpen: true })} />
|
||||
<MenuItem icon={IconNames.TH} text="Docs" href={DRUID_DOCS} target="_blank" />
|
||||
<MenuItem icon={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.GRAPH} text="About" onClick={() => this.setState({ aboutDialogOpen: true })} />
|
||||
<MenuItem iconName={IconNames.TH} text="Docs" href={DRUID_DOCS} target="_blank" />
|
||||
<MenuItem iconName={IconNames.USER} text="User group" href={DRUID_USER_GROUP} target="_blank" />
|
||||
<MenuItem iconName={IconNames.GIT_BRANCH} text="GitHub" href={DRUID_GITHUB} target="_blank" />
|
||||
</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>;
|
||||
|
||||
return <Navbar className="header-bar">
|
||||
|
@ -111,32 +108,30 @@ export class HeaderBar extends React.Component<HeaderBarProps, HeaderBarState> {
|
|||
{this.renderLogo()}
|
||||
</a>
|
||||
<NavbarDivider />
|
||||
<AnchorButton className={Classes.MINIMAL} icon={IconNames.MULTI_SELECT} text="Datasources" href="#datasources" active={active === 'datasources'} />
|
||||
<AnchorButton className={Classes.MINIMAL} icon={IconNames.FULL_STACKED_CHART} text="Segments" href="#segments" active={active === 'segments'} />
|
||||
<AnchorButton className={Classes.MINIMAL} icon={IconNames.GANTT_CHART} text="Tasks" href="#tasks" active={active === 'tasks'} />
|
||||
<AnchorButton className={Classes.MINIMAL} icon={IconNames.DATABASE} text="Data servers" href="#servers" active={active === 'servers'} />
|
||||
<AnchorButton className={classNames(Classes.MINIMAL, { 'pt-active': active === 'datasources' })} iconName={IconNames.MULTI_SELECT} text="Datasources" href="#datasources" />
|
||||
<AnchorButton className={classNames(Classes.MINIMAL, { 'pt-active': active === 'segments' })} iconName={IconNames.STACKED_CHART} text="Segments" href="#segments" />
|
||||
<AnchorButton className={classNames(Classes.MINIMAL, { 'pt-active': active === 'tasks' })} iconName={IconNames.GANTT_CHART} text="Tasks" href="#tasks" />
|
||||
<AnchorButton className={classNames(Classes.MINIMAL, { 'pt-active': active === 'servers' })} iconName={IconNames.DATABASE} text="Data servers" href="#servers" />
|
||||
<NavbarDivider />
|
||||
<AnchorButton className={Classes.MINIMAL} icon={IconNames.CONSOLE} text="SQL" href="#sql" active={active === 'sql'} />
|
||||
<Popover content={configMenu} position={Position.BOTTOM_LEFT}>
|
||||
<Button className={Classes.MINIMAL} icon={IconNames.SETTINGS} text="Config"/>
|
||||
<AnchorButton className={classNames(Classes.MINIMAL, { 'pt-active': active === 'sql' })} iconName={IconNames.APPLICATION} text="SQL" href="#sql" />
|
||||
<Popover className="config-popover" content={configMenu} position={Position.BOTTOM_LEFT} inline>
|
||||
<Button className={Classes.MINIMAL} iconName={IconNames.SETTINGS} text="Config"/>
|
||||
</Popover>
|
||||
</NavbarGroup>
|
||||
<NavbarGroup align={Alignment.RIGHT}>
|
||||
<Popover content={legacyMenu} position={Position.BOTTOM_LEFT}>
|
||||
<Button className={Classes.MINIMAL} icon={IconNames.SHARE} text="Legacy" />
|
||||
<Popover className="legacy-popover" content={legacyMenu} position={Position.BOTTOM_RIGHT} inline>
|
||||
<Button className={Classes.MINIMAL} iconName={IconNames.SHARE} text="Legacy" />
|
||||
</Popover>
|
||||
<Popover content={helpMenu} position={Position.BOTTOM_LEFT}>
|
||||
<Button className={Classes.MINIMAL} icon={IconNames.LIFESAVER} text="Help" />
|
||||
<Popover className="help-popover" content={helpMenu} position={Position.BOTTOM_RIGHT} inline>
|
||||
<Button className={Classes.MINIMAL} iconName={IconNames.HELP} text="Help" />
|
||||
</Popover>
|
||||
</NavbarGroup>
|
||||
<AboutDialog
|
||||
isOpen={aboutDialogOpen}
|
||||
{ aboutDialogOpen ? <AboutDialog
|
||||
onClose={() => this.setState({ aboutDialogOpen: false })}
|
||||
/>
|
||||
<CoordinatorDynamicConfigDialog
|
||||
isOpen={coordinatorDynamicConfigDialogOpen}
|
||||
/> : null }
|
||||
{ coordinatorDynamicConfigDialogOpen ? <CoordinatorDynamicConfigDialog
|
||||
onClose={() => this.setState({ coordinatorDynamicConfigDialogOpen: false })}
|
||||
/>
|
||||
/> : null }
|
||||
</Navbar>;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,16 +18,8 @@
|
|||
|
||||
import * as React from 'react';
|
||||
import axios from 'axios';
|
||||
import {
|
||||
FormGroup,
|
||||
Button,
|
||||
ControlGroup,
|
||||
Card,
|
||||
InputGroup,
|
||||
HTMLSelect,
|
||||
Collapse, NumericInput, TagInput,
|
||||
} from "@blueprintjs/core";
|
||||
import { IconNames } from "@blueprintjs/icons";
|
||||
import { Button, InputGroup, Collapse } from "@blueprintjs/core";
|
||||
import { IconNames, FormGroup, HTMLSelect, Card, ControlGroup, NumericInput, TagInput } from "../components/filler";
|
||||
import './rule-editor.scss';
|
||||
|
||||
export interface Rule {
|
||||
|
@ -156,27 +148,32 @@ export class RuleEditor extends React.Component<RuleEditorProps, RuleEditorState
|
|||
const ruleTiers = Object.keys(tieredReplicants).sort();
|
||||
return ruleTiers.map(tier => {
|
||||
return <ControlGroup key={tier}>
|
||||
<Button minimal style={{pointerEvents: 'none'}}>Replicants:</Button>
|
||||
<Button className="pt-minimal" style={{pointerEvents: 'none'}}>Replicants:</Button>
|
||||
<NumericInput
|
||||
value={tieredReplicants[tier]}
|
||||
onValueChange={v => {
|
||||
onValueChange={(v: number) => {
|
||||
if (isNaN(v)) return;
|
||||
onChange(RuleEditor.changeTierReplication(rule, tier, v));
|
||||
}}
|
||||
min={1}
|
||||
max={256}
|
||||
/>
|
||||
<Button minimal style={{pointerEvents: 'none'}}>Tier:</Button>
|
||||
<Button className="pt-minimal" style={{pointerEvents: 'none'}}>Tier:</Button>
|
||||
<HTMLSelect
|
||||
fill={true}
|
||||
value={tier}
|
||||
options={tiers.filter(t => t === tier || !tieredReplicants[t])}
|
||||
onChange={e => onChange(RuleEditor.changeTier(rule, tier, e.target.value))}
|
||||
/>
|
||||
onChange={(e: any) => 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
|
||||
disabled={ruleTiers.length === 1}
|
||||
onClick={() => this.removeTier(tier)}
|
||||
icon={IconNames.TRASH}
|
||||
iconName={IconNames.TRASH}
|
||||
/>
|
||||
</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;
|
||||
|
||||
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>;
|
||||
}
|
||||
|
||||
renderColocatedDataSources() {
|
||||
const { rule, onChange } = this.props;
|
||||
|
||||
return <ControlGroup>
|
||||
<Button minimal style={{pointerEvents: 'none'}}>Colocated datasources:</Button>
|
||||
return <FormGroup label="Colocated datasources:">
|
||||
<TagInput
|
||||
values={rule.colocatedDataSources || []}
|
||||
onChange={(v: any) => onChange(RuleEditor.changeColocatedDataSources(rule, v)) }
|
||||
addOnBlur={true}
|
||||
fill={true}
|
||||
fill
|
||||
/>
|
||||
</ControlGroup>;
|
||||
</FormGroup>;
|
||||
}
|
||||
|
||||
render() {
|
||||
|
@ -211,30 +206,18 @@ export class RuleEditor extends React.Component<RuleEditorProps, RuleEditorState
|
|||
|
||||
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 ruleTimeType = RuleEditor.getTimeType(rule);
|
||||
|
||||
return <div className="rule-editor">
|
||||
<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)}
|
||||
</Button>
|
||||
<div className="spacer"/>
|
||||
{moveUp ? <Button minimal icon={IconNames.ARROW_UP} onClick={moveUp}/> : null}
|
||||
{moveDown ? <Button minimal icon={IconNames.ARROW_DOWN} onClick={moveDown}/> : null}
|
||||
<Button minimal icon={IconNames.TRASH} onClick={onDelete}/>
|
||||
{moveUp ? <Button className="pt-minimal" iconName={IconNames.ARROW_UP} onClick={moveUp}/> : null}
|
||||
{moveDown ? <Button className="pt-minimal" iconName={IconNames.ARROW_DOWN} onClick={moveDown}/> : null}
|
||||
<Button className="pt-minimal" iconName={IconNames.TRASH} onClick={onDelete}/>
|
||||
</div>
|
||||
|
||||
<Collapse isOpen={isOpen}>
|
||||
|
@ -243,14 +226,20 @@ export class RuleEditor extends React.Component<RuleEditorProps, RuleEditorState
|
|||
<ControlGroup>
|
||||
<HTMLSelect
|
||||
value={ruleLoadType}
|
||||
options={ruleLoadTypes}
|
||||
onChange={e => onChange(RuleEditor.changeLoadType(rule, e.target.value as any))}
|
||||
/>
|
||||
onChange={(e: any) => 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
|
||||
value={ruleTimeType}
|
||||
options={ruleTimeTypes}
|
||||
onChange={e => onChange(RuleEditor.changeTimeType(rule, e.target.value as any))}
|
||||
/>
|
||||
onChange={(e: any) => 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 === 'ByInterval' && <InputGroup value={rule.interval || ''} onChange={(e: any) => onChange(RuleEditor.changeInterval(rule, e.target.value as any))}/>}
|
||||
</ControlGroup>
|
||||
|
|
|
@ -18,11 +18,14 @@
|
|||
|
||||
import * as React from 'react';
|
||||
import * as classNames from 'classnames';
|
||||
import {
|
||||
TextArea,
|
||||
Intent,
|
||||
Button
|
||||
} from "@blueprintjs/core";
|
||||
import * as ace from 'brace'
|
||||
import AceEditor from "react-ace";
|
||||
import 'brace/mode/sql';
|
||||
import 'brace/mode/hjson';
|
||||
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> {
|
||||
initSql: string | null;
|
||||
|
@ -31,39 +34,56 @@ export interface SqlControlProps extends React.Props<any> {
|
|||
|
||||
export interface SqlControlState {
|
||||
query: string;
|
||||
autoCompleteOn: boolean;
|
||||
}
|
||||
|
||||
export class SqlControl extends React.Component<SqlControlProps, SqlControlState> {
|
||||
constructor(props: SqlControlProps, context: any) {
|
||||
super(props, context);
|
||||
this.state = {
|
||||
query: props.initSql || ''
|
||||
query: props.initSql || '',
|
||||
autoCompleteOn: true
|
||||
};
|
||||
}
|
||||
|
||||
private handleChange = (e: React.ChangeEvent<HTMLTextAreaElement>) => {
|
||||
private handleChange = (newValue: string): void => {
|
||||
this.setState({
|
||||
query: e.target.value
|
||||
});
|
||||
query: newValue
|
||||
})
|
||||
}
|
||||
|
||||
render() {
|
||||
const { onRun } = this.props;
|
||||
const { query } = this.state;
|
||||
const { query, autoCompleteOn } = this.state;
|
||||
|
||||
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">
|
||||
<TextArea
|
||||
className="bp3-fill"
|
||||
large={true}
|
||||
intent={Intent.PRIMARY}
|
||||
<AceEditor
|
||||
key={isRune ? "hjson" : "sql"}
|
||||
mode={isRune ? "hjson" : "sql"}
|
||||
theme="solarized_dark"
|
||||
name="ace-editor"
|
||||
onChange={this.handleChange}
|
||||
focus={true}
|
||||
fontSize={14}
|
||||
width={'100%'}
|
||||
height={"30vh"}
|
||||
showPrintMargin={false}
|
||||
value={query}
|
||||
editorProps={{
|
||||
$blockScrolling: Infinity
|
||||
}}
|
||||
setOptions={{
|
||||
enableBasicAutocompletion: isRune ? false : autoCompleteOn,
|
||||
enableLiveAutocompletion: isRune ? false : autoCompleteOn,
|
||||
showLineNumbers: true,
|
||||
tabSize: 2,
|
||||
}}
|
||||
/>
|
||||
<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>
|
||||
}
|
||||
|
|
|
@ -19,20 +19,9 @@
|
|||
@import "./variables";
|
||||
|
||||
.console-application {
|
||||
position: relative;
|
||||
height: 100%;
|
||||
|
||||
.bp3-navbar {
|
||||
position: absolute;
|
||||
top: 0;
|
||||
left: 0;
|
||||
right: 0;
|
||||
}
|
||||
|
||||
.bp3-navbar-group > a {
|
||||
text-decoration: inherit;
|
||||
color: inherit;
|
||||
}
|
||||
|
||||
.view-container {
|
||||
position: absolute;
|
||||
top: 50px;
|
||||
|
@ -44,6 +33,10 @@
|
|||
&.scrollable {
|
||||
overflow-y: scroll;
|
||||
}
|
||||
|
||||
.app-view {
|
||||
position: relative;
|
||||
}
|
||||
}
|
||||
|
||||
.control-separator {
|
||||
|
@ -66,9 +59,5 @@
|
|||
font-size: 35px;
|
||||
line-height: 35px;
|
||||
}
|
||||
|
||||
label.bp3-label {
|
||||
margin-bottom: 7px;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,10 +26,10 @@ import { HeaderBar, HeaderActiveTab } from './components/header-bar';
|
|||
import { localStorageGet, localStorageSet } from './utils';
|
||||
import { DRUID_DOCS_SQL, LEGACY_COORDINATOR_CONSOLE, LEGACY_OVERLORD_CONSOLE } from './variables';
|
||||
import { HomeView } from './views/home-view';
|
||||
import { ServersView } from './views/servers-view';
|
||||
import { DatasourcesView } from './views/datasource-view';
|
||||
import { TasksView } from './views/tasks-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 "./console-application.scss";
|
||||
|
||||
|
@ -59,7 +59,7 @@ export class ConsoleApplication extends React.Component<ConsoleApplicationProps,
|
|||
|
||||
// Status works but SQL 405s => the SQL endpoint is disabled
|
||||
AppToaster.show({
|
||||
icon: 'error',
|
||||
iconName: 'error',
|
||||
intent: Intent.DANGER,
|
||||
timeout: 120000,
|
||||
message: <>
|
||||
|
@ -161,7 +161,7 @@ export class ConsoleApplication extends React.Component<ConsoleApplicationProps,
|
|||
}} />
|
||||
</Switch>
|
||||
</div>
|
||||
</HashRouter>
|
||||
</HashRouter>;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -18,11 +18,10 @@
|
|||
|
||||
import * as React from 'react';
|
||||
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';
|
||||
|
||||
export interface AboutDialogProps extends React.Props<any> {
|
||||
isOpen: boolean,
|
||||
onClose: () => void
|
||||
}
|
||||
|
||||
|
@ -36,15 +35,15 @@ export class AboutDialog extends React.Component<AboutDialogProps, AboutDialogSt
|
|||
}
|
||||
|
||||
render() {
|
||||
const { isOpen, onClose } = this.props;
|
||||
const { onClose } = this.props;
|
||||
|
||||
return <Dialog
|
||||
icon={IconNames.INFO_SIGN}
|
||||
iconName={IconNames.GRAPH}
|
||||
onClose={onClose}
|
||||
title="Apache Druid"
|
||||
isOpen={isOpen}
|
||||
usePortal={true}
|
||||
canEscapeKeyClose={true}
|
||||
isOpen
|
||||
inline
|
||||
canEscapeKeyClose
|
||||
>
|
||||
<div className={Classes.DIALOG_BODY}>
|
||||
<p>
|
||||
|
|
|
@ -19,22 +19,14 @@
|
|||
import classNames from 'classnames';
|
||||
import * as React from 'react';
|
||||
import {
|
||||
FormGroup,
|
||||
Button,
|
||||
InputGroup,
|
||||
Dialog,
|
||||
NumericInput,
|
||||
Classes,
|
||||
Tooltip,
|
||||
AnchorButton,
|
||||
TagInput,
|
||||
Intent,
|
||||
ButtonGroup,
|
||||
ProgressBar,
|
||||
MaybeElement,
|
||||
Icon,
|
||||
IconName
|
||||
ProgressBar
|
||||
} from "@blueprintjs/core";
|
||||
import { Icon, FormGroup, ButtonGroup, NumericInput, TagInput } from '../components/filler';
|
||||
import { AppToaster } from '../singletons/toaster';
|
||||
|
||||
export interface AsyncAlertDialogProps extends React.Props<any> {
|
||||
|
@ -43,7 +35,7 @@ export interface AsyncAlertDialogProps extends React.Props<any> {
|
|||
confirmButtonText: string;
|
||||
cancelButtonText?: string;
|
||||
className?: string,
|
||||
icon?: IconName | MaybeElement;
|
||||
icon?: string;
|
||||
intent?: Intent;
|
||||
successText: string;
|
||||
failText: string;
|
||||
|
@ -88,17 +80,19 @@ export class AsyncActionDialog extends React.Component<AsyncAlertDialogProps, As
|
|||
render() {
|
||||
const { action, onClose, className, icon, intent, confirmButtonText, cancelButtonText, children } = this.props;
|
||||
const { working } = this.state;
|
||||
if (!action) return null;
|
||||
|
||||
const handleClose = () => onClose(false);
|
||||
|
||||
return <Dialog
|
||||
isOpen={Boolean(action)}
|
||||
isOpen
|
||||
inline
|
||||
className={classNames(Classes.ALERT, 'async-alert-dialog', className)}
|
||||
canEscapeKeyClose={!working}
|
||||
onClose={handleClose}
|
||||
>
|
||||
<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> }
|
||||
</div>
|
||||
{
|
||||
|
|
|
@ -19,15 +19,14 @@
|
|||
import { Intent } from '@blueprintjs/core';
|
||||
import * as React from 'react';
|
||||
import axios from 'axios';
|
||||
import { IconNames } from "@blueprintjs/icons";
|
||||
import { AppToaster } from '../singletons/toaster';
|
||||
import { IconNames } from '../components/filler';
|
||||
import { AutoForm } from '../components/auto-form';
|
||||
import { getDruidErrorMessage } from '../utils';
|
||||
import { SnitchDialog } from './snitch-dialog';
|
||||
import './coordinator-dynamic-config.scss';
|
||||
|
||||
export interface CoordinatorDynamicConfigDialogProps extends React.Props<any> {
|
||||
isOpen: boolean,
|
||||
onClose: () => void
|
||||
}
|
||||
|
||||
|
@ -43,6 +42,10 @@ export class CoordinatorDynamicConfigDialog extends React.Component<CoordinatorD
|
|||
}
|
||||
}
|
||||
|
||||
componentDidMount(): void {
|
||||
this.getClusterConfig();
|
||||
}
|
||||
|
||||
async getClusterConfig() {
|
||||
let config: Record<string, any> | null = null;
|
||||
try {
|
||||
|
@ -50,7 +53,7 @@ export class CoordinatorDynamicConfigDialog extends React.Component<CoordinatorD
|
|||
config = configResp.data
|
||||
} catch (e) {
|
||||
AppToaster.show({
|
||||
icon: IconNames.ERROR,
|
||||
iconName: IconNames.ERROR,
|
||||
intent: Intent.DANGER,
|
||||
message: `Could not load coordinator dynamic config: ${getDruidErrorMessage(e)}`
|
||||
});
|
||||
|
@ -73,7 +76,7 @@ export class CoordinatorDynamicConfigDialog extends React.Component<CoordinatorD
|
|||
});
|
||||
} catch (e) {
|
||||
AppToaster.show({
|
||||
icon: IconNames.ERROR,
|
||||
iconName: IconNames.ERROR,
|
||||
intent: Intent.DANGER,
|
||||
message: `Could not save coordinator dynamic config: ${getDruidErrorMessage(e)}`
|
||||
});
|
||||
|
@ -87,15 +90,14 @@ export class CoordinatorDynamicConfigDialog extends React.Component<CoordinatorD
|
|||
}
|
||||
|
||||
render() {
|
||||
const { isOpen, onClose } = this.props;
|
||||
const { onClose } = this.props;
|
||||
const { dynamicConfig } = this.state;
|
||||
|
||||
return <SnitchDialog
|
||||
className="coordinator-dynamic-config"
|
||||
isOpen={ isOpen }
|
||||
isOpen
|
||||
onSave={this.saveClusterConfig}
|
||||
onOpening={() => {this.getClusterConfig()}}
|
||||
onClose={ onClose }
|
||||
onClose={onClose}
|
||||
title="Coordinator dynamic config"
|
||||
>
|
||||
<p>
|
||||
|
|
|
@ -16,6 +16,9 @@
|
|||
* 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';
|
||||
|
||||
describe('reorderArray', () => {
|
||||
|
|
|
@ -18,11 +18,8 @@
|
|||
|
||||
import * as React from 'react';
|
||||
import axios from 'axios';
|
||||
import {
|
||||
FormGroup,
|
||||
Button,
|
||||
} from "@blueprintjs/core";
|
||||
import { IconNames } from "@blueprintjs/icons";
|
||||
import { Button } from "@blueprintjs/core";
|
||||
import { FormGroup, IconNames } from '../components/filler';
|
||||
import { RuleEditor, Rule } from '../components/rule-editor';
|
||||
import { SnitchDialog } from './snitch-dialog';
|
||||
|
||||
|
@ -148,6 +145,7 @@ export class RetentionDialog extends React.Component<RetentionDialogProps, Reten
|
|||
saveDisabled={false}
|
||||
canOutsideClickClose={false}
|
||||
isOpen
|
||||
inline
|
||||
onClose={onCancel}
|
||||
title={`Edit retention rules: ${datasource}${datasource === '_default' ? ' (cluster defaults)' : ''}`}
|
||||
onReset={this.reset}
|
||||
|
@ -162,7 +160,7 @@ export class RetentionDialog extends React.Component<RetentionDialogProps, Reten
|
|||
{(currentRules || []).map(this.renderRule)}
|
||||
</FormGroup>
|
||||
<FormGroup className="right">
|
||||
<Button icon={IconNames.PLUS} onClick={this.addRule}>New rule</Button>
|
||||
<Button iconName={IconNames.PLUS} onClick={this.addRule}>New rule</Button>
|
||||
</FormGroup>
|
||||
{
|
||||
(!currentRules.length && datasource !== '_default') &&
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
|
||||
import * as React from 'react';
|
||||
import {
|
||||
FormGroup,
|
||||
Button,
|
||||
InputGroup,
|
||||
Dialog,
|
||||
|
@ -26,8 +25,8 @@ import {
|
|||
Classes,
|
||||
Intent,
|
||||
} from "@blueprintjs/core";
|
||||
import { IconNames, FormGroup } from '../components/filler';
|
||||
|
||||
import { IconNames } from "@blueprintjs/icons";
|
||||
|
||||
export interface SnitchDialogProps extends IDialogProps {
|
||||
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?"}>
|
||||
<InputGroup value={author} onChange={(e: any) => this.changeAuthor(e.target.value)}/>
|
||||
</FormGroup>
|
||||
<FormGroup className={"comment"}>
|
||||
<FormGroup label={"Why are you making this change?"} className={"comment"}>
|
||||
<InputGroup
|
||||
className="pt-large"
|
||||
value={comment}
|
||||
placeholder={"Why are you making this change?"}
|
||||
placeholder={"Enter description here"}
|
||||
onChange={(e: any) => this.changeComment(e.target.value)}
|
||||
large={true}
|
||||
/>
|
||||
</FormGroup>
|
||||
</div>
|
||||
|
@ -128,38 +127,25 @@ export class SnitchDialog extends React.Component<SnitchDialogProps, SnitchDialo
|
|||
const { showFinalStep } = this.state;
|
||||
|
||||
return <div className={Classes.DIALOG_FOOTER_ACTIONS}>
|
||||
<FormGroup>
|
||||
{ showFinalStep
|
||||
? <Button onClick={this.back} icon={IconNames.ARROW_LEFT}>Back</Button>
|
||||
: onReset ? <Button onClick={this.reset} intent={"none"}>Reset</Button> : null
|
||||
}
|
||||
{ showFinalStep
|
||||
? <Button onClick={this.back} iconName={IconNames.ARROW_LEFT}>Back</Button>
|
||||
: onReset ? <Button onClick={this.reset} intent={"none" as any}>Reset</Button> : null
|
||||
}
|
||||
|
||||
{ showFinalStep
|
||||
? <Button disabled={saveDisabled} text="Save" onClick={this.save} intent={Intent.PRIMARY} rightIcon={IconNames.TICK}/>
|
||||
: <Button disabled={saveDisabled} text="Next" onClick={this.goToFinalStep} intent={Intent.PRIMARY} rightIcon={IconNames.ARROW_RIGHT}/>
|
||||
}
|
||||
</FormGroup>
|
||||
{ showFinalStep
|
||||
? <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 as any} rightIconName={IconNames.ARROW_RIGHT}/>
|
||||
}
|
||||
</div>
|
||||
}
|
||||
|
||||
onOpening = (node: HTMLElement) => {
|
||||
const { onOpening } = this.props;
|
||||
|
||||
this.setState({
|
||||
author: '',
|
||||
comment: ''
|
||||
});
|
||||
|
||||
onOpening && onOpening(node);
|
||||
}
|
||||
|
||||
render() {
|
||||
const { isOpen, onClose, className, children, saveDisabled } = this.props;
|
||||
const { onClose, className, children, saveDisabled } = this.props;
|
||||
const { showFinalStep } = this.state;
|
||||
|
||||
if (showFinalStep) return this.renderFinalStep();
|
||||
|
||||
return <Dialog {...this.props} onOpening={this.onOpening}>
|
||||
return <Dialog isOpen inline {...this.props}>
|
||||
<div className={Classes.DIALOG_BODY}>
|
||||
{children}
|
||||
</div>
|
||||
|
|
|
@ -18,6 +18,11 @@
|
|||
|
||||
.post-spec-dialog {
|
||||
.post-spec-dialog-textarea {
|
||||
margin: 20px;
|
||||
background-color: #232C35;
|
||||
margin-bottom: 10px;
|
||||
|
||||
.ace_gutter {
|
||||
background-color: #232C35;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,13 +17,13 @@
|
|||
*/
|
||||
|
||||
import * as React from "react";
|
||||
import axios from 'axios';
|
||||
import {Button, Classes, Dialog, Intent, EditableText} from "@blueprintjs/core";
|
||||
import { Button, Classes, Dialog, Intent } from "@blueprintjs/core";
|
||||
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> {
|
||||
isOpen: boolean;
|
||||
onSubmit: (spec: JSON) => void;
|
||||
onClose: () => void;
|
||||
title: string;
|
||||
|
@ -59,22 +59,33 @@ export class SpecDialog extends React.Component<SpecDialogProps, SpecDialogState
|
|||
}
|
||||
|
||||
render() {
|
||||
const { isOpen, onClose, title } = this.props;
|
||||
const { onClose, title } = this.props;
|
||||
const { spec } = this.state;
|
||||
|
||||
return <Dialog
|
||||
className={"post-spec-dialog"}
|
||||
isOpen={isOpen}
|
||||
isOpen
|
||||
onClose={onClose}
|
||||
title={title}
|
||||
>
|
||||
<EditableText
|
||||
<AceEditor
|
||||
mode="json"
|
||||
theme="solarized_dark"
|
||||
className={"post-spec-dialog-textarea"}
|
||||
multiline={true}
|
||||
minLines={30}
|
||||
maxLines={30}
|
||||
placeholder={"Enter the spec JSON to post"}
|
||||
onChange={ (e) => {this.setState({ spec: e })}}
|
||||
onChange={ (e) => {this.setState({ spec: e })} }
|
||||
fontSize={12}
|
||||
showPrintMargin={false}
|
||||
showGutter={true}
|
||||
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_ACTIONS}>
|
||||
|
|
|
@ -17,13 +17,9 @@
|
|||
*/
|
||||
|
||||
@import '../node_modules/normalize.css/normalize';
|
||||
@import '../node_modules/@blueprintjs/core/lib/css/blueprint';
|
||||
@import '../lib/blueprint';
|
||||
@import '../lib/react-table';
|
||||
|
||||
* {
|
||||
position: relative;
|
||||
}
|
||||
|
||||
html,
|
||||
body {
|
||||
//font-family: 'Open Sans', Helvetica, Arial, sans-serif;
|
||||
|
@ -33,7 +29,7 @@ body {
|
|||
}
|
||||
|
||||
body {
|
||||
&.bp3-dark {
|
||||
&.pt-dark {
|
||||
background: rgb(41, 55, 66);
|
||||
}
|
||||
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
import 'es6-shim/es6-shim';
|
||||
import 'es7-shim'; // Webpack with automatically pick browser.js which does the shim()
|
||||
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 "./singletons/react-table-defaults";
|
||||
import "./entry.scss";
|
||||
|
@ -34,7 +35,7 @@ ReactDOM.render(
|
|||
{
|
||||
version: '0.0.1'
|
||||
}
|
||||
),
|
||||
) as any,
|
||||
container
|
||||
);
|
||||
|
||||
|
|
|
@ -25,7 +25,7 @@ import { countBy, makeTextFilter } from '../utils';
|
|||
|
||||
class FullButton extends React.Component {
|
||||
render() {
|
||||
return <Button fill={true} {...this.props}/>;
|
||||
return <Button className="pt-fill" {...this.props}/>;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -16,8 +16,8 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import { Button, InputGroup, Intent, HTMLSelect } from '@blueprintjs/core';
|
||||
import { IconNames } from "@blueprintjs/icons";
|
||||
import { Button, InputGroup, Intent } from '@blueprintjs/core';
|
||||
import { IconNames, HTMLSelect } from "../components/filler";
|
||||
import * as numeral from "numeral";
|
||||
import * as React from 'react';
|
||||
import { Filter, FilterRender } from 'react-table';
|
||||
|
@ -43,7 +43,7 @@ export function makeTextFilter(placeholder: string = ''): FilterRender {
|
|||
key={key}
|
||||
onChange={(e: any) => onChange(e.target.value)}
|
||||
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}
|
||||
/>
|
||||
}
|
||||
|
@ -55,7 +55,7 @@ export function makeBooleanFilter(): FilterRender {
|
|||
return <HTMLSelect
|
||||
key={key}
|
||||
style={{ width: '100%' }}
|
||||
onChange={event => onChange(event.target.value)}
|
||||
onChange={(event: any) => onChange(event.target.value)}
|
||||
value={filterValue || "all"}
|
||||
fill={true}
|
||||
>
|
||||
|
|
|
@ -22,7 +22,7 @@
|
|||
|
||||
.ReactTable {
|
||||
position: absolute;
|
||||
top: 60px;
|
||||
top: 50px;
|
||||
bottom: 0;
|
||||
width: 100%;
|
||||
}
|
||||
|
|
|
@ -22,10 +22,11 @@ import * as classNames from 'classnames';
|
|||
import ReactTable from "react-table";
|
||||
import { Filter } from "react-table";
|
||||
import { Button, Intent, Switch } from "@blueprintjs/core";
|
||||
import { IconNames } from "@blueprintjs/icons";
|
||||
import { IconNames } from "../components/filler";
|
||||
import { AppToaster } from '../singletons/toaster';
|
||||
import { RuleEditor } from '../components/rule-editor';
|
||||
import { AsyncActionDialog } from '../dialogs/async-action-dialog';
|
||||
import { RetentionDialog } from '../dialogs/retention-dialog';
|
||||
import {
|
||||
addFilter,
|
||||
formatNumber,
|
||||
|
@ -35,7 +36,6 @@ import {
|
|||
QueryManager,
|
||||
pluralIfNeeded, queryDruidSql, getDruidErrorMessage
|
||||
} from "../utils";
|
||||
import { RetentionDialog } from '../dialogs/retention-dialog';
|
||||
|
||||
import "./datasource-view.scss";
|
||||
|
||||
|
@ -439,12 +439,12 @@ GROUP BY 1`);
|
|||
<div className="control-bar">
|
||||
<div className="control-label">Datasources</div>
|
||||
<Button
|
||||
icon={IconNames.REFRESH}
|
||||
iconName={IconNames.REFRESH}
|
||||
text="Refresh"
|
||||
onClick={() => this.datasourceQueryManager.rerunLastQuery()}
|
||||
/>
|
||||
<Button
|
||||
icon={IconNames.CONSOLE}
|
||||
iconName={IconNames.APPLICATION}
|
||||
text="Go to SQL"
|
||||
onClick={() => goToSql(this.datasourceQueryManager.getLastQuery())}
|
||||
/>
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
color: inherit;
|
||||
}
|
||||
|
||||
.bp3-card {
|
||||
.pt-card {
|
||||
height: 160px;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,14 +19,13 @@
|
|||
import axios from 'axios';
|
||||
import * as React from 'react';
|
||||
import * as classNames from 'classnames';
|
||||
import { H5, Card, Icon } from "@blueprintjs/core";
|
||||
import { IconName, IconNames } from "@blueprintjs/icons";
|
||||
import { H5, Card, Icon, IconNames } from "../components/filler";
|
||||
import { QueryManager, pluralIfNeeded, queryDruidSql, getHeadProp } from '../utils';
|
||||
import './home-view.scss';
|
||||
|
||||
export interface CardOptions {
|
||||
href: string;
|
||||
icon: IconName;
|
||||
icon: string;
|
||||
title: string;
|
||||
loading?: boolean;
|
||||
content: JSX.Element | string;
|
||||
|
@ -268,7 +267,7 @@ GROUP BY 1`);
|
|||
return <div className="home-view app-view">
|
||||
{this.renderCard({
|
||||
href: "/status",
|
||||
icon: IconNames.INFO_SIGN,
|
||||
icon: IconNames.GRAPH,
|
||||
title: "Status",
|
||||
loading: state.statusLoading,
|
||||
content: state.status ? `Apache Druid is running version ${state.status.version}` : '',
|
||||
|
@ -286,7 +285,7 @@ GROUP BY 1`);
|
|||
|
||||
{this.renderCard({
|
||||
href: "#segments",
|
||||
icon: IconNames.FULL_STACKED_CHART,
|
||||
icon: IconNames.STACKED_CHART,
|
||||
title: "Segments",
|
||||
loading: state.segmentCountLoading,
|
||||
content: pluralIfNeeded(state.segmentCount, 'segment'),
|
||||
|
|
|
@ -22,7 +22,7 @@
|
|||
|
||||
.ReactTable {
|
||||
position: absolute;
|
||||
top: 60px;
|
||||
top: 50px;
|
||||
bottom: 0;
|
||||
width: 100%;
|
||||
|
||||
|
|
|
@ -21,8 +21,8 @@ import * as React from 'react';
|
|||
import * as classNames from 'classnames';
|
||||
import ReactTable from "react-table";
|
||||
import { Filter } from "react-table";
|
||||
import { H5, Button } from "@blueprintjs/core";
|
||||
import { IconNames } from "@blueprintjs/icons";
|
||||
import { Button } from "@blueprintjs/core";
|
||||
import { H5, IconNames } from "../components/filler";
|
||||
import {
|
||||
addFilter,
|
||||
makeBooleanFilter,
|
||||
|
@ -261,12 +261,12 @@ export class SegmentsView extends React.Component<SegmentsViewProps, SegmentsVie
|
|||
<div className="control-bar">
|
||||
<div className="control-label">Segments</div>
|
||||
<Button
|
||||
icon={IconNames.REFRESH}
|
||||
iconName={IconNames.REFRESH}
|
||||
text="Refresh"
|
||||
onClick={() => this.segmentsQueryManager.rerunLastQuery()}
|
||||
/>
|
||||
<Button
|
||||
icon={IconNames.CONSOLE}
|
||||
iconName={IconNames.APPLICATION}
|
||||
text="Go to SQL"
|
||||
onClick={() => goToSql(this.segmentsQueryManager.getLastQuery().query)}
|
||||
/>
|
||||
|
|
|
@ -31,6 +31,7 @@
|
|||
}
|
||||
|
||||
.fill-indicator {
|
||||
position: relative;
|
||||
width: 100%;
|
||||
height: 100%;
|
||||
background-color: #dadada;
|
||||
|
|
|
@ -22,8 +22,8 @@ import * as classNames from 'classnames';
|
|||
import ReactTable from "react-table";
|
||||
import { Filter } from "react-table";
|
||||
import { sum } from "d3-array";
|
||||
import { Button, H1, Switch } from "@blueprintjs/core";
|
||||
import { IconNames } from "@blueprintjs/icons";
|
||||
import { Button, Switch } from "@blueprintjs/core";
|
||||
import { IconNames } from '../components/filler';
|
||||
import { addFilter, formatBytes, formatBytesCompact, QueryManager, queryDruidSql } from "../utils";
|
||||
import "./servers-view.scss";
|
||||
|
||||
|
@ -334,12 +334,12 @@ WHERE "server_type" = 'historical'`);
|
|||
<div className="control-bar">
|
||||
<div className="control-label">Historicals</div>
|
||||
<Button
|
||||
icon={IconNames.REFRESH}
|
||||
iconName={IconNames.REFRESH}
|
||||
text="Refresh"
|
||||
onClick={() => this.serverQueryManager.rerunLastQuery()}
|
||||
/>
|
||||
<Button
|
||||
icon={IconNames.CONSOLE}
|
||||
iconName={IconNames.APPLICATION}
|
||||
text="Go to SQL"
|
||||
onClick={() => goToSql(this.serverQueryManager.getLastQuery())}
|
||||
/>
|
||||
|
@ -356,7 +356,7 @@ WHERE "server_type" = 'historical'`);
|
|||
<div className="control-bar">
|
||||
<div className="control-label">MiddleManagers</div>
|
||||
<Button
|
||||
icon={IconNames.REFRESH}
|
||||
iconName={IconNames.REFRESH}
|
||||
text="Refresh"
|
||||
onClick={() => this.middleManagerQueryManager.rerunLastQuery()}
|
||||
/>
|
||||
|
|
|
@ -21,8 +21,8 @@ import * as React from 'react';
|
|||
import * as classNames from 'classnames';
|
||||
import ReactTable from "react-table";
|
||||
import { Filter } from "react-table";
|
||||
import { Button, H1, ButtonGroup, Intent, Label, Alert } from "@blueprintjs/core";
|
||||
import { IconNames } from "@blueprintjs/icons";
|
||||
import { Button, Intent, Alert } from "@blueprintjs/core";
|
||||
import { ButtonGroup, Label, IconNames } from "../components/filler";
|
||||
import { addFilter, QueryManager, getDruidErrorMessage, countBy, formatDuration, queryDruidSql } from "../utils";
|
||||
import { AsyncActionDialog } from "../dialogs/async-action-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-label">Supervisors</div>
|
||||
<Button
|
||||
icon={IconNames.REFRESH}
|
||||
iconName={IconNames.REFRESH}
|
||||
text="Refresh"
|
||||
onClick={() => this.supervisorQueryManager.rerunLastQuery()}
|
||||
/>
|
||||
<Button
|
||||
icon={IconNames.PLUS}
|
||||
iconName={IconNames.PLUS}
|
||||
text="Submit supervisor"
|
||||
onClick={() => this.setState({ supervisorSpecDialogOpen: true })}
|
||||
/>
|
||||
|
@ -542,43 +542,40 @@ ORDER BY "rank" DESC, "created_time" DESC`);
|
|||
<div className="control-label">Tasks</div>
|
||||
<Label>Group by</Label>
|
||||
<ButtonGroup>
|
||||
<Button active={groupTasksBy === null} onClick={() => this.setState({ groupTasksBy: null })}>None</Button>
|
||||
<Button active={groupTasksBy === 'type'} onClick={() => this.setState({ groupTasksBy: 'type' })}>Type</Button>
|
||||
<Button 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 === null })} onClick={() => this.setState({ groupTasksBy: null })}>None</Button>
|
||||
<Button className={classNames({ 'pt-active': groupTasksBy === 'type' })} onClick={() => this.setState({ groupTasksBy: 'type' })}>Type</Button>
|
||||
<Button className={classNames({ 'pt-active': groupTasksBy === 'datasource' })} onClick={() => this.setState({ groupTasksBy: 'datasource' })}>Datasource</Button>
|
||||
<Button className={classNames({ 'pt-active': groupTasksBy === 'status' })} onClick={() => this.setState({ groupTasksBy: 'status' })}>Status</Button>
|
||||
</ButtonGroup>
|
||||
<Button
|
||||
icon={IconNames.REFRESH}
|
||||
iconName={IconNames.REFRESH}
|
||||
text="Refresh"
|
||||
onClick={() => this.taskQueryManager.rerunLastQuery()}
|
||||
/>
|
||||
<Button
|
||||
icon={IconNames.CONSOLE}
|
||||
iconName={IconNames.APPLICATION}
|
||||
text="Go to SQL"
|
||||
onClick={() => goToSql(this.taskQueryManager.getLastQuery())}
|
||||
/>
|
||||
<Button
|
||||
icon={IconNames.PLUS}
|
||||
iconName={IconNames.PLUS}
|
||||
text="Submit task"
|
||||
onClick={() => this.setState({ taskSpecDialogOpen: true })}
|
||||
/>
|
||||
</div>
|
||||
{this.renderTaskTable()}
|
||||
|
||||
<SpecDialog
|
||||
isOpen={ supervisorSpecDialogOpen }
|
||||
{ supervisorSpecDialogOpen ? <SpecDialog
|
||||
onClose={() => this.setState({ supervisorSpecDialogOpen: false })}
|
||||
onSubmit={this.submitSupervisor}
|
||||
title="Submit supervisor"
|
||||
/>
|
||||
<SpecDialog
|
||||
isOpen={ taskSpecDialogOpen }
|
||||
/> : null }
|
||||
{ taskSpecDialogOpen ? <SpecDialog
|
||||
onClose={() => this.setState({ taskSpecDialogOpen: false })}
|
||||
onSubmit={this.submitTask}
|
||||
title="Submit task"
|
||||
/>
|
||||
/> : null }
|
||||
<Alert
|
||||
icon={IconNames.ERROR}
|
||||
iconName={IconNames.ERROR}
|
||||
intent={Intent.PRIMARY}
|
||||
isOpen={Boolean(alertErrorMsg)}
|
||||
confirmButtonText="OK"
|
||||
|
|
|
@ -21,8 +21,8 @@
|
|||
},
|
||||
|
||||
"include": [
|
||||
"**/*.ts",
|
||||
"**/*.tsx"
|
||||
"src/**/*.ts",
|
||||
"src/**/*.tsx"
|
||||
],
|
||||
"exclude": [
|
||||
"**/*.test.ts"
|
||||
|
|
|
@ -24,7 +24,7 @@
|
|||
<meta name="description" content="Apache Druid web console">
|
||||
<link rel="shortcut icon" href="/favicon.png">
|
||||
</head>
|
||||
<body class="bp3-dark mouse-mode">
|
||||
<body class="pt-dark mouse-mode">
|
||||
<div class="app-container"></div>
|
||||
<script src="public/web-console-0.14.0.js"></script>
|
||||
</body>
|
||||
|
|
Loading…
Reference in New Issue