mirror of https://github.com/apache/druid.git
Remove unused code and exception declarations (#5461)
* Remove unused code and exception declarations * Address comments * Remove redundant Exception declarations * Make FirehoseFactoryV2.connect() to throw IOException again
This commit is contained in:
parent
afa25202a3
commit
693e3575f9
|
@ -91,6 +91,7 @@
|
|||
<inspection_tool class="ObjectEqualsNull" enabled="true" level="ERROR" enabled_by_default="true" />
|
||||
<inspection_tool class="ObjectToString" enabled="true" level="ERROR" enabled_by_default="true" />
|
||||
<inspection_tool class="PrimitiveArrayArgumentToVariableArgMethod" enabled="true" level="ERROR" enabled_by_default="true" />
|
||||
<inspection_tool class="RedundantThrows" enabled="true" level="ERROR" enabled_by_default="true" />
|
||||
<inspection_tool class="ReflectionForUnavailableAnnotation" enabled="true" level="ERROR" enabled_by_default="true" />
|
||||
<inspection_tool class="ReplaceAllDot" enabled="true" level="ERROR" enabled_by_default="true" />
|
||||
<inspection_tool class="ResultOfObjectAllocationIgnored" enabled="true" level="WARNING" enabled_by_default="true">
|
||||
|
|
|
@ -1,26 +1,34 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<project version="4">
|
||||
<component name="EntryPointsManager">
|
||||
<list size="12">
|
||||
<list size="14">
|
||||
<item index="0" class="java.lang.String" itemvalue="com.fasterxml.jackson.annotation.JsonCreator" />
|
||||
<item index="1" class="java.lang.String" itemvalue="com.fasterxml.jackson.annotation.JsonProperty" />
|
||||
<item index="2" class="java.lang.String" itemvalue="com.fasterxml.jackson.annotation.JsonValue" />
|
||||
<item index="3" class="java.lang.String" itemvalue="com.google.inject.Inject" />
|
||||
<item index="4" class="java.lang.String" itemvalue="com.google.inject.Provides" />
|
||||
<item index="5" class="java.lang.String" itemvalue="io.druid.annotations.UsedInGeneratedCode" />
|
||||
<item index="6" class="java.lang.String" itemvalue="io.druid.guice.annotations.ExtensionPoint" />
|
||||
<item index="7" class="java.lang.String" itemvalue="io.druid.guice.annotations.PublicApi" />
|
||||
<item index="8" class="java.lang.String" itemvalue="io.druid.java.util.common.lifecycle.LifecycleStart" />
|
||||
<item index="9" class="java.lang.String" itemvalue="io.druid.java.util.common.lifecycle.LifecycleStop" />
|
||||
<item index="10" class="java.lang.String" itemvalue="javax.inject.Inject" />
|
||||
<item index="11" class="java.lang.String" itemvalue="org.openjdk.jmh.annotations.Benchmark" />
|
||||
<item index="5" class="java.lang.String" itemvalue="io.airlift.airline.Command" />
|
||||
<item index="6" class="java.lang.String" itemvalue="io.druid.annotations.UsedByJUnitParamsRunner" />
|
||||
<item index="7" class="java.lang.String" itemvalue="io.druid.annotations.UsedInGeneratedCode" />
|
||||
<item index="8" class="java.lang.String" itemvalue="io.druid.guice.annotations.ExtensionPoint" />
|
||||
<item index="9" class="java.lang.String" itemvalue="io.druid.guice.annotations.PublicApi" />
|
||||
<item index="10" class="java.lang.String" itemvalue="io.druid.java.util.common.lifecycle.LifecycleStart" />
|
||||
<item index="11" class="java.lang.String" itemvalue="io.druid.java.util.common.lifecycle.LifecycleStop" />
|
||||
<item index="12" class="java.lang.String" itemvalue="javax.inject.Inject" />
|
||||
<item index="13" class="java.lang.String" itemvalue="org.openjdk.jmh.annotations.Benchmark" />
|
||||
</list>
|
||||
<pattern value="io.druid.cli.GuiceRunnable" hierarchically="true" method="run" />
|
||||
<pattern value="io.druid.cli.GuiceRunnable" hierarchically="true" />
|
||||
<pattern value="io.druid.initialization.DruidModule" hierarchically="true" method="getJacksonModules" />
|
||||
<writeAnnotations>
|
||||
<writeAnnotation name="com.fasterxml.jackson.annotation.JacksonInject" />
|
||||
<writeAnnotation name="com.fasterxml.jackson.annotation.JsonProperty" />
|
||||
<writeAnnotation name="com.google.caliper.Param" />
|
||||
<writeAnnotation name="io.airlift.airline.Option" />
|
||||
<writeAnnotation name="org.easymock.Mock" />
|
||||
<writeAnnotation name="org.mockito.Mock" />
|
||||
<writeAnnotation name="org.openjdk.jmh.annotations.Param" />
|
||||
<writeAnnotation name="org.powermock.api.easymock.annotation.Mock" />
|
||||
</writeAnnotations>
|
||||
</component>
|
||||
<component name="MavenProjectsManager">
|
||||
|
|
|
@ -1,3 +1,7 @@
|
|||
<component name="DependencyValidationManager">
|
||||
<scope name="UnusedInspectionsScope" pattern="src[druid-processing]:*..*" />
|
||||
<scope name="UnusedInspectionsScope" pattern="src[java-util]:*..*" />
|
||||
<scope name="UnusedInspectionsScope" pattern="src[druid-common]:*..*" />
|
||||
<scope name="UnusedInspectionsScope" pattern="src[extendedset]:*..*" />
|
||||
<scope name="UnusedInspectionsScope" pattern="src[druid-indexing-service]:*..*" />
|
||||
</component>
|
|
@ -25,6 +25,7 @@ import io.druid.guice.annotations.ExtensionPoint;
|
|||
import io.druid.java.util.common.parsers.ParseException;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Initialization method that connects up the FirehoseV2. If this method returns successfully it should be safe to
|
||||
* call start() on the returned FirehoseV2 (which might subsequently block).
|
||||
|
@ -41,5 +42,10 @@ import java.io.IOException;
|
|||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
||||
public interface FirehoseFactoryV2<T extends InputRowParser>
|
||||
{
|
||||
/**
|
||||
* This method is declared to throw {@link IOException}, although it's not thrown in the implementations in Druid
|
||||
* code, for compatibility with third-party extensions.
|
||||
*/
|
||||
@SuppressWarnings("RedundantThrows")
|
||||
FirehoseV2 connect(T parser, Object lastCommit) throws IOException, ParseException;
|
||||
}
|
||||
|
|
|
@ -52,7 +52,7 @@ public interface FirehoseV2 extends Closeable
|
|||
/**
|
||||
* For initial start
|
||||
*/
|
||||
void start() throws Exception;
|
||||
void start();
|
||||
|
||||
/**
|
||||
* Advance the firehose to the next offset. Implementations of this interface should make sure that
|
||||
|
|
|
@ -27,7 +27,6 @@ import io.druid.segment.loading.DataSegmentArchiver;
|
|||
import io.druid.segment.loading.DataSegmentFinder;
|
||||
import io.druid.segment.loading.DataSegmentKiller;
|
||||
import io.druid.segment.loading.DataSegmentMover;
|
||||
import io.druid.segment.loading.DataSegmentPuller;
|
||||
import io.druid.segment.loading.DataSegmentPusher;
|
||||
import io.druid.tasklogs.TaskLogs;
|
||||
|
||||
|
@ -36,10 +35,6 @@ import io.druid.tasklogs.TaskLogs;
|
|||
@PublicApi
|
||||
public class Binders
|
||||
{
|
||||
public static MapBinder<String, DataSegmentPuller> dataSegmentPullerBinder(Binder binder)
|
||||
{
|
||||
return MapBinder.newMapBinder(binder, String.class, DataSegmentPuller.class);
|
||||
}
|
||||
|
||||
public static MapBinder<String, DataSegmentKiller> dataSegmentKillerBinder(Binder binder)
|
||||
{
|
||||
|
|
|
@ -1,42 +0,0 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets 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 io.druid.segment.loading;
|
||||
|
||||
import io.druid.guice.annotations.ExtensionPoint;
|
||||
import io.druid.timeline.DataSegment;
|
||||
|
||||
import java.io.File;
|
||||
|
||||
/**
|
||||
* A DataSegmentPuller is responsible for pulling data for a particular segment into a particular directory
|
||||
*/
|
||||
@ExtensionPoint
|
||||
public interface DataSegmentPuller
|
||||
{
|
||||
/**
|
||||
* Pull down segment files for the given DataSegment and put them in the given directory.
|
||||
*
|
||||
* @param segment The segment to pull down files for
|
||||
* @param dir The directory to store the files in
|
||||
*
|
||||
* @throws SegmentLoadingException if there are any errors
|
||||
*/
|
||||
void getSegmentFiles(DataSegment segment, File dir) throws SegmentLoadingException;
|
||||
}
|
|
@ -24,32 +24,31 @@ import com.google.common.io.ByteSource;
|
|||
import io.druid.java.util.common.logger.Logger;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
|
||||
public class NoopTaskLogs implements TaskLogs
|
||||
{
|
||||
private final Logger log = new Logger(TaskLogs.class);
|
||||
|
||||
@Override
|
||||
public Optional<ByteSource> streamTaskLog(String taskid, long offset) throws IOException
|
||||
public Optional<ByteSource> streamTaskLog(String taskid, long offset)
|
||||
{
|
||||
return Optional.absent();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void pushTaskLog(String taskid, File logFile) throws IOException
|
||||
public void pushTaskLog(String taskid, File logFile)
|
||||
{
|
||||
log.info("Not pushing logs for task: %s", taskid);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void killAll() throws IOException
|
||||
public void killAll()
|
||||
{
|
||||
log.info("Noop: No task logs are deleted.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void killOlderThan(long timestamp) throws IOException
|
||||
public void killOlderThan(long timestamp)
|
||||
{
|
||||
log.info("Noop: No task logs are deleted.");
|
||||
}
|
||||
|
|
|
@ -72,7 +72,7 @@ public class CompressionUtils
|
|||
* @throws IOException on propogated IO exception, IAE if it cannot determine the proper new name for `pulledFile`
|
||||
*/
|
||||
@Deprecated // See description for alternative
|
||||
public static void gunzip(File pulledFile, File outDir) throws IOException
|
||||
public static void gunzip(File pulledFile, File outDir)
|
||||
{
|
||||
final File outFile = new File(outDir, io.druid.java.util.common.CompressionUtils.getGzBaseName(pulledFile.getName()));
|
||||
io.druid.java.util.common.CompressionUtils.gunzip(pulledFile, outFile);
|
||||
|
|
|
@ -28,7 +28,7 @@ import java.util.Collections;
|
|||
public class CSVParseSpecTest
|
||||
{
|
||||
@Test(expected = IllegalArgumentException.class)
|
||||
public void testColumnMissing() throws Exception
|
||||
public void testColumnMissing()
|
||||
{
|
||||
@SuppressWarnings("unused") // expected exception
|
||||
final ParseSpec spec = new CSVParseSpec(
|
||||
|
@ -50,7 +50,7 @@ public class CSVParseSpecTest
|
|||
}
|
||||
|
||||
@Test(expected = IllegalArgumentException.class)
|
||||
public void testComma() throws Exception
|
||||
public void testComma()
|
||||
{
|
||||
@SuppressWarnings("unused") // expected exception
|
||||
final ParseSpec spec = new CSVParseSpec(
|
||||
|
|
|
@ -59,7 +59,7 @@ public class DelimitedParseSpecTest
|
|||
}
|
||||
|
||||
@Test(expected = IllegalArgumentException.class)
|
||||
public void testColumnMissing() throws Exception
|
||||
public void testColumnMissing()
|
||||
{
|
||||
@SuppressWarnings("unused") // expected exception
|
||||
final ParseSpec spec = new DelimitedParseSpec(
|
||||
|
@ -82,7 +82,7 @@ public class DelimitedParseSpecTest
|
|||
}
|
||||
|
||||
@Test(expected = IllegalArgumentException.class)
|
||||
public void testComma() throws Exception
|
||||
public void testComma()
|
||||
{
|
||||
@SuppressWarnings("unused") // expected exception
|
||||
final ParseSpec spec = new DelimitedParseSpec(
|
||||
|
|
|
@ -45,7 +45,7 @@ import java.util.stream.IntStream;
|
|||
public class FileIteratingFirehoseTest
|
||||
{
|
||||
@Parameters(name = "{0}, {1}")
|
||||
public static Collection<Object[]> constructorFeeder() throws IOException
|
||||
public static Collection<Object[]> constructorFeeder()
|
||||
{
|
||||
final List<List<String>> inputTexts = ImmutableList.of(
|
||||
ImmutableList.of("2000,foo"),
|
||||
|
@ -133,14 +133,14 @@ public class FileIteratingFirehoseTest
|
|||
final LineIterator lineIterator = new LineIterator(new Reader()
|
||||
{
|
||||
@Override
|
||||
public int read(char[] cbuf, int off, int len) throws IOException
|
||||
public int read(char[] cbuf, int off, int len)
|
||||
{
|
||||
System.arraycopy(LINE_CHARS, 0, cbuf, 0, LINE_CHARS.length);
|
||||
return LINE_CHARS.length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
public void close()
|
||||
{
|
||||
throw new RuntimeException("close test for FileIteratingFirehose");
|
||||
}
|
||||
|
@ -162,7 +162,7 @@ public class FileIteratingFirehoseTest
|
|||
private boolean closed;
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
public void close()
|
||||
{
|
||||
closed = true;
|
||||
}
|
||||
|
|
|
@ -30,7 +30,7 @@ import java.util.Map;
|
|||
public class JSONLowercaseParseSpecTest
|
||||
{
|
||||
@Test
|
||||
public void testLowercasing() throws Exception
|
||||
public void testLowercasing()
|
||||
{
|
||||
JSONLowercaseParseSpec spec = new JSONLowercaseParseSpec(
|
||||
new TimestampSpec(
|
||||
|
|
|
@ -34,7 +34,7 @@ public class ParseSpecTest
|
|||
public ExpectedException expectedException = ExpectedException.none();
|
||||
|
||||
@Test(expected = ParseException.class)
|
||||
public void testDuplicateNames() throws Exception
|
||||
public void testDuplicateNames()
|
||||
{
|
||||
@SuppressWarnings("unused") // expected exception
|
||||
final ParseSpec spec = new DelimitedParseSpec(
|
||||
|
@ -57,7 +57,7 @@ public class ParseSpecTest
|
|||
}
|
||||
|
||||
@Test(expected = IllegalArgumentException.class)
|
||||
public void testDimAndDimExcluOverlap() throws Exception
|
||||
public void testDimAndDimExcluOverlap()
|
||||
{
|
||||
@SuppressWarnings("unused") // expected exception
|
||||
final ParseSpec spec = new DelimitedParseSpec(
|
||||
|
@ -80,7 +80,7 @@ public class ParseSpecTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testDimExclusionDuplicate() throws Exception
|
||||
public void testDimExclusionDuplicate()
|
||||
{
|
||||
@SuppressWarnings("unused") // expected exception
|
||||
final ParseSpec spec = new DelimitedParseSpec(
|
||||
|
@ -103,7 +103,7 @@ public class ParseSpecTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testDefaultTimestampSpec() throws Exception
|
||||
public void testDefaultTimestampSpec()
|
||||
{
|
||||
expectedException.expect(NullPointerException.class);
|
||||
expectedException.expectMessage("parseSpec requires timestampSpec");
|
||||
|
@ -124,7 +124,7 @@ public class ParseSpecTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testDimensionSpecRequired() throws Exception
|
||||
public void testDimensionSpecRequired()
|
||||
{
|
||||
expectedException.expect(NullPointerException.class);
|
||||
expectedException.expectMessage("parseSpec requires dimensionSpec");
|
||||
|
|
|
@ -29,7 +29,7 @@ import org.junit.Test;
|
|||
public class TimestampSpecTest
|
||||
{
|
||||
@Test
|
||||
public void testExtractTimestamp() throws Exception
|
||||
public void testExtractTimestamp()
|
||||
{
|
||||
TimestampSpec spec = new TimestampSpec("TIMEstamp", "yyyy-MM-dd", null);
|
||||
Assert.assertEquals(
|
||||
|
@ -39,7 +39,7 @@ public class TimestampSpecTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testExtractTimestampWithMissingTimestampColumn() throws Exception
|
||||
public void testExtractTimestampWithMissingTimestampColumn()
|
||||
{
|
||||
TimestampSpec spec = new TimestampSpec(null, null, DateTimes.EPOCH);
|
||||
Assert.assertEquals(
|
||||
|
@ -49,7 +49,7 @@ public class TimestampSpecTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testContextualTimestampList() throws Exception
|
||||
public void testContextualTimestampList()
|
||||
{
|
||||
String DATE_FORMAT = "yyyy-MM-dd'T'HH:mm:ss";
|
||||
String[] dates = new String[]{
|
||||
|
|
|
@ -570,7 +570,7 @@ public class PrefetchableTextFilesFirehoseFactoryTest
|
|||
}
|
||||
|
||||
@Override
|
||||
protected InputStream wrapObjectStream(File object, InputStream stream) throws IOException
|
||||
protected InputStream wrapObjectStream(File object, InputStream stream)
|
||||
{
|
||||
return stream;
|
||||
}
|
||||
|
|
|
@ -51,7 +51,7 @@ public class ConditionalMultibindTest
|
|||
private Properties props;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception
|
||||
public void setUp()
|
||||
{
|
||||
props = new Properties();
|
||||
}
|
||||
|
|
|
@ -104,7 +104,7 @@ public class JsonConfiguratorTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testsimpleConfigurate() throws Exception
|
||||
public void testsimpleConfigurate()
|
||||
{
|
||||
final JsonConfigurator configurator = new JsonConfigurator(mapper, validator);
|
||||
properties.setProperty(PROP_PREFIX + "prop1", "prop1");
|
||||
|
|
|
@ -42,7 +42,7 @@ public class PolyBindTest
|
|||
private Properties props;
|
||||
private Injector injector;
|
||||
|
||||
public void setUp(Module... modules) throws Exception
|
||||
public void setUp(Module... modules)
|
||||
{
|
||||
props = new Properties();
|
||||
injector = Guice.createInjector(
|
||||
|
@ -66,7 +66,7 @@ public class PolyBindTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testSanity() throws Exception
|
||||
public void testSanity()
|
||||
{
|
||||
setUp(
|
||||
new Module()
|
||||
|
|
|
@ -217,7 +217,7 @@ public class DataSegmentTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testBucketMonthComparator() throws Exception
|
||||
public void testBucketMonthComparator()
|
||||
{
|
||||
DataSegment[] sortedOrder = {
|
||||
makeDataSegment("test1", "2011-01-01/2011-01-02", "a"),
|
||||
|
|
|
@ -50,7 +50,6 @@ import org.openjdk.jmh.annotations.Setup;
|
|||
import org.openjdk.jmh.annotations.State;
|
||||
import org.openjdk.jmh.annotations.Warmup;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
@ -152,7 +151,7 @@ public class BoundFilterBenchmark
|
|||
BitmapIndexSelector selector;
|
||||
|
||||
@Setup
|
||||
public void setup() throws IOException
|
||||
public void setup()
|
||||
{
|
||||
step = (END_INT - START_INT) / cardinality;
|
||||
final BitmapFactory bitmapFactory = new RoaringBitmapFactory();
|
||||
|
|
|
@ -106,7 +106,7 @@ public class CompressedColumnarIntsBenchmark
|
|||
WritableByteChannel channel = new WritableByteChannel()
|
||||
{
|
||||
@Override
|
||||
public int write(ByteBuffer src) throws IOException
|
||||
public int write(ByteBuffer src)
|
||||
{
|
||||
int size = src.remaining();
|
||||
buffer.put(src);
|
||||
|
@ -120,7 +120,7 @@ public class CompressedColumnarIntsBenchmark
|
|||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
public void close()
|
||||
{
|
||||
}
|
||||
};
|
||||
|
|
|
@ -121,7 +121,7 @@ public class CompressedVSizeColumnarMultiIntsBenchmark
|
|||
WritableByteChannel channel = new WritableByteChannel()
|
||||
{
|
||||
@Override
|
||||
public int write(ByteBuffer src) throws IOException
|
||||
public int write(ByteBuffer src)
|
||||
{
|
||||
int size = src.remaining();
|
||||
buffer.put(src);
|
||||
|
@ -135,7 +135,7 @@ public class CompressedVSizeColumnarMultiIntsBenchmark
|
|||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
public void close()
|
||||
{
|
||||
}
|
||||
};
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package io.druid.benchmark;
|
||||
|
||||
import com.google.common.collect.Sets;
|
||||
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.server.router.ConsistentHasher;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
|
@ -36,7 +35,6 @@ import org.openjdk.jmh.annotations.State;
|
|||
import org.openjdk.jmh.annotations.Warmup;
|
||||
import org.openjdk.jmh.infra.Blackhole;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
@ -57,7 +55,7 @@ public class ConsistentHasherBenchmark
|
|||
Set<String> servers;
|
||||
|
||||
@Setup
|
||||
public void setup() throws IOException
|
||||
public void setup()
|
||||
{
|
||||
hasher = new ConsistentHasher(null);
|
||||
uuids = new ArrayList<>();
|
||||
|
@ -85,7 +83,7 @@ public class ConsistentHasherBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void hash(Blackhole blackhole) throws Exception
|
||||
public void hash(Blackhole blackhole)
|
||||
{
|
||||
for (String uuid : uuids) {
|
||||
String server = hasher.findKey(StringUtils.toUtf8(uuid));
|
||||
|
|
|
@ -52,7 +52,6 @@ import org.openjdk.jmh.annotations.Setup;
|
|||
import org.openjdk.jmh.annotations.State;
|
||||
import org.openjdk.jmh.annotations.Warmup;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
@ -114,7 +113,7 @@ public class DimensionPredicateFilterBenchmark
|
|||
BitmapIndexSelector selector;
|
||||
|
||||
@Setup
|
||||
public void setup() throws IOException
|
||||
public void setup()
|
||||
{
|
||||
final BitmapFactory bitmapFactory = new RoaringBitmapFactory();
|
||||
final BitmapSerdeFactory serdeFactory = new RoaringBitmapSerdeFactory(null);
|
||||
|
|
|
@ -79,7 +79,7 @@ public class ExpressionAggregationBenchmark
|
|||
private ByteBuffer aggregationBuffer = ByteBuffer.allocate(Double.BYTES);
|
||||
|
||||
@Setup(Level.Trial)
|
||||
public void setup() throws Exception
|
||||
public void setup()
|
||||
{
|
||||
final BenchmarkSchemaInfo schemaInfo = new BenchmarkSchemaInfo(
|
||||
ImmutableList.of(
|
||||
|
@ -131,21 +131,21 @@ public class ExpressionAggregationBenchmark
|
|||
}
|
||||
|
||||
@Benchmark
|
||||
public void queryUsingJavaScript(Blackhole blackhole) throws Exception
|
||||
public void queryUsingJavaScript(Blackhole blackhole)
|
||||
{
|
||||
final Double result = compute(javaScriptAggregatorFactory::factorizeBuffered);
|
||||
blackhole.consume(result);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
public void queryUsingExpression(Blackhole blackhole) throws Exception
|
||||
public void queryUsingExpression(Blackhole blackhole)
|
||||
{
|
||||
final Double result = compute(expressionAggregatorFactory::factorizeBuffered);
|
||||
blackhole.consume(result);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
public void queryUsingNative(Blackhole blackhole) throws Exception
|
||||
public void queryUsingNative(Blackhole blackhole)
|
||||
{
|
||||
final Double result = compute(
|
||||
columnSelectorFactory ->
|
||||
|
|
|
@ -76,7 +76,7 @@ public class ExpressionSelectorBenchmark
|
|||
private QueryableIndex index;
|
||||
|
||||
@Setup(Level.Trial)
|
||||
public void setup() throws Exception
|
||||
public void setup()
|
||||
{
|
||||
final BenchmarkSchemaInfo schemaInfo = new BenchmarkSchemaInfo(
|
||||
ImmutableList.of(
|
||||
|
@ -123,7 +123,7 @@ public class ExpressionSelectorBenchmark
|
|||
}
|
||||
|
||||
@Benchmark
|
||||
public void timeFloorUsingExpression(Blackhole blackhole) throws Exception
|
||||
public void timeFloorUsingExpression(Blackhole blackhole)
|
||||
{
|
||||
final Sequence<Cursor> cursors = new QueryableIndexStorageAdapter(index).makeCursors(
|
||||
null,
|
||||
|
@ -158,7 +158,7 @@ public class ExpressionSelectorBenchmark
|
|||
}
|
||||
|
||||
@Benchmark
|
||||
public void timeFloorUsingExtractionFn(Blackhole blackhole) throws Exception
|
||||
public void timeFloorUsingExtractionFn(Blackhole blackhole)
|
||||
{
|
||||
final Sequence<Cursor> cursors = new QueryableIndexStorageAdapter(index).makeCursors(
|
||||
null,
|
||||
|
@ -189,7 +189,7 @@ public class ExpressionSelectorBenchmark
|
|||
}
|
||||
|
||||
@Benchmark
|
||||
public void timeFloorUsingCursor(Blackhole blackhole) throws Exception
|
||||
public void timeFloorUsingCursor(Blackhole blackhole)
|
||||
{
|
||||
final Sequence<Cursor> cursors = new QueryableIndexStorageAdapter(index).makeCursors(
|
||||
null,
|
||||
|
@ -220,7 +220,7 @@ public class ExpressionSelectorBenchmark
|
|||
}
|
||||
|
||||
@Benchmark
|
||||
public void strlenUsingExpressionAsLong(Blackhole blackhole) throws Exception
|
||||
public void strlenUsingExpressionAsLong(Blackhole blackhole)
|
||||
{
|
||||
final Sequence<Cursor> cursors = new QueryableIndexStorageAdapter(index).makeCursors(
|
||||
null,
|
||||
|
@ -252,7 +252,7 @@ public class ExpressionSelectorBenchmark
|
|||
}
|
||||
|
||||
@Benchmark
|
||||
public void strlenUsingExpressionAsString(Blackhole blackhole) throws Exception
|
||||
public void strlenUsingExpressionAsString(Blackhole blackhole)
|
||||
{
|
||||
final Sequence<Cursor> cursors = new QueryableIndexStorageAdapter(index).makeCursors(
|
||||
null,
|
||||
|
@ -287,7 +287,7 @@ public class ExpressionSelectorBenchmark
|
|||
}
|
||||
|
||||
@Benchmark
|
||||
public void strlenUsingExtractionFn(Blackhole blackhole) throws Exception
|
||||
public void strlenUsingExtractionFn(Blackhole blackhole)
|
||||
{
|
||||
final Sequence<Cursor> cursors = new QueryableIndexStorageAdapter(index).makeCursors(
|
||||
null,
|
||||
|
|
|
@ -35,7 +35,6 @@ import io.druid.java.util.common.guava.Sequence;
|
|||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.js.JavaScriptConfig;
|
||||
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
|
||||
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||
import io.druid.query.extraction.ExtractionFn;
|
||||
|
@ -73,6 +72,7 @@ import io.druid.segment.filter.OrFilter;
|
|||
import io.druid.segment.filter.SelectorFilter;
|
||||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
import io.druid.segment.serde.ComplexMetrics;
|
||||
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.joda.time.Interval;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
|
@ -238,7 +238,7 @@ public class FilterPartitionBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void stringRead(Blackhole blackhole) throws Exception
|
||||
public void stringRead(Blackhole blackhole)
|
||||
{
|
||||
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
|
||||
Sequence<Cursor> cursors = makeCursors(sa, null);
|
||||
|
@ -253,7 +253,7 @@ public class FilterPartitionBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void longRead(Blackhole blackhole) throws Exception
|
||||
public void longRead(Blackhole blackhole)
|
||||
{
|
||||
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
|
||||
Sequence<Cursor> cursors = makeCursors(sa, null);
|
||||
|
@ -268,7 +268,7 @@ public class FilterPartitionBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void timeFilterNone(Blackhole blackhole) throws Exception
|
||||
public void timeFilterNone(Blackhole blackhole)
|
||||
{
|
||||
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
|
||||
Sequence<Cursor> cursors = makeCursors(sa, timeFilterNone);
|
||||
|
@ -283,7 +283,7 @@ public class FilterPartitionBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void timeFilterHalf(Blackhole blackhole) throws Exception
|
||||
public void timeFilterHalf(Blackhole blackhole)
|
||||
{
|
||||
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
|
||||
Sequence<Cursor> cursors = makeCursors(sa, timeFilterHalf);
|
||||
|
@ -298,7 +298,7 @@ public class FilterPartitionBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void timeFilterAll(Blackhole blackhole) throws Exception
|
||||
public void timeFilterAll(Blackhole blackhole)
|
||||
{
|
||||
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
|
||||
Sequence<Cursor> cursors = makeCursors(sa, timeFilterAll);
|
||||
|
@ -313,7 +313,7 @@ public class FilterPartitionBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void readWithPreFilter(Blackhole blackhole) throws Exception
|
||||
public void readWithPreFilter(Blackhole blackhole)
|
||||
{
|
||||
Filter filter = new SelectorFilter("dimSequential", "199");
|
||||
|
||||
|
@ -330,7 +330,7 @@ public class FilterPartitionBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void readWithPostFilter(Blackhole blackhole) throws Exception
|
||||
public void readWithPostFilter(Blackhole blackhole)
|
||||
{
|
||||
Filter filter = new NoBitmapSelectorFilter("dimSequential", "199");
|
||||
|
||||
|
@ -347,7 +347,7 @@ public class FilterPartitionBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void readWithExFnPreFilter(Blackhole blackhole) throws Exception
|
||||
public void readWithExFnPreFilter(Blackhole blackhole)
|
||||
{
|
||||
Filter filter = new SelectorDimFilter("dimSequential", "super-199", JS_EXTRACTION_FN).toFilter();
|
||||
|
||||
|
@ -364,7 +364,7 @@ public class FilterPartitionBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void readWithExFnPostFilter(Blackhole blackhole) throws Exception
|
||||
public void readWithExFnPostFilter(Blackhole blackhole)
|
||||
{
|
||||
Filter filter = new NoBitmapSelectorDimFilter("dimSequential", "super-199", JS_EXTRACTION_FN).toFilter();
|
||||
|
||||
|
@ -381,7 +381,7 @@ public class FilterPartitionBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void readOrFilter(Blackhole blackhole) throws Exception
|
||||
public void readOrFilter(Blackhole blackhole)
|
||||
{
|
||||
Filter filter = new NoBitmapSelectorFilter("dimSequential", "199");
|
||||
Filter filter2 = new AndFilter(Arrays.<Filter>asList(new SelectorFilter("dimMultivalEnumerated2", "Corundum"), new NoBitmapSelectorFilter("dimMultivalEnumerated", "Bar")));
|
||||
|
@ -400,7 +400,7 @@ public class FilterPartitionBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void readOrFilterCNF(Blackhole blackhole) throws Exception
|
||||
public void readOrFilterCNF(Blackhole blackhole)
|
||||
{
|
||||
Filter filter = new NoBitmapSelectorFilter("dimSequential", "199");
|
||||
Filter filter2 = new AndFilter(Arrays.<Filter>asList(new SelectorFilter("dimMultivalEnumerated2", "Corundum"), new NoBitmapSelectorFilter("dimMultivalEnumerated", "Bar")));
|
||||
|
@ -419,7 +419,7 @@ public class FilterPartitionBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void readComplexOrFilter(Blackhole blackhole) throws Exception
|
||||
public void readComplexOrFilter(Blackhole blackhole)
|
||||
{
|
||||
DimFilter dimFilter1 = new OrDimFilter(Arrays.<DimFilter>asList(
|
||||
new SelectorDimFilter("dimSequential", "199", null),
|
||||
|
@ -461,7 +461,7 @@ public class FilterPartitionBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void readComplexOrFilterCNF(Blackhole blackhole) throws Exception
|
||||
public void readComplexOrFilterCNF(Blackhole blackhole)
|
||||
{
|
||||
DimFilter dimFilter1 = new OrDimFilter(Arrays.<DimFilter>asList(
|
||||
new SelectorDimFilter("dimSequential", "199", null),
|
||||
|
|
|
@ -263,7 +263,7 @@ public class FilteredAggregatorBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleIncrementalIndex(Blackhole blackhole) throws Exception
|
||||
public void querySingleIncrementalIndex(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Result<TimeseriesResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -280,7 +280,7 @@ public class FilteredAggregatorBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndex(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndex(Blackhole blackhole)
|
||||
{
|
||||
final QueryRunner<Result<TimeseriesResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
|
|
@ -23,8 +23,8 @@ package io.druid.benchmark;
|
|||
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.io.Files;
|
||||
import io.druid.segment.data.CompressedColumnarFloatsSupplier;
|
||||
import io.druid.segment.data.ColumnarFloats;
|
||||
import io.druid.segment.data.CompressedColumnarFloatsSupplier;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
import org.openjdk.jmh.annotations.Fork;
|
||||
|
@ -39,7 +39,6 @@ import org.openjdk.jmh.annotations.Warmup;
|
|||
import org.openjdk.jmh.infra.Blackhole;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
import java.util.Random;
|
||||
|
@ -76,7 +75,7 @@ public class FloatCompressionBenchmark
|
|||
}
|
||||
|
||||
@Benchmark
|
||||
public void readContinuous(Blackhole bh) throws IOException
|
||||
public void readContinuous(Blackhole bh)
|
||||
{
|
||||
ColumnarFloats columnarFloats = supplier.get();
|
||||
int count = columnarFloats.size();
|
||||
|
@ -89,7 +88,7 @@ public class FloatCompressionBenchmark
|
|||
}
|
||||
|
||||
@Benchmark
|
||||
public void readSkipping(Blackhole bh) throws IOException
|
||||
public void readSkipping(Blackhole bh)
|
||||
{
|
||||
ColumnarFloats columnarFloats = supplier.get();
|
||||
int count = columnarFloats.size();
|
||||
|
|
|
@ -23,17 +23,16 @@ import com.google.common.collect.ImmutableList;
|
|||
import io.druid.benchmark.datagen.BenchmarkColumnSchema;
|
||||
import io.druid.benchmark.datagen.BenchmarkColumnValueGenerator;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium;
|
||||
import io.druid.segment.column.ValueType;
|
||||
import io.druid.segment.data.ColumnarFloatsSerializer;
|
||||
import io.druid.segment.data.CompressionFactory;
|
||||
import io.druid.segment.data.CompressionStrategy;
|
||||
import io.druid.segment.data.ColumnarFloatsSerializer;
|
||||
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.Writer;
|
||||
import java.net.URISyntaxException;
|
||||
import java.nio.ByteOrder;
|
||||
import java.nio.channels.FileChannel;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
|
@ -55,7 +54,7 @@ public class FloatCompressionBenchmarkFileGenerator
|
|||
|
||||
private static String dirPath = "floatCompress/";
|
||||
|
||||
public static void main(String[] args) throws IOException, URISyntaxException
|
||||
public static void main(String[] args) throws IOException
|
||||
{
|
||||
if (args.length >= 1) {
|
||||
dirPath = args[0];
|
||||
|
|
|
@ -39,7 +39,6 @@ import io.druid.data.input.InputRow;
|
|||
import io.druid.data.input.Row;
|
||||
import io.druid.hll.HyperLogLogHash;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.concurrent.Execs;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
|
@ -100,7 +99,6 @@ import java.util.Collections;
|
|||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
|
@ -148,8 +146,6 @@ public class GroupByTypeInterfaceBenchmark
|
|||
private GroupByQuery floatQuery;
|
||||
private GroupByQuery longQuery;
|
||||
|
||||
private ExecutorService executorService;
|
||||
|
||||
static {
|
||||
JSON_MAPPER = new DefaultObjectMapper();
|
||||
INDEX_IO = new IndexIO(
|
||||
|
@ -290,7 +286,6 @@ public class GroupByTypeInterfaceBenchmark
|
|||
if (ComplexMetrics.getSerdeForType("hyperUnique") == null) {
|
||||
ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde(HyperLogLogHash.getDefault()));
|
||||
}
|
||||
executorService = Execs.multiThreaded(numProcessingThreads, "GroupByThreadPool[%d]");
|
||||
|
||||
setupQueries();
|
||||
|
||||
|
@ -483,7 +478,7 @@ public class GroupByTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexStringOnly(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexStringOnly(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -501,7 +496,7 @@ public class GroupByTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexLongOnly(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexLongOnly(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -519,7 +514,7 @@ public class GroupByTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexFloatOnly(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexFloatOnly(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -537,7 +532,7 @@ public class GroupByTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexNumericOnly(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexNumericOnly(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -555,7 +550,7 @@ public class GroupByTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexNumericThenString(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexNumericThenString(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -586,7 +581,7 @@ public class GroupByTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexLongThenString(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexLongThenString(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -616,7 +611,7 @@ public class GroupByTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexLongThenFloat(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexLongThenFloat(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -646,7 +641,7 @@ public class GroupByTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexStringThenNumeric(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexStringThenNumeric(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -676,7 +671,7 @@ public class GroupByTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexStringThenLong(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexStringThenLong(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -706,7 +701,7 @@ public class GroupByTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexStringTwice(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexStringTwice(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -736,7 +731,7 @@ public class GroupByTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexLongTwice(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexLongTwice(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -767,7 +762,7 @@ public class GroupByTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexFloatTwice(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexFloatTwice(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -797,7 +792,7 @@ public class GroupByTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexFloatThenLong(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexFloatThenLong(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -827,7 +822,7 @@ public class GroupByTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexFloatThenString(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexFloatThenString(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
|
|
@ -39,7 +39,6 @@ import org.openjdk.jmh.annotations.Setup;
|
|||
import org.openjdk.jmh.annotations.State;
|
||||
import org.openjdk.jmh.infra.Blackhole;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
@ -128,7 +127,7 @@ public class IncrementalIndexRowTypeBenchmark
|
|||
}
|
||||
|
||||
@Setup
|
||||
public void setup() throws IOException
|
||||
public void setup()
|
||||
{
|
||||
rng = new Random(9999);
|
||||
|
||||
|
@ -146,7 +145,7 @@ public class IncrementalIndexRowTypeBenchmark
|
|||
}
|
||||
|
||||
@Setup(Level.Iteration)
|
||||
public void setup2() throws IOException
|
||||
public void setup2()
|
||||
{
|
||||
incIndex = makeIncIndex();
|
||||
incFloatIndex = makeIncIndex();
|
||||
|
|
|
@ -52,7 +52,6 @@ import org.openjdk.jmh.annotations.State;
|
|||
import org.openjdk.jmh.annotations.Warmup;
|
||||
import org.openjdk.jmh.infra.Blackhole;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
@ -113,7 +112,7 @@ public class LikeFilterBenchmark
|
|||
BitmapIndexSelector selector;
|
||||
|
||||
@Setup
|
||||
public void setup() throws IOException
|
||||
public void setup()
|
||||
{
|
||||
step = (END_INT - START_INT) / cardinality;
|
||||
final BitmapFactory bitmapFactory = new RoaringBitmapFactory();
|
||||
|
|
|
@ -37,7 +37,6 @@ import org.openjdk.jmh.annotations.Warmup;
|
|||
import org.openjdk.jmh.infra.Blackhole;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
import java.util.Random;
|
||||
|
@ -79,7 +78,7 @@ public class LongCompressionBenchmark
|
|||
}
|
||||
|
||||
@Benchmark
|
||||
public void readContinuous(Blackhole bh) throws IOException
|
||||
public void readContinuous(Blackhole bh)
|
||||
{
|
||||
ColumnarLongs columnarLongs = supplier.get();
|
||||
int count = columnarLongs.size();
|
||||
|
@ -92,7 +91,7 @@ public class LongCompressionBenchmark
|
|||
}
|
||||
|
||||
@Benchmark
|
||||
public void readSkipping(Blackhole bh) throws IOException
|
||||
public void readSkipping(Blackhole bh)
|
||||
{
|
||||
ColumnarLongs columnarLongs = supplier.get();
|
||||
int count = columnarLongs.size();
|
||||
|
|
|
@ -23,17 +23,16 @@ import com.google.common.collect.ImmutableList;
|
|||
import io.druid.benchmark.datagen.BenchmarkColumnSchema;
|
||||
import io.druid.benchmark.datagen.BenchmarkColumnValueGenerator;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium;
|
||||
import io.druid.segment.column.ValueType;
|
||||
import io.druid.segment.data.ColumnarLongsSerializer;
|
||||
import io.druid.segment.data.CompressionFactory;
|
||||
import io.druid.segment.data.CompressionStrategy;
|
||||
import io.druid.segment.data.ColumnarLongsSerializer;
|
||||
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.Writer;
|
||||
import java.net.URISyntaxException;
|
||||
import java.nio.ByteOrder;
|
||||
import java.nio.channels.FileChannel;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
|
@ -56,7 +55,7 @@ public class LongCompressionBenchmarkFileGenerator
|
|||
|
||||
private static String dirPath = "longCompress/";
|
||||
|
||||
public static void main(String[] args) throws IOException, URISyntaxException
|
||||
public static void main(String[] args) throws IOException
|
||||
{
|
||||
if (args.length >= 1) {
|
||||
dirPath = args[0];
|
||||
|
|
|
@ -35,7 +35,6 @@ import org.openjdk.jmh.annotations.State;
|
|||
import org.openjdk.jmh.annotations.Warmup;
|
||||
import org.openjdk.jmh.infra.Blackhole;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
@ -56,7 +55,7 @@ public class RendezvousHasherBenchmark
|
|||
Set<String> servers;
|
||||
|
||||
@Setup
|
||||
public void setup() throws IOException
|
||||
public void setup()
|
||||
{
|
||||
hasher = new RendezvousHasher();
|
||||
uuids = new ArrayList<>();
|
||||
|
@ -83,7 +82,7 @@ public class RendezvousHasherBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void hash(Blackhole blackhole) throws Exception
|
||||
public void hash(Blackhole blackhole)
|
||||
{
|
||||
for (String uuid : uuids) {
|
||||
String server = hasher.chooseNode(servers, StringUtils.toUtf8(uuid));
|
||||
|
|
|
@ -31,7 +31,6 @@ import org.openjdk.jmh.annotations.Scope;
|
|||
import org.openjdk.jmh.annotations.State;
|
||||
import org.openjdk.jmh.infra.Blackhole;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
|
@ -60,7 +59,7 @@ public class StupidPoolConcurrencyBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.Throughput)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void hammerQueue(BenchmarkPool pool, Blackhole blackhole) throws IOException
|
||||
public void hammerQueue(BenchmarkPool pool, Blackhole blackhole)
|
||||
{
|
||||
try (ResourceHolder<Object> holder = pool.pool.take()) {
|
||||
blackhole.consume(holder);
|
||||
|
|
|
@ -30,7 +30,6 @@ import io.druid.collections.StupidPool;
|
|||
import io.druid.data.input.InputRow;
|
||||
import io.druid.hll.HyperLogLogHash;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.concurrent.Execs;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
|
@ -90,7 +89,6 @@ import java.util.Collections;
|
|||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
// Benchmark for determining the interface overhead of TopN with multiple type implementations
|
||||
|
@ -107,9 +105,6 @@ public class TopNTypeInterfaceBenchmark
|
|||
@Param({"750000"})
|
||||
private int rowsPerSegment;
|
||||
|
||||
@Param({"basic.A"})
|
||||
private String schemaAndQuery;
|
||||
|
||||
@Param({"10"})
|
||||
private int threshold;
|
||||
|
||||
|
@ -129,8 +124,6 @@ public class TopNTypeInterfaceBenchmark
|
|||
private TopNQuery longQuery;
|
||||
private TopNQuery floatQuery;
|
||||
|
||||
private ExecutorService executorService;
|
||||
|
||||
static {
|
||||
JSON_MAPPER = new DefaultObjectMapper();
|
||||
INDEX_IO = new IndexIO(
|
||||
|
@ -242,8 +235,6 @@ public class TopNTypeInterfaceBenchmark
|
|||
ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde(HyperLogLogHash.getDefault()));
|
||||
}
|
||||
|
||||
executorService = Execs.multiThreaded(numSegments, "TopNThreadPool");
|
||||
|
||||
setupQueries();
|
||||
|
||||
schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get("basic");
|
||||
|
@ -336,7 +327,7 @@ public class TopNTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexStringOnly(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexStringOnly(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -353,7 +344,7 @@ public class TopNTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexStringTwice(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexStringTwice(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -381,7 +372,7 @@ public class TopNTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexStringThenLong(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexStringThenLong(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -409,7 +400,7 @@ public class TopNTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexStringThenFloat(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexStringThenFloat(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -437,7 +428,7 @@ public class TopNTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexLongOnly(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexLongOnly(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -454,7 +445,7 @@ public class TopNTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexLongTwice(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexLongTwice(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -482,7 +473,7 @@ public class TopNTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexLongThenString(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexLongThenString(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -510,7 +501,7 @@ public class TopNTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexLongThenFloat(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexLongThenFloat(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -538,7 +529,7 @@ public class TopNTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexFloatOnly(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexFloatOnly(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -555,7 +546,7 @@ public class TopNTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexFloatTwice(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexFloatTwice(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -583,7 +574,7 @@ public class TopNTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexFloatThenString(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexFloatThenString(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -611,7 +602,7 @@ public class TopNTypeInterfaceBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndexFloatThenLong(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndexFloatThenLong(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
|
|
@ -135,7 +135,7 @@ public class IncrementalIndexReadBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void read(Blackhole blackhole) throws Exception
|
||||
public void read(Blackhole blackhole)
|
||||
{
|
||||
IncrementalIndexStorageAdapter sa = new IncrementalIndexStorageAdapter(incIndex);
|
||||
Sequence<Cursor> cursors = makeCursors(sa, null);
|
||||
|
@ -160,7 +160,7 @@ public class IncrementalIndexReadBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void readWithFilters(Blackhole blackhole) throws Exception
|
||||
public void readWithFilters(Blackhole blackhole)
|
||||
{
|
||||
DimFilter filter = new OrDimFilter(
|
||||
Arrays.asList(
|
||||
|
|
|
@ -43,7 +43,6 @@ import org.openjdk.jmh.annotations.State;
|
|||
import org.openjdk.jmh.annotations.Warmup;
|
||||
import org.openjdk.jmh.infra.Blackhole;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
|
@ -70,7 +69,7 @@ public class IndexIngestionBenchmark
|
|||
private BenchmarkSchemaInfo schemaInfo;
|
||||
|
||||
@Setup
|
||||
public void setup() throws IOException
|
||||
public void setup()
|
||||
{
|
||||
ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde(HyperLogLogHash.getDefault()));
|
||||
|
||||
|
@ -94,7 +93,7 @@ public class IndexIngestionBenchmark
|
|||
}
|
||||
|
||||
@Setup(Level.Invocation)
|
||||
public void setup2() throws IOException
|
||||
public void setup2()
|
||||
{
|
||||
incIndex = makeIncIndex();
|
||||
}
|
||||
|
|
|
@ -28,7 +28,6 @@ import io.druid.data.input.InputRow;
|
|||
import io.druid.hll.HyperLogLogHash;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
|
||||
import io.druid.segment.IndexIO;
|
||||
import io.druid.segment.IndexMergerV9;
|
||||
|
@ -37,6 +36,7 @@ import io.druid.segment.column.ColumnConfig;
|
|||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
import io.druid.segment.incremental.IncrementalIndexSchema;
|
||||
import io.druid.segment.serde.ComplexMetrics;
|
||||
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
|
@ -103,7 +103,7 @@ public class IndexPersistBenchmark
|
|||
}
|
||||
|
||||
@Setup
|
||||
public void setup() throws IOException
|
||||
public void setup()
|
||||
{
|
||||
log.info("SETUP CALLED AT " + System.currentTimeMillis());
|
||||
|
||||
|
@ -143,7 +143,7 @@ public class IndexPersistBenchmark
|
|||
}
|
||||
|
||||
@TearDown(Level.Iteration)
|
||||
public void teardown() throws IOException
|
||||
public void teardown()
|
||||
{
|
||||
incIndex.close();
|
||||
incIndex = null;
|
||||
|
|
|
@ -582,7 +582,7 @@ public class GroupByBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleIncrementalIndex(Blackhole blackhole) throws Exception
|
||||
public void querySingleIncrementalIndex(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -600,7 +600,7 @@ public class GroupByBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndex(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndex(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -618,7 +618,7 @@ public class GroupByBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void queryMultiQueryableIndex(Blackhole blackhole) throws Exception
|
||||
public void queryMultiQueryableIndex(Blackhole blackhole)
|
||||
{
|
||||
QueryToolChest<Row, GroupByQuery> toolChest = factory.getToolchest();
|
||||
QueryRunner<Row> theRunner = new FinalizeResultsQueryRunner<>(
|
||||
|
@ -639,7 +639,7 @@ public class GroupByBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void queryMultiQueryableIndexWithSpilling(Blackhole blackhole) throws Exception
|
||||
public void queryMultiQueryableIndexWithSpilling(Blackhole blackhole)
|
||||
{
|
||||
QueryToolChest<Row, GroupByQuery> toolChest = factory.getToolchest();
|
||||
QueryRunner<Row> theRunner = new FinalizeResultsQueryRunner<>(
|
||||
|
@ -663,7 +663,7 @@ public class GroupByBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void queryMultiQueryableIndexWithSerde(Blackhole blackhole) throws Exception
|
||||
public void queryMultiQueryableIndexWithSerde(Blackhole blackhole)
|
||||
{
|
||||
QueryToolChest<Row, GroupByQuery> toolChest = factory.getToolchest();
|
||||
QueryRunner<Row> theRunner = new FinalizeResultsQueryRunner<>(
|
||||
|
|
|
@ -410,7 +410,7 @@ public class SearchBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleIncrementalIndex(Blackhole blackhole) throws Exception
|
||||
public void querySingleIncrementalIndex(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<SearchHit> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -428,7 +428,7 @@ public class SearchBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndex(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndex(Blackhole blackhole)
|
||||
{
|
||||
final QueryRunner<Result<SearchResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -447,7 +447,7 @@ public class SearchBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void queryMultiQueryableIndex(Blackhole blackhole) throws Exception
|
||||
public void queryMultiQueryableIndex(Blackhole blackhole)
|
||||
{
|
||||
List<QueryRunner<Row>> singleSegmentRunners = Lists.newArrayList();
|
||||
QueryToolChest toolChest = factory.getToolchest();
|
||||
|
|
|
@ -290,7 +290,7 @@ public class SelectBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void queryIncrementalIndex(Blackhole blackhole) throws Exception
|
||||
public void queryIncrementalIndex(Blackhole blackhole)
|
||||
{
|
||||
SelectQuery queryCopy = query.withPagingSpec(PagingSpec.newSpec(pagingThreshold));
|
||||
|
||||
|
@ -320,7 +320,7 @@ public class SelectBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void queryQueryableIndex(Blackhole blackhole) throws Exception
|
||||
public void queryQueryableIndex(Blackhole blackhole)
|
||||
{
|
||||
SelectQuery queryCopy = query.withPagingSpec(PagingSpec.newSpec(pagingThreshold));
|
||||
|
||||
|
@ -350,7 +350,7 @@ public class SelectBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void queryMultiQueryableIndex(Blackhole blackhole) throws Exception
|
||||
public void queryMultiQueryableIndex(Blackhole blackhole)
|
||||
{
|
||||
SelectQuery queryCopy = query.withPagingSpec(PagingSpec.newSpec(pagingThreshold));
|
||||
|
||||
|
|
|
@ -90,7 +90,7 @@ public class SqlBenchmark
|
|||
private String sqlQuery;
|
||||
|
||||
@Setup(Level.Trial)
|
||||
public void setup() throws Exception
|
||||
public void setup()
|
||||
{
|
||||
tmpDir = Files.createTempDir();
|
||||
log.info("Starting benchmark setup using tmpDir[%s], rows[%,d].", tmpDir, rowsPerSegment);
|
||||
|
@ -163,7 +163,7 @@ public class SqlBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MILLISECONDS)
|
||||
public void queryNative(Blackhole blackhole) throws Exception
|
||||
public void queryNative(Blackhole blackhole)
|
||||
{
|
||||
final Sequence<Row> resultSequence = QueryPlus.wrap(groupByQuery).run(walker, Maps.newHashMap());
|
||||
final List<Row> resultList = resultSequence.toList();
|
||||
|
|
|
@ -334,7 +334,7 @@ public class TimeseriesBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleIncrementalIndex(Blackhole blackhole) throws Exception
|
||||
public void querySingleIncrementalIndex(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Result<TimeseriesResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -351,7 +351,7 @@ public class TimeseriesBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndex(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndex(Blackhole blackhole)
|
||||
{
|
||||
final QueryRunner<Result<TimeseriesResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -368,7 +368,7 @@ public class TimeseriesBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void queryFilteredSingleQueryableIndex(Blackhole blackhole) throws Exception
|
||||
public void queryFilteredSingleQueryableIndex(Blackhole blackhole)
|
||||
{
|
||||
final QueryRunner<Result<TimeseriesResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -388,7 +388,7 @@ public class TimeseriesBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void queryMultiQueryableIndex(Blackhole blackhole) throws Exception
|
||||
public void queryMultiQueryableIndex(Blackhole blackhole)
|
||||
{
|
||||
List<QueryRunner<Result<TimeseriesResultValue>>> singleSegmentRunners = Lists.newArrayList();
|
||||
QueryToolChest toolChest = factory.getToolchest();
|
||||
|
|
|
@ -315,7 +315,7 @@ public class TopNBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleIncrementalIndex(Blackhole blackhole) throws Exception
|
||||
public void querySingleIncrementalIndex(Blackhole blackhole)
|
||||
{
|
||||
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -332,7 +332,7 @@ public class TopNBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void querySingleQueryableIndex(Blackhole blackhole) throws Exception
|
||||
public void querySingleQueryableIndex(Blackhole blackhole)
|
||||
{
|
||||
final QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
factory,
|
||||
|
@ -349,7 +349,7 @@ public class TopNBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void queryMultiQueryableIndex(Blackhole blackhole) throws Exception
|
||||
public void queryMultiQueryableIndex(Blackhole blackhole)
|
||||
{
|
||||
List<QueryRunner<Result<TopNResultValue>>> singleSegmentRunners = Lists.newArrayList();
|
||||
QueryToolChest toolChest = factory.getToolchest();
|
||||
|
|
|
@ -87,7 +87,7 @@ public class CachingCostBalancerStrategyBenchmark
|
|||
}
|
||||
|
||||
@Benchmark
|
||||
public double measureCostStrategy() throws InterruptedException
|
||||
public double measureCostStrategy()
|
||||
{
|
||||
double cost = 0.0;
|
||||
for (DataSegment segment : segmentQueries) {
|
||||
|
@ -97,7 +97,7 @@ public class CachingCostBalancerStrategyBenchmark
|
|||
}
|
||||
|
||||
@Benchmark
|
||||
public double measureCachingCostStrategy() throws InterruptedException
|
||||
public double measureCachingCostStrategy()
|
||||
{
|
||||
double cost = 0.0;
|
||||
for (DataSegment segment : segmentQueries) {
|
||||
|
|
|
@ -83,7 +83,7 @@ public class CostBalancerStrategyBenchmark
|
|||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
@Fork(1)
|
||||
public double measureCostStrategySingle() throws InterruptedException
|
||||
public double measureCostStrategySingle()
|
||||
{
|
||||
double totalCost = 0;
|
||||
for (DataSegment s : segments) {
|
||||
|
@ -96,7 +96,7 @@ public class CostBalancerStrategyBenchmark
|
|||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
@Fork(1)
|
||||
public double measureIntervalPenalty() throws InterruptedException
|
||||
public double measureIntervalPenalty()
|
||||
{
|
||||
return CostBalancerStrategy.intervalCost(x1, y0, y1);
|
||||
}
|
||||
|
|
|
@ -37,7 +37,7 @@ import java.util.Map;
|
|||
public class BenchmarkDataGeneratorTest
|
||||
{
|
||||
@Test
|
||||
public void testSequential() throws Exception
|
||||
public void testSequential()
|
||||
{
|
||||
List<BenchmarkColumnSchema> schemas = new ArrayList<>();
|
||||
RowValueTracker tracker = new RowValueTracker();
|
||||
|
@ -87,7 +87,7 @@ public class BenchmarkDataGeneratorTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testDiscreteUniform() throws Exception
|
||||
public void testDiscreteUniform()
|
||||
{
|
||||
List<BenchmarkColumnSchema> schemas = new ArrayList<>();
|
||||
RowValueTracker tracker = new RowValueTracker();
|
||||
|
@ -152,7 +152,7 @@ public class BenchmarkDataGeneratorTest
|
|||
|
||||
|
||||
@Test
|
||||
public void testRoundedNormal() throws Exception
|
||||
public void testRoundedNormal()
|
||||
{
|
||||
List<BenchmarkColumnSchema> schemas = new ArrayList<>();
|
||||
RowValueTracker tracker = new RowValueTracker();
|
||||
|
@ -195,7 +195,7 @@ public class BenchmarkDataGeneratorTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testZipf() throws Exception
|
||||
public void testZipf()
|
||||
{
|
||||
List<BenchmarkColumnSchema> schemas = new ArrayList<>();
|
||||
RowValueTracker tracker = new RowValueTracker();
|
||||
|
@ -250,7 +250,7 @@ public class BenchmarkDataGeneratorTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testEnumerated() throws Exception
|
||||
public void testEnumerated()
|
||||
{
|
||||
List<BenchmarkColumnSchema> schemas = new ArrayList<>();
|
||||
RowValueTracker tracker = new RowValueTracker();
|
||||
|
@ -279,7 +279,7 @@ public class BenchmarkDataGeneratorTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testNormal() throws Exception
|
||||
public void testNormal()
|
||||
{
|
||||
List<BenchmarkColumnSchema> schemas = new ArrayList<>();
|
||||
RowValueTracker tracker = new RowValueTracker();
|
||||
|
@ -322,7 +322,7 @@ public class BenchmarkDataGeneratorTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testRealUniform() throws Exception
|
||||
public void testRealUniform()
|
||||
{
|
||||
List<BenchmarkColumnSchema> schemas = new ArrayList<>();
|
||||
RowValueTracker tracker = new RowValueTracker();
|
||||
|
@ -363,7 +363,7 @@ public class BenchmarkDataGeneratorTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testIntervalBasedTimeGeneration() throws Exception
|
||||
public void testIntervalBasedTimeGeneration()
|
||||
{
|
||||
List<BenchmarkColumnSchema> schemas = new ArrayList<>();
|
||||
|
||||
|
|
|
@ -17,35 +17,20 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package io.druid.java.util.common.guava;
|
||||
package io.druid.annotations;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.lang.annotation.ElementType;
|
||||
import java.lang.annotation.Retention;
|
||||
import java.lang.annotation.RetentionPolicy;
|
||||
import java.lang.annotation.Target;
|
||||
|
||||
/**
|
||||
* Annotating test methods, which names have "parametersFor" prefix, and used by {@code JUnitParamsRunner}, see
|
||||
* https://github.com/Pragmatists/junitparams/wiki/Quickstart. IntelliJ's inspection "Unused declarations" knows about
|
||||
* this annotation.
|
||||
*/
|
||||
public class DefaultingHashMap<K, V> extends HashMap<K, V>
|
||||
@Retention(RetentionPolicy.SOURCE)
|
||||
@Target(ElementType.METHOD)
|
||||
public @interface UsedByJUnitParamsRunner
|
||||
{
|
||||
private final Supplier<V> supplier;
|
||||
|
||||
public DefaultingHashMap(
|
||||
Supplier<V> supplier
|
||||
)
|
||||
{
|
||||
this.supplier = supplier;
|
||||
}
|
||||
|
||||
@Override
|
||||
public V get(Object o)
|
||||
{
|
||||
V retVal = super.get(o);
|
||||
|
||||
if (retVal == null) {
|
||||
retVal = supplier.get();
|
||||
super.put((K) o, retVal);
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
}
|
|
@ -23,8 +23,8 @@ import java.lang.annotation.Retention;
|
|||
import java.lang.annotation.RetentionPolicy;
|
||||
|
||||
/**
|
||||
* Annotation for members, which are used in generated code (e. g. by Antlr), but not in regular code. IntelliJ
|
||||
* inspection "unused declarations" knows about this annotation.
|
||||
* Annotation for members, which are used in generated code (e. g. by Antlr), but not in regular code. IntelliJ's
|
||||
* inspection "Unused declarations" knows about this annotation.
|
||||
*/
|
||||
@Retention(RetentionPolicy.SOURCE)
|
||||
public @interface UsedInGeneratedCode
|
||||
|
|
|
@ -27,7 +27,6 @@ import com.google.common.collect.Lists;
|
|||
import io.druid.java.util.common.ISE;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayDeque;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
@ -106,7 +105,7 @@ public class DefaultBlockingPool<T> implements BlockingPool<T>
|
|||
new Closeable()
|
||||
{
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
public void close()
|
||||
{
|
||||
offer(theObject);
|
||||
}
|
||||
|
@ -192,7 +191,7 @@ public class DefaultBlockingPool<T> implements BlockingPool<T>
|
|||
new Closeable()
|
||||
{
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
public void close()
|
||||
{
|
||||
offerBatch(theObjects);
|
||||
}
|
||||
|
|
|
@ -129,7 +129,7 @@ public class ConfigManager
|
|||
{
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public ConfigHolder<T> call() throws Exception
|
||||
public ConfigHolder<T> call()
|
||||
{
|
||||
if (!started) {
|
||||
watchedConfigs.put(key, new ConfigHolder<T>(null, serde));
|
||||
|
@ -272,7 +272,7 @@ public class ConfigManager
|
|||
}
|
||||
|
||||
@Override
|
||||
public ScheduledExecutors.Signal call() throws Exception
|
||||
public ScheduledExecutors.Signal call()
|
||||
{
|
||||
if (stop) {
|
||||
return ScheduledExecutors.Signal.STOP;
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package io.druid.common.utils;
|
||||
|
||||
import com.google.common.io.ByteStreams;
|
||||
import com.google.common.io.OutputSupplier;
|
||||
import com.google.common.primitives.Ints;
|
||||
import com.google.common.primitives.Longs;
|
||||
import io.druid.io.Channels;
|
||||
|
@ -30,31 +29,11 @@ import java.io.IOException;
|
|||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
import java.nio.channels.WritableByteChannel;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
public class SerializerUtils
|
||||
{
|
||||
|
||||
/**
|
||||
* Writes the given int value into the given OutputStream in big-endian byte order, using the helperBuffer. Faster
|
||||
* alternative to out.write(Ints.toByteArray(value)), more convenient (sometimes) than wrapping the OutputStream into
|
||||
* {@link java.io.DataOutputStream}.
|
||||
*
|
||||
* @param helperBuffer a big-endian heap ByteBuffer with capacity of at least 4
|
||||
*/
|
||||
public static void writeBigEndianIntToOutputStream(OutputStream out, int value, ByteBuffer helperBuffer)
|
||||
throws IOException
|
||||
{
|
||||
if (helperBuffer.order() != ByteOrder.BIG_ENDIAN || !helperBuffer.hasArray()) {
|
||||
throw new IllegalArgumentException("Expected writable, big-endian, heap byteBuffer");
|
||||
}
|
||||
helperBuffer.putInt(0, value);
|
||||
out.write(helperBuffer.array(), helperBuffer.arrayOffset(), Integer.BYTES);
|
||||
}
|
||||
|
||||
public <T extends OutputStream> void writeString(T out, String name) throws IOException
|
||||
{
|
||||
byte[] nameBytes = StringUtils.toUtf8(name);
|
||||
|
@ -62,13 +41,6 @@ public class SerializerUtils
|
|||
out.write(nameBytes);
|
||||
}
|
||||
|
||||
public void writeString(OutputSupplier<? extends OutputStream> supplier, String name) throws IOException
|
||||
{
|
||||
try (OutputStream out = supplier.getOutput()) {
|
||||
writeString(out, name);
|
||||
}
|
||||
}
|
||||
|
||||
public void writeString(WritableByteChannel out, String name) throws IOException
|
||||
{
|
||||
byte[] nameBytes = StringUtils.toUtf8(name);
|
||||
|
@ -84,33 +56,19 @@ public class SerializerUtils
|
|||
return StringUtils.fromUtf8(stringBytes);
|
||||
}
|
||||
|
||||
public String readString(ByteBuffer in) throws IOException
|
||||
public String readString(ByteBuffer in)
|
||||
{
|
||||
final int length = in.getInt();
|
||||
return StringUtils.fromUtf8(readBytes(in, length));
|
||||
}
|
||||
|
||||
public byte[] readBytes(ByteBuffer in, int length) throws IOException
|
||||
public byte[] readBytes(ByteBuffer in, int length)
|
||||
{
|
||||
byte[] bytes = new byte[length];
|
||||
in.get(bytes);
|
||||
return bytes;
|
||||
}
|
||||
|
||||
void writeStrings(OutputStream out, String[] names) throws IOException
|
||||
{
|
||||
writeStrings(out, Arrays.asList(names));
|
||||
}
|
||||
|
||||
private void writeStrings(OutputStream out, List<String> names) throws IOException
|
||||
{
|
||||
writeInt(out, names.size());
|
||||
|
||||
for (String name : names) {
|
||||
writeString(out, name);
|
||||
}
|
||||
}
|
||||
|
||||
String[] readStrings(InputStream in) throws IOException
|
||||
{
|
||||
int length = readInt(in);
|
||||
|
@ -124,7 +82,7 @@ public class SerializerUtils
|
|||
return retVal;
|
||||
}
|
||||
|
||||
String[] readStrings(ByteBuffer in) throws IOException
|
||||
String[] readStrings(ByteBuffer in)
|
||||
{
|
||||
int length = in.getInt();
|
||||
|
||||
|
|
|
@ -53,7 +53,7 @@ public class JacksonConfigManagerModule implements Module
|
|||
new Lifecycle.Handler()
|
||||
{
|
||||
@Override
|
||||
public void start() throws Exception
|
||||
public void start()
|
||||
{
|
||||
dbConnector.createConfigTable();
|
||||
}
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package io.druid.io;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
|
@ -64,7 +63,7 @@ public final class ByteBufferInputStream extends InputStream
|
|||
}
|
||||
|
||||
@Override
|
||||
public int available() throws IOException
|
||||
public int available()
|
||||
{
|
||||
return buffer.remaining();
|
||||
}
|
||||
|
|
|
@ -30,16 +30,6 @@ public class ZeroCopyByteArrayOutputStream extends ByteArrayOutputStream
|
|||
{
|
||||
}
|
||||
|
||||
public ZeroCopyByteArrayOutputStream(int capacity)
|
||||
{
|
||||
super(capacity);
|
||||
}
|
||||
|
||||
public void writeTo(ByteBuffer outputBuffer)
|
||||
{
|
||||
outputBuffer.put(buf, 0, count);
|
||||
}
|
||||
|
||||
public void writeTo(WritableByteChannel channel) throws IOException
|
||||
{
|
||||
channel.write(ByteBuffer.wrap(buf, 0, count));
|
||||
|
|
|
@ -34,7 +34,7 @@ public interface MetadataStorageConnector
|
|||
String valueColumn,
|
||||
String key,
|
||||
byte[] value
|
||||
) throws Exception;
|
||||
);
|
||||
|
||||
byte[] lookup(
|
||||
String tableName,
|
||||
|
@ -53,7 +53,7 @@ public interface MetadataStorageConnector
|
|||
*/
|
||||
default boolean compareAndSwap(
|
||||
List<MetadataCASUpdate> updates
|
||||
) throws Exception
|
||||
)
|
||||
{
|
||||
throw new UnsupportedOperationException("compareAndSwap is not implemented.");
|
||||
}
|
||||
|
|
|
@ -112,7 +112,7 @@ public class BlockingPoolTest
|
|||
new Callable<ReferenceCountingResourceHolder<List<Integer>>>()
|
||||
{
|
||||
@Override
|
||||
public ReferenceCountingResourceHolder<List<Integer>> call() throws Exception
|
||||
public ReferenceCountingResourceHolder<List<Integer>> call()
|
||||
{
|
||||
return POOL.takeBatch(8, 100);
|
||||
}
|
||||
|
@ -147,7 +147,7 @@ public class BlockingPoolTest
|
|||
new Callable<List<ReferenceCountingResourceHolder<Integer>>>()
|
||||
{
|
||||
@Override
|
||||
public List<ReferenceCountingResourceHolder<Integer>> call() throws Exception
|
||||
public List<ReferenceCountingResourceHolder<Integer>> call()
|
||||
{
|
||||
List<ReferenceCountingResourceHolder<Integer>> result = Lists.newArrayList();
|
||||
for (int i = 0; i < limit1; i++) {
|
||||
|
@ -161,7 +161,7 @@ public class BlockingPoolTest
|
|||
new Callable<List<ReferenceCountingResourceHolder<Integer>>>()
|
||||
{
|
||||
@Override
|
||||
public List<ReferenceCountingResourceHolder<Integer>> call() throws Exception
|
||||
public List<ReferenceCountingResourceHolder<Integer>> call()
|
||||
{
|
||||
List<ReferenceCountingResourceHolder<Integer>> result = Lists.newArrayList();
|
||||
for (int i = 0; i < limit2; i++) {
|
||||
|
@ -231,7 +231,7 @@ public class BlockingPoolTest
|
|||
new Callable<ReferenceCountingResourceHolder<List<Integer>>>()
|
||||
{
|
||||
@Override
|
||||
public ReferenceCountingResourceHolder<List<Integer>> call() throws Exception
|
||||
public ReferenceCountingResourceHolder<List<Integer>> call()
|
||||
{
|
||||
return POOL.takeBatch(batch1, 10);
|
||||
}
|
||||
|
@ -242,7 +242,7 @@ public class BlockingPoolTest
|
|||
new Callable<ReferenceCountingResourceHolder<List<Integer>>>()
|
||||
{
|
||||
@Override
|
||||
public ReferenceCountingResourceHolder<List<Integer>> call() throws Exception
|
||||
public ReferenceCountingResourceHolder<List<Integer>> call()
|
||||
{
|
||||
return POOL.takeBatch(batch2, 10);
|
||||
}
|
||||
|
@ -277,7 +277,7 @@ public class BlockingPoolTest
|
|||
new Callable<ReferenceCountingResourceHolder<List<Integer>>>()
|
||||
{
|
||||
@Override
|
||||
public ReferenceCountingResourceHolder<List<Integer>> call() throws Exception
|
||||
public ReferenceCountingResourceHolder<List<Integer>> call()
|
||||
{
|
||||
return POOL.takeBatch(batch1, 10);
|
||||
}
|
||||
|
@ -288,7 +288,7 @@ public class BlockingPoolTest
|
|||
new Callable<ReferenceCountingResourceHolder<List<Integer>>>()
|
||||
{
|
||||
@Override
|
||||
public ReferenceCountingResourceHolder<List<Integer>> call() throws Exception
|
||||
public ReferenceCountingResourceHolder<List<Integer>> call()
|
||||
{
|
||||
return POOL.takeBatch(batch2, 10);
|
||||
}
|
||||
|
@ -338,7 +338,7 @@ public class BlockingPoolTest
|
|||
new Callable<ReferenceCountingResourceHolder<List<Integer>>>()
|
||||
{
|
||||
@Override
|
||||
public ReferenceCountingResourceHolder<List<Integer>> call() throws Exception
|
||||
public ReferenceCountingResourceHolder<List<Integer>> call()
|
||||
{
|
||||
return POOL.takeBatch(10, 100);
|
||||
}
|
||||
|
|
|
@ -20,9 +20,7 @@
|
|||
package io.druid.collections;
|
||||
|
||||
import com.google.common.collect.PeekingIterator;
|
||||
|
||||
import io.druid.java.util.common.guava.nary.BinaryFn;
|
||||
|
||||
import org.easymock.EasyMock;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
|
@ -110,7 +108,7 @@ public class CombiningIteratorTest
|
|||
}
|
||||
|
||||
@Test(expected = NoSuchElementException.class)
|
||||
public void testExceptionInNext() throws Exception
|
||||
public void testExceptionInNext()
|
||||
{
|
||||
boolean expected = false;
|
||||
EasyMock.expect(peekIterator.hasNext()).andReturn(expected);
|
||||
|
@ -120,7 +118,7 @@ public class CombiningIteratorTest
|
|||
}
|
||||
|
||||
@Test(expected = UnsupportedOperationException.class)
|
||||
public void testRemove() throws Exception
|
||||
public void testRemove()
|
||||
{
|
||||
testingIterator.remove();
|
||||
}
|
||||
|
|
|
@ -34,7 +34,7 @@ import java.util.NoSuchElementException;
|
|||
public class OrderedMergeIteratorTest
|
||||
{
|
||||
@Test
|
||||
public void testSanity() throws Exception
|
||||
public void testSanity()
|
||||
{
|
||||
final ArrayList<Iterator<Integer>> iterators = Lists.newArrayList();
|
||||
iterators.add(Arrays.asList(1, 3, 5, 7, 9).iterator());
|
||||
|
@ -50,7 +50,7 @@ public class OrderedMergeIteratorTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testScrewsUpOnOutOfOrderBeginningOfList() throws Exception
|
||||
public void testScrewsUpOnOutOfOrderBeginningOfList()
|
||||
{
|
||||
final ArrayList<Iterator<Integer>> iterators = Lists.newArrayList();
|
||||
iterators.add(Arrays.asList(1, 3, 5, 7, 9).iterator());
|
||||
|
@ -66,7 +66,7 @@ public class OrderedMergeIteratorTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testScrewsUpOnOutOfOrderInList() throws Exception
|
||||
public void testScrewsUpOnOutOfOrderInList()
|
||||
{
|
||||
final ArrayList<Iterator<Integer>> iterators = Lists.newArrayList();
|
||||
iterators.add(Arrays.asList(1, 3, 5, 4, 7, 9).iterator());
|
||||
|
@ -82,7 +82,7 @@ public class OrderedMergeIteratorTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testLaziness() throws Exception
|
||||
public void testLaziness()
|
||||
{
|
||||
final boolean[] done = new boolean[]{false, false};
|
||||
|
||||
|
|
|
@ -151,7 +151,7 @@ public class OrderedMergeSequenceTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testLazinessAccumulation() throws Exception
|
||||
public void testLazinessAccumulation()
|
||||
{
|
||||
final ArrayList<Sequence<Integer>> sequences = makeSyncedSequences();
|
||||
OrderedMergeSequence<Integer> seq = new OrderedMergeSequence<Integer>(
|
||||
|
|
|
@ -23,7 +23,6 @@ import org.junit.Assert;
|
|||
import org.junit.Test;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
@ -77,7 +76,7 @@ public class ReferenceCountingResourceHolderTest
|
|||
.fromCloseable((Closeable) new Closeable()
|
||||
{
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
public void close()
|
||||
{
|
||||
released.set(true);
|
||||
}
|
||||
|
|
|
@ -28,8 +28,6 @@ import org.junit.Assert;
|
|||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class StupidPoolTest
|
||||
{
|
||||
private Supplier<String> generator;
|
||||
|
@ -48,7 +46,7 @@ public class StupidPoolTest
|
|||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws IOException
|
||||
public void tearDown()
|
||||
{
|
||||
if (resourceHolderObj != null) {
|
||||
resourceHolderObj.close();
|
||||
|
@ -64,7 +62,7 @@ public class StupidPoolTest
|
|||
}
|
||||
|
||||
@Test(expected = ISE.class)
|
||||
public void testExceptionInResourceHolderGet() throws IOException
|
||||
public void testExceptionInResourceHolderGet()
|
||||
{
|
||||
resourceHolderObj.close();
|
||||
resourceHolderObj.get();
|
||||
|
|
|
@ -22,14 +22,12 @@ package io.druid.collections;
|
|||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class StupidResourceHolderTest
|
||||
{
|
||||
private StupidResourceHolder<String> resourceHolder;
|
||||
|
||||
@Test
|
||||
public void testCreateAndGet() throws IOException
|
||||
public void testCreateAndGet()
|
||||
{
|
||||
String expected = "String";
|
||||
resourceHolder = StupidResourceHolder.create(expected);
|
||||
|
|
|
@ -36,7 +36,6 @@ import org.junit.runners.Parameterized;
|
|||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
|
@ -218,18 +217,11 @@ public class CombiningSequenceTest
|
|||
{
|
||||
// Test that closing works too
|
||||
final CountDownLatch closed = new CountDownLatch(1);
|
||||
final Closeable closeable = new Closeable()
|
||||
{
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
closed.countDown();
|
||||
}
|
||||
};
|
||||
final Closeable closeable = closed::countDown;
|
||||
|
||||
Sequence<Pair<Integer, Integer>> seq = CombiningSequence.create(
|
||||
Sequences.simple(pairs).withBaggage(closeable),
|
||||
Ordering.natural().onResultOf(Pair.lhsFn()),
|
||||
Ordering.natural().onResultOf(p -> p.lhs),
|
||||
(lhs, rhs) -> {
|
||||
if (lhs == null) {
|
||||
return rhs;
|
||||
|
|
|
@ -36,7 +36,7 @@ import java.util.List;
|
|||
public class JodaUtilsTest
|
||||
{
|
||||
@Test
|
||||
public void testUmbrellaIntervalsSimple() throws Exception
|
||||
public void testUmbrellaIntervalsSimple()
|
||||
{
|
||||
List<Interval> intervals = Arrays.asList(
|
||||
Intervals.of("2011-03-03/2011-03-04"),
|
||||
|
@ -56,13 +56,13 @@ public class JodaUtilsTest
|
|||
}
|
||||
|
||||
@Test(expected = IllegalArgumentException.class)
|
||||
public void testUmbrellaIntervalsNull() throws Exception
|
||||
public void testUmbrellaIntervalsNull()
|
||||
{
|
||||
JodaUtils.umbrellaInterval(Collections.emptyList());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCondenseIntervalsSimple() throws Exception
|
||||
public void testCondenseIntervalsSimple()
|
||||
{
|
||||
List<Interval> intervals = Arrays.asList(
|
||||
Intervals.of("2011-01-01/2011-01-02"),
|
||||
|
@ -88,7 +88,7 @@ public class JodaUtilsTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testCondenseIntervalsMixedUp() throws Exception
|
||||
public void testCondenseIntervalsMixedUp()
|
||||
{
|
||||
List<Interval> intervals = Arrays.asList(
|
||||
Intervals.of("2011-01-01/2011-01-02"),
|
||||
|
|
|
@ -131,14 +131,6 @@ public class SerializerUtilsTest
|
|||
Assert.assertArrayEquals(longsByte, actuals);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWriteStrings() throws IOException
|
||||
{
|
||||
serializerUtils.writeStrings(outStream, strings);
|
||||
byte[] actuals = outStream.toByteArray();
|
||||
Assert.assertArrayEquals(stringsByte, actuals);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testChannelWritelong() throws IOException
|
||||
{
|
||||
|
@ -204,7 +196,7 @@ public class SerializerUtilsTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testByteBufferReadStrings() throws IOException
|
||||
public void testByteBufferReadStrings()
|
||||
{
|
||||
ByteBuffer buffer = ByteBuffer.allocate(stringsByte.length);
|
||||
buffer.put(stringsByte);
|
||||
|
|
|
@ -26,7 +26,7 @@ public class ServletResourceUtilsTest
|
|||
{
|
||||
|
||||
@Test
|
||||
public void testSanitizeException() throws Exception
|
||||
public void testSanitizeException()
|
||||
{
|
||||
final String message = "some message";
|
||||
Assert.assertEquals(message, ServletResourceUtils.sanitizeException(new Throwable(message)).get("error"));
|
||||
|
|
|
@ -23,8 +23,6 @@ import io.druid.java.util.common.StringUtils;
|
|||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.UnsupportedEncodingException;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class StringUtilsTest
|
||||
|
@ -50,7 +48,7 @@ public class StringUtilsTest
|
|||
};
|
||||
|
||||
@Test
|
||||
public void binaryLengthAsUTF8Test() throws UnsupportedEncodingException
|
||||
public void binaryLengthAsUTF8Test()
|
||||
{
|
||||
for (String string : TEST_STRINGS) {
|
||||
Assert.assertEquals(StringUtils.toUtf8(string).length, StringUtils.estimatedBinaryLengthAsUTF8(string));
|
||||
|
@ -58,7 +56,7 @@ public class StringUtilsTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void binaryLengthAsUTF8InvalidTest() throws UnsupportedEncodingException
|
||||
public void binaryLengthAsUTF8InvalidTest()
|
||||
{
|
||||
// we can fix this but looks trivial case, imho
|
||||
String invalid = "\uD841"; // high only
|
||||
|
|
|
@ -54,7 +54,7 @@ public class VersionedIntervalTimelineTest
|
|||
VersionedIntervalTimeline<String, Integer> timeline;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception
|
||||
public void setUp()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -78,7 +78,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testApril() throws Exception
|
||||
public void testApril()
|
||||
{
|
||||
assertValues(
|
||||
Arrays.asList(
|
||||
|
@ -91,7 +91,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testApril2() throws Exception
|
||||
public void testApril2()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
makeSingle(1),
|
||||
|
@ -109,7 +109,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testApril3() throws Exception
|
||||
public void testApril3()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
makeSingle(1),
|
||||
|
@ -130,7 +130,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testApril4() throws Exception
|
||||
public void testApril4()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
makeSingle(1),
|
||||
|
@ -155,7 +155,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testMay() throws Exception
|
||||
public void testMay()
|
||||
{
|
||||
assertValues(
|
||||
Collections.singletonList(
|
||||
|
@ -166,7 +166,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testMay2() throws Exception
|
||||
public void testMay2()
|
||||
{
|
||||
Assert.assertNotNull(timeline.remove(Intervals.of("2011-05-01/2011-05-10"), "4", makeSingle(1)));
|
||||
assertValues(
|
||||
|
@ -180,7 +180,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testMay3() throws Exception
|
||||
public void testMay3()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
makeSingle(9),
|
||||
|
@ -200,7 +200,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testInsertInWrongOrder() throws Exception
|
||||
public void testInsertInWrongOrder()
|
||||
{
|
||||
DateTime overallStart = DateTimes.nowUtc().minus(Hours.TWO);
|
||||
|
||||
|
@ -225,7 +225,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testRemove() throws Exception
|
||||
public void testRemove()
|
||||
{
|
||||
for (TimelineObjectHolder<String, Integer> holder : timeline.findOvershadowed()) {
|
||||
for (PartitionChunk<Integer> chunk : holder.getObject()) {
|
||||
|
@ -237,7 +237,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testFindEntry() throws Exception
|
||||
public void testFindEntry()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
new ImmutablePartitionHolder<Integer>(new PartitionHolder<Integer>(makeSingle(1))),
|
||||
|
@ -271,7 +271,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testFindEntryWithOverlap() throws Exception
|
||||
public void testFindEntryWithOverlap()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -285,7 +285,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testPartitioning() throws Exception
|
||||
public void testPartitioning()
|
||||
{
|
||||
assertValues(
|
||||
ImmutableList.of(
|
||||
|
@ -306,7 +306,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testPartialPartitionNotReturned() throws Exception
|
||||
public void testPartialPartitionNotReturned()
|
||||
{
|
||||
testRemove();
|
||||
|
||||
|
@ -343,7 +343,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testIncompletePartitionDoesNotOvershadow() throws Exception
|
||||
public void testIncompletePartitionDoesNotOvershadow()
|
||||
{
|
||||
testRemove();
|
||||
|
||||
|
@ -363,7 +363,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testRemovePartitionMakesIncomplete() throws Exception
|
||||
public void testRemovePartitionMakesIncomplete()
|
||||
{
|
||||
testIncompletePartitionDoesNotOvershadow();
|
||||
|
||||
|
@ -377,7 +377,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testInsertAndRemoveSameThingsion() throws Exception
|
||||
public void testInsertAndRemoveSameThingsion()
|
||||
{
|
||||
add("2011-05-01/2011-05-10", "5", 10);
|
||||
assertValues(
|
||||
|
@ -421,7 +421,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 1|----|
|
||||
// 1|----|
|
||||
@Test(expected = UnsupportedOperationException.class)
|
||||
public void testOverlapSameVersionThrowException() throws Exception
|
||||
public void testOverlapSameVersionThrowException()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -432,7 +432,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 1|----|
|
||||
// 1|----|
|
||||
@Test
|
||||
public void testOverlapSameVersionIsOkay() throws Exception
|
||||
public void testOverlapSameVersionIsOkay()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -452,7 +452,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 1|----|----|
|
||||
// 2|----|
|
||||
@Test
|
||||
public void testOverlapSecondBetween() throws Exception
|
||||
public void testOverlapSecondBetween()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -473,7 +473,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 2|----|
|
||||
// 1|----|----|
|
||||
@Test
|
||||
public void testOverlapFirstBetween() throws Exception
|
||||
public void testOverlapFirstBetween()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -494,7 +494,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 1|----|
|
||||
// 2|----|
|
||||
@Test
|
||||
public void testOverlapFirstBefore() throws Exception
|
||||
public void testOverlapFirstBefore()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -513,7 +513,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 2|----|
|
||||
// 1|----|
|
||||
@Test
|
||||
public void testOverlapFirstAfter() throws Exception
|
||||
public void testOverlapFirstAfter()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -532,7 +532,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 1|----|
|
||||
// 2|----|
|
||||
@Test
|
||||
public void testOverlapSecondBefore() throws Exception
|
||||
public void testOverlapSecondBefore()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -551,7 +551,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 2|----|
|
||||
// 1|----|
|
||||
@Test
|
||||
public void testOverlapSecondAfter() throws Exception
|
||||
public void testOverlapSecondAfter()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -570,7 +570,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 1|----------|
|
||||
// 2|----|
|
||||
@Test
|
||||
public void testOverlapFirstLarger() throws Exception
|
||||
public void testOverlapFirstLarger()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -590,7 +590,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 2|----|
|
||||
// 1|----------|
|
||||
@Test
|
||||
public void testOverlapSecondLarger() throws Exception
|
||||
public void testOverlapSecondLarger()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -610,7 +610,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 1|----|-----|
|
||||
// 2|-------|
|
||||
@Test
|
||||
public void testOverlapSecondPartialAlign() throws Exception
|
||||
public void testOverlapSecondPartialAlign()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -630,7 +630,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 2|-------|
|
||||
// 1|----|-----|
|
||||
@Test
|
||||
public void testOverlapFirstPartialAlign() throws Exception
|
||||
public void testOverlapFirstPartialAlign()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -651,7 +651,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 2|------------|
|
||||
// 3|---|
|
||||
@Test
|
||||
public void testOverlapAscending() throws Exception
|
||||
public void testOverlapAscending()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -673,7 +673,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 2|------------|
|
||||
// 1|-------|
|
||||
@Test
|
||||
public void testOverlapDescending() throws Exception
|
||||
public void testOverlapDescending()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -695,7 +695,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 3|---|
|
||||
// 1|-------|
|
||||
@Test
|
||||
public void testOverlapMixed() throws Exception
|
||||
public void testOverlapMixed()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -717,7 +717,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 2|--------|
|
||||
// 3|-----|
|
||||
@Test
|
||||
public void testOverlapContainedAscending() throws Exception
|
||||
public void testOverlapContainedAscending()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -740,7 +740,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 2|--------|
|
||||
// 1|-------------|
|
||||
@Test
|
||||
public void testOverlapContainedDescending() throws Exception
|
||||
public void testOverlapContainedDescending()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -763,7 +763,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 3|-----|
|
||||
// 1|-------------|
|
||||
@Test
|
||||
public void testOverlapContainedmixed() throws Exception
|
||||
public void testOverlapContainedmixed()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -785,7 +785,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 1|------|------|----|
|
||||
// 2|-----|
|
||||
@Test
|
||||
public void testOverlapSecondContained() throws Exception
|
||||
public void testOverlapSecondContained()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -809,7 +809,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 2|-----|
|
||||
// 1|------|------|----|
|
||||
@Test
|
||||
public void testOverlapFirstContained() throws Exception
|
||||
public void testOverlapFirstContained()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -833,7 +833,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 1|----|----|
|
||||
// 2|---------|
|
||||
@Test
|
||||
public void testOverlapSecondContainsFirst() throws Exception
|
||||
public void testOverlapSecondContainsFirst()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -853,7 +853,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 2|---------|
|
||||
// 1|----|----|
|
||||
@Test
|
||||
public void testOverlapFirstContainsSecond() throws Exception
|
||||
public void testOverlapFirstContainsSecond()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -874,7 +874,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 2|----|
|
||||
// 3|----|
|
||||
@Test
|
||||
public void testOverlapLayeredAscending() throws Exception
|
||||
public void testOverlapLayeredAscending()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -896,7 +896,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 2|----|
|
||||
// 1|----|
|
||||
@Test
|
||||
public void testOverlapLayeredDescending() throws Exception
|
||||
public void testOverlapLayeredDescending()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -917,7 +917,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 2|----| |----|
|
||||
// 1|-------------|
|
||||
@Test
|
||||
public void testOverlapV1Large() throws Exception
|
||||
public void testOverlapV1Large()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -939,7 +939,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 2|-------------|
|
||||
// 1|----| |----|
|
||||
@Test
|
||||
public void testOverlapV2Large() throws Exception
|
||||
public void testOverlapV2Large()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -959,7 +959,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 1|-------------|
|
||||
// 2|----| |----|
|
||||
@Test
|
||||
public void testOverlapV1LargeIsAfter() throws Exception
|
||||
public void testOverlapV1LargeIsAfter()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -981,7 +981,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 2|----| |----|
|
||||
// 1|-------------|
|
||||
@Test
|
||||
public void testOverlapV1SecondLargeIsAfter() throws Exception
|
||||
public void testOverlapV1SecondLargeIsAfter()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1003,7 +1003,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 1|-----------|
|
||||
// 2|----| |----|
|
||||
@Test
|
||||
public void testOverlapV1FirstBetween() throws Exception
|
||||
public void testOverlapV1FirstBetween()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1024,7 +1024,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 2|----| |----|
|
||||
// 1|-----------|
|
||||
@Test
|
||||
public void testOverlapV1SecondBetween() throws Exception
|
||||
public void testOverlapV1SecondBetween()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1047,7 +1047,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 2|---|
|
||||
// 1|-------------|
|
||||
@Test
|
||||
public void testOverlapLargeUnderlyingWithSmallDayAlignedOverlays() throws Exception
|
||||
public void testOverlapLargeUnderlyingWithSmallDayAlignedOverlays()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1070,7 +1070,7 @@ public class VersionedIntervalTimelineTest
|
|||
// |----3---||---1---|
|
||||
// |---2---|
|
||||
@Test
|
||||
public void testOverlapCausesNullEntries() throws Exception
|
||||
public void testOverlapCausesNullEntries()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1092,7 +1092,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 2|------| |------|
|
||||
// 3|------------------|
|
||||
@Test
|
||||
public void testOverlapOvershadowedThirdContains() throws Exception
|
||||
public void testOverlapOvershadowedThirdContains()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1117,7 +1117,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 1|-------------|
|
||||
// 3|-------------|
|
||||
@Test
|
||||
public void testOverlapOvershadowedAligned() throws Exception
|
||||
public void testOverlapOvershadowedAligned()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1140,7 +1140,7 @@ public class VersionedIntervalTimelineTest
|
|||
// 1|---------|
|
||||
// 3|-----------|
|
||||
@Test
|
||||
public void testOverlapOvershadowedSomeComplexOverlapsCantThinkOfBetterName() throws Exception
|
||||
public void testOverlapOvershadowedSomeComplexOverlapsCantThinkOfBetterName()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1159,7 +1159,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testOverlapAndRemove() throws Exception
|
||||
public void testOverlapAndRemove()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1177,7 +1177,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testOverlapAndRemove2() throws Exception
|
||||
public void testOverlapAndRemove2()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1198,7 +1198,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testOverlapAndRemove3() throws Exception
|
||||
public void testOverlapAndRemove3()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1219,7 +1219,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testOverlapAndRemove4() throws Exception
|
||||
public void testOverlapAndRemove4()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1240,7 +1240,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testOverlapAndRemove5() throws Exception
|
||||
public void testOverlapAndRemove5()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1258,7 +1258,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testRemoveSomethingDontHave() throws Exception
|
||||
public void testRemoveSomethingDontHave()
|
||||
{
|
||||
Assert.assertNull(
|
||||
"Don't have it, should be null",
|
||||
|
@ -1271,7 +1271,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testRemoveNothingBacking() throws Exception
|
||||
public void testRemoveNothingBacking()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1293,7 +1293,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testOvershadowingHigherVersionWins1() throws Exception
|
||||
public void testOvershadowingHigherVersionWins1()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1313,7 +1313,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testOvershadowingHigherVersionWins2() throws Exception
|
||||
public void testOvershadowingHigherVersionWins2()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1331,7 +1331,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testOvershadowingHigherVersionWins3() throws Exception
|
||||
public void testOvershadowingHigherVersionWins3()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1351,7 +1351,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testOvershadowingHigherVersionWins4() throws Exception
|
||||
public void testOvershadowingHigherVersionWins4()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1369,7 +1369,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testOvershadowingHigherVersionNeverOvershadowedByLower1() throws Exception
|
||||
public void testOvershadowingHigherVersionNeverOvershadowedByLower1()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1384,7 +1384,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testOvershadowingHigherVersionNeverOvershadowedByLower2() throws Exception
|
||||
public void testOvershadowingHigherVersionNeverOvershadowedByLower2()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1399,7 +1399,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testOvershadowingHigherVersionNeverOvershadowedByLower3() throws Exception
|
||||
public void testOvershadowingHigherVersionNeverOvershadowedByLower3()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1414,7 +1414,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testOvershadowingHigherVersionNeverOvershadowedByLower4() throws Exception
|
||||
public void testOvershadowingHigherVersionNeverOvershadowedByLower4()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1432,7 +1432,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testOvershadowingHigherVersionNeverOvershadowedByLower5() throws Exception
|
||||
public void testOvershadowingHigherVersionNeverOvershadowedByLower5()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1452,7 +1452,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testOvershadowingSameIntervalHighVersionWins() throws Exception
|
||||
public void testOvershadowingSameIntervalHighVersionWins()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1470,7 +1470,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testOvershadowingSameIntervalSameVersionAllKept() throws Exception
|
||||
public void testOvershadowingSameIntervalSameVersionAllKept()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1489,7 +1489,7 @@ public class VersionedIntervalTimelineTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testNotFoundReturnsEmpty() throws Exception
|
||||
public void testNotFoundReturnsEmpty()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
@ -1500,7 +1500,7 @@ public class VersionedIntervalTimelineTest
|
|||
|
||||
// https://github.com/druid-io/druid/issues/3010
|
||||
@Test
|
||||
public void testRemoveIncompleteKeepsComplete() throws Exception
|
||||
public void testRemoveIncompleteKeepsComplete()
|
||||
{
|
||||
timeline = makeStringIntegerTimeline();
|
||||
|
||||
|
|
|
@ -29,7 +29,7 @@ import static io.druid.timeline.partition.IntegerPartitionChunk.make;
|
|||
public class IntegerPartitionChunkTest
|
||||
{
|
||||
@Test
|
||||
public void testAbuts() throws Exception
|
||||
public void testAbuts()
|
||||
{
|
||||
IntegerPartitionChunk<Integer> lhs = make(null, 10, 0, 1);
|
||||
|
||||
|
@ -41,7 +41,7 @@ public class IntegerPartitionChunkTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testIsStart() throws Exception
|
||||
public void testIsStart()
|
||||
{
|
||||
Assert.assertTrue(make(null, 10, 0, 1).isStart());
|
||||
Assert.assertFalse(make(10, null, 0, 1).isStart());
|
||||
|
@ -50,7 +50,7 @@ public class IntegerPartitionChunkTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testIsEnd() throws Exception
|
||||
public void testIsEnd()
|
||||
{
|
||||
Assert.assertFalse(make(null, 10, 0, 1).isEnd());
|
||||
Assert.assertTrue(make(10, null, 0, 1).isEnd());
|
||||
|
@ -59,7 +59,7 @@ public class IntegerPartitionChunkTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testCompareTo() throws Exception
|
||||
public void testCompareTo()
|
||||
{
|
||||
Assert.assertEquals(0, make(null, null, 0, 1).compareTo(make(null, null, 0, 1)));
|
||||
Assert.assertEquals(0, make(10, null, 0, 1).compareTo(make(10, null, 0, 2)));
|
||||
|
@ -72,7 +72,7 @@ public class IntegerPartitionChunkTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testEquals() throws Exception
|
||||
public void testEquals()
|
||||
{
|
||||
Assert.assertEquals(make(null, null, 0, 1), make(null, null, 0, 1));
|
||||
Assert.assertEquals(make(null, 10, 0, 1), make(null, 10, 0, 1));
|
||||
|
|
|
@ -29,7 +29,7 @@ import static io.druid.timeline.partition.StringPartitionChunk.make;
|
|||
public class StringPartitionChunkTest
|
||||
{
|
||||
@Test
|
||||
public void testAbuts() throws Exception
|
||||
public void testAbuts()
|
||||
{
|
||||
StringPartitionChunk<Integer> lhs = make(null, "10", 0, 1);
|
||||
|
||||
|
@ -41,7 +41,7 @@ public class StringPartitionChunkTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testIsStart() throws Exception
|
||||
public void testIsStart()
|
||||
{
|
||||
Assert.assertTrue(make(null, "10", 0, 1).isStart());
|
||||
Assert.assertFalse(make("10", null, 0, 1).isStart());
|
||||
|
@ -50,7 +50,7 @@ public class StringPartitionChunkTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testIsEnd() throws Exception
|
||||
public void testIsEnd()
|
||||
{
|
||||
Assert.assertFalse(make(null, "10", 0, 1).isEnd());
|
||||
Assert.assertTrue(make("10", null, 0, 1).isEnd());
|
||||
|
@ -59,7 +59,7 @@ public class StringPartitionChunkTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testCompareTo() throws Exception
|
||||
public void testCompareTo()
|
||||
{
|
||||
Assert.assertEquals(0, make(null, null, 0, 1).compareTo(make(null, null, 0, 2)));
|
||||
Assert.assertEquals(0, make("10", null, 0, 1).compareTo(make("10", null, 0, 2)));
|
||||
|
@ -72,7 +72,7 @@ public class StringPartitionChunkTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testEquals() throws Exception
|
||||
public void testEquals()
|
||||
{
|
||||
Assert.assertEquals(make(null, null, 0, 1), make(null, null, 0, 1));
|
||||
Assert.assertEquals(make(null, "10", 0, 1), make(null, "10", 0, 1));
|
||||
|
|
|
@ -30,8 +30,8 @@ import io.druid.data.input.FirehoseFactory;
|
|||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.MapBasedInputRow;
|
||||
import io.druid.data.input.impl.InputRowParser;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import twitter4j.ConnectionLifeCycleListener;
|
||||
import twitter4j.GeoLocation;
|
||||
import twitter4j.HashtagEntity;
|
||||
|
@ -45,7 +45,6 @@ import twitter4j.User;
|
|||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
@ -115,7 +114,7 @@ public class TwitterSpritzerFirehoseFactory implements FirehoseFactory<InputRowP
|
|||
}
|
||||
|
||||
@Override
|
||||
public Firehose connect(InputRowParser parser, File temporaryDirectory) throws IOException
|
||||
public Firehose connect(InputRowParser parser, File temporaryDirectory)
|
||||
{
|
||||
final ConnectionLifeCycleListener connectionLifeCycleListener = new ConnectionLifeCycleListener()
|
||||
{
|
||||
|
@ -368,7 +367,7 @@ public class TwitterSpritzerFirehoseFactory implements FirehoseFactory<InputRowP
|
|||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
public void close()
|
||||
{
|
||||
log.info("CLOSE twitterstream");
|
||||
twitterStream.shutdown(); // invokes twitterStream.cleanUp()
|
||||
|
|
|
@ -19,9 +19,6 @@
|
|||
package io.druid.extendedset.intset;
|
||||
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.NoSuchElementException;
|
||||
|
||||
/**
|
||||
* This class provides a skeletal implementation of the {@link IntSet}
|
||||
* interface to minimize the effort required to implement this interface.
|
||||
|
@ -31,58 +28,6 @@ import java.util.NoSuchElementException;
|
|||
*/
|
||||
public abstract class AbstractIntSet implements IntSet
|
||||
{
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public IntSet union(IntSet other)
|
||||
{
|
||||
IntSet res = clone();
|
||||
res.addAll(other);
|
||||
return res;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public IntSet difference(IntSet other)
|
||||
{
|
||||
IntSet res = clone();
|
||||
res.removeAll(other);
|
||||
return res;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public IntSet intersection(IntSet other)
|
||||
{
|
||||
IntSet res = clone();
|
||||
res.retainAll(other);
|
||||
return res;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void complement()
|
||||
{
|
||||
if (isEmpty()) {
|
||||
return;
|
||||
}
|
||||
for (int e = last(); e >= 0; e--) {
|
||||
flip(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public abstract IntSet empty();
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
|
@ -90,18 +35,6 @@ public abstract class AbstractIntSet implements IntSet
|
|||
@Override
|
||||
public abstract IntSet clone();
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public abstract double bitmapCompressionRatio();
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public abstract double collectionCompressionRatio();
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
|
@ -114,106 +47,6 @@ public abstract class AbstractIntSet implements IntSet
|
|||
@Override
|
||||
public abstract IntIterator descendingIterator();
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public abstract String debugInfo();
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void clear()
|
||||
{
|
||||
IntIterator itr = iterator();
|
||||
while (itr.hasNext()) {
|
||||
itr.next();
|
||||
itr.remove();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void clear(int from, int to)
|
||||
{
|
||||
if (from > to) {
|
||||
throw new IndexOutOfBoundsException("from: " + from + " > to: " + to);
|
||||
}
|
||||
for (int e = from; e <= to; e++) {
|
||||
remove(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void fill(int from, int to)
|
||||
{
|
||||
if (from > to) {
|
||||
throw new IndexOutOfBoundsException("from: " + from + " > to: " + to);
|
||||
}
|
||||
for (int e = from; e <= to; e++) {
|
||||
add(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void flip(int e)
|
||||
{
|
||||
if (!add(e)) {
|
||||
remove(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public abstract int get(int i);
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public abstract int indexOf(int e);
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public abstract IntSet convert(int... a);
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public abstract IntSet convert(Collection<Integer> c);
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public int first()
|
||||
{
|
||||
if (isEmpty()) {
|
||||
throw new NoSuchElementException();
|
||||
}
|
||||
return iterator().next();
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public abstract int last();
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
|
@ -226,111 +59,6 @@ public abstract class AbstractIntSet implements IntSet
|
|||
@Override
|
||||
public abstract boolean isEmpty();
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public abstract boolean contains(int i);
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public abstract boolean add(int i);
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public abstract boolean remove(int i);
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public boolean addAll(IntSet c)
|
||||
{
|
||||
if (c == null || c.isEmpty()) {
|
||||
return false;
|
||||
}
|
||||
IntIterator itr = c.iterator();
|
||||
boolean res = false;
|
||||
while (itr.hasNext()) {
|
||||
res |= add(itr.next());
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public boolean removeAll(IntSet c)
|
||||
{
|
||||
if (c == null || c.isEmpty()) {
|
||||
return false;
|
||||
}
|
||||
IntIterator itr = c.iterator();
|
||||
boolean res = false;
|
||||
while (itr.hasNext()) {
|
||||
res |= remove(itr.next());
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public boolean retainAll(IntSet c)
|
||||
{
|
||||
if (c == null || c.isEmpty()) {
|
||||
return false;
|
||||
}
|
||||
IntIterator itr = iterator();
|
||||
boolean res = false;
|
||||
while (itr.hasNext()) {
|
||||
int e = itr.next();
|
||||
if (!c.contains(e)) {
|
||||
res = true;
|
||||
itr.remove();
|
||||
}
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public int[] toArray()
|
||||
{
|
||||
if (isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
return toArray(new int[size()]);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public int[] toArray(int[] a)
|
||||
{
|
||||
if (a.length < size()) {
|
||||
a = new int[size()];
|
||||
}
|
||||
IntIterator itr = iterator();
|
||||
int i = 0;
|
||||
while (itr.hasNext()) {
|
||||
a[i++] = itr.next();
|
||||
}
|
||||
for (; i < a.length; i++) {
|
||||
a[i] = 0;
|
||||
}
|
||||
return a;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
|
|
|
@ -95,12 +95,6 @@ public final class BitIterator implements IntSet.IntIterator
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void skipAllBefore(int element)
|
||||
{
|
||||
|
|
|
@ -20,23 +20,14 @@
|
|||
package io.druid.extendedset.intset;
|
||||
|
||||
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.ObjectInputStream;
|
||||
import java.io.ObjectOutputStream;
|
||||
import java.io.Serializable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.ConcurrentModificationException;
|
||||
import java.util.Formatter;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.NoSuchElementException;
|
||||
import java.util.SortedSet;
|
||||
|
||||
/**
|
||||
* This is CONCISE: COmpressed 'N' Composable Integer SEt.
|
||||
|
@ -285,25 +276,6 @@ public class ConciseSet extends AbstractIntSet implements Serializable
|
|||
return (word & 0xC1FFFFFF);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the position of the flipped bit within a sequence word. If the
|
||||
* sequence has no set/unset bit, returns -1.
|
||||
* <p/>
|
||||
* Note that the parameter <i>must</i> a sequence word, otherwise the
|
||||
* result is meaningless.
|
||||
*
|
||||
* @param word sequence word to check
|
||||
*
|
||||
* @return the position of the set bit, from 0 to 31. If the sequence has no
|
||||
* set/unset bit, returns -1.
|
||||
*/
|
||||
private static int getFlippedBit(int word)
|
||||
{
|
||||
// get bits from 30 to 26
|
||||
// NOTE: "-1" is required since 00000 represents no bits and 00001 the LSB bit set
|
||||
return ((word >>> 25) & 0x0000001F) - 1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the number of set bits within the literal word
|
||||
*
|
||||
|
@ -342,23 +314,6 @@ public class ConciseSet extends AbstractIntSet implements Serializable
|
|||
return (literal & (literal - 1)) == 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates the 32-bit binary representation of a given word (debug only)
|
||||
*
|
||||
* @param word word to represent
|
||||
*
|
||||
* @return 32-character string that represents the given word
|
||||
*/
|
||||
private static String toBinaryString(int word)
|
||||
{
|
||||
String lsb = Integer.toBinaryString(word);
|
||||
StringBuilder pad = new StringBuilder();
|
||||
for (int i = lsb.length(); i < 32; i++) {
|
||||
pad.append('0');
|
||||
}
|
||||
return pad.append(lsb).toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Resets to an empty set
|
||||
*
|
||||
|
@ -424,17 +379,6 @@ public class ConciseSet extends AbstractIntSet implements Serializable
|
|||
: (ConciseSetUtils.ALL_ONES_LITERAL & ~literal);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clears bits from MSB (excluded, since it indicates the word type) to the
|
||||
* specified bit (excluded). Last word is supposed to be a literal one.
|
||||
*
|
||||
* @param lastSetBit leftmost bit to preserve
|
||||
*/
|
||||
private void clearBitsAfterInLastWord(int lastSetBit)
|
||||
{
|
||||
words[lastWordIndex] &= ConciseSetUtils.ALL_ZEROS_LITERAL | (0xFFFFFFFF >>> (31 - lastSetBit));
|
||||
}
|
||||
|
||||
/**
|
||||
* Assures that the length of {@link #words} is sufficient to contain
|
||||
* the given index.
|
||||
|
@ -791,16 +735,6 @@ public class ConciseSet extends AbstractIntSet implements Serializable
|
|||
return res;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
public ByteBuffer toByteBuffer()
|
||||
{
|
||||
ByteBuffer buffer = ByteBuffer.allocate((lastWordIndex + 1) * 4);
|
||||
buffer.asIntBuffer().put(Arrays.copyOf(words, lastWordIndex + 1));
|
||||
return buffer;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
|
@ -815,153 +749,6 @@ public class ConciseSet extends AbstractIntSet implements Serializable
|
|||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public int get(int i)
|
||||
{
|
||||
if (i < 0) {
|
||||
throw new IndexOutOfBoundsException();
|
||||
}
|
||||
|
||||
// initialize data
|
||||
int firstSetBitInWord = 0;
|
||||
int position = i;
|
||||
int setBitsInCurrentWord = 0;
|
||||
for (int j = 0; j <= lastWordIndex; j++) {
|
||||
int w = words[j];
|
||||
if (isLiteral(w)) {
|
||||
// number of bits in the current word
|
||||
setBitsInCurrentWord = getLiteralBitCount(w);
|
||||
|
||||
// check if the desired bit is in the current word
|
||||
if (position < setBitsInCurrentWord) {
|
||||
int currSetBitInWord = -1;
|
||||
for (; position >= 0; position--) {
|
||||
currSetBitInWord = Integer.numberOfTrailingZeros(w & (0xFFFFFFFF << (currSetBitInWord + 1)));
|
||||
}
|
||||
return firstSetBitInWord + currSetBitInWord;
|
||||
}
|
||||
|
||||
// skip the 31-bit block
|
||||
firstSetBitInWord += ConciseSetUtils.MAX_LITERAL_LENGTH;
|
||||
} else {
|
||||
// number of involved bits (31 * blocks)
|
||||
int sequenceLength = maxLiteralLengthMultiplication(getSequenceCount(w) + 1);
|
||||
|
||||
// check the sequence type
|
||||
if (isOneSequence(w)) {
|
||||
if (simulateWAH || isSequenceWithNoBits(w)) {
|
||||
setBitsInCurrentWord = sequenceLength;
|
||||
if (position < setBitsInCurrentWord) {
|
||||
return firstSetBitInWord + position;
|
||||
}
|
||||
} else {
|
||||
setBitsInCurrentWord = sequenceLength - 1;
|
||||
if (position < setBitsInCurrentWord) {
|
||||
// check whether the desired set bit is after the
|
||||
// flipped bit (or after the first block)
|
||||
return firstSetBitInWord + position + (position < getFlippedBit(w) ? 0 : 1);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
if (simulateWAH || isSequenceWithNoBits(w)) {
|
||||
setBitsInCurrentWord = 0;
|
||||
} else {
|
||||
setBitsInCurrentWord = 1;
|
||||
if (position == 0) {
|
||||
return firstSetBitInWord + getFlippedBit(w);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// skip the 31-bit blocks
|
||||
firstSetBitInWord += sequenceLength;
|
||||
}
|
||||
|
||||
// update the number of found set bits
|
||||
position -= setBitsInCurrentWord;
|
||||
}
|
||||
|
||||
throw new IndexOutOfBoundsException(Integer.toString(i));
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public int indexOf(int e)
|
||||
{
|
||||
if (e < 0) {
|
||||
throw new IllegalArgumentException("positive integer expected: " + Integer.toString(e));
|
||||
}
|
||||
if (isEmpty()) {
|
||||
return -1;
|
||||
}
|
||||
|
||||
// returned value
|
||||
int index = 0;
|
||||
|
||||
int blockIndex = maxLiteralLengthDivision(e);
|
||||
int bitPosition = maxLiteralLengthModulus(e);
|
||||
for (int i = 0; i <= lastWordIndex && blockIndex >= 0; i++) {
|
||||
int w = words[i];
|
||||
if (isLiteral(w)) {
|
||||
// check if the current literal word is the "right" one
|
||||
if (blockIndex == 0) {
|
||||
if ((w & (1 << bitPosition)) == 0) {
|
||||
return -1;
|
||||
}
|
||||
return index + Integer.bitCount(w & ~(0xFFFFFFFF << bitPosition));
|
||||
}
|
||||
blockIndex--;
|
||||
index += getLiteralBitCount(w);
|
||||
} else {
|
||||
if (simulateWAH) {
|
||||
if (isOneSequence(w) && blockIndex <= getSequenceCount(w)) {
|
||||
return index + maxLiteralLengthMultiplication(blockIndex) + bitPosition;
|
||||
}
|
||||
} else {
|
||||
// if we are at the beginning of a sequence, and it is
|
||||
// a set bit, the bit already exists
|
||||
if (blockIndex == 0) {
|
||||
int l = getLiteral(w);
|
||||
if ((l & (1 << bitPosition)) == 0) {
|
||||
return -1;
|
||||
}
|
||||
return index + Integer.bitCount(l & ~(0xFFFFFFFF << bitPosition));
|
||||
}
|
||||
|
||||
// if we are in the middle of a sequence of 1's, the bit already exist
|
||||
if (blockIndex > 0
|
||||
&& blockIndex <= getSequenceCount(w)
|
||||
&& isOneSequence(w)) {
|
||||
return index + maxLiteralLengthMultiplication(blockIndex) + bitPosition - (isSequenceWithNoBits(w) ? 0 : 1);
|
||||
}
|
||||
}
|
||||
|
||||
// next word
|
||||
int blocks = getSequenceCount(w) + 1;
|
||||
blockIndex -= blocks;
|
||||
if (isZeroSequence(w)) {
|
||||
if (!simulateWAH && !isSequenceWithNoBits(w)) {
|
||||
index++;
|
||||
}
|
||||
} else {
|
||||
index += maxLiteralLengthMultiplication(blocks);
|
||||
if (!simulateWAH && !isSequenceWithNoBits(w)) {
|
||||
index--;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// not found
|
||||
return -1;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public ConciseSet intersection(IntSet other)
|
||||
{
|
||||
if (isEmpty() || other == null || other.isEmpty()) {
|
||||
|
@ -973,97 +760,6 @@ public class ConciseSet extends AbstractIntSet implements Serializable
|
|||
return performOperation(convert(other), Operator.AND);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public ConciseSet union(IntSet other)
|
||||
{
|
||||
if (other == null || other.isEmpty() || other == this) {
|
||||
return clone();
|
||||
}
|
||||
return performOperation(convert(other), Operator.OR);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public ConciseSet difference(IntSet other)
|
||||
{
|
||||
if (other == this) {
|
||||
return empty();
|
||||
}
|
||||
if (other == null || other.isEmpty()) {
|
||||
return clone();
|
||||
}
|
||||
return performOperation(convert(other), Operator.ANDNOT);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void complement()
|
||||
{
|
||||
|
||||
if (isEmpty()) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (last == ConciseSetUtils.MIN_ALLOWED_SET_BIT) {
|
||||
clear();
|
||||
return;
|
||||
}
|
||||
|
||||
// update size
|
||||
if (size >= 0) {
|
||||
size = last - size + 1;
|
||||
}
|
||||
|
||||
// complement each word
|
||||
for (int i = 0; i <= lastWordIndex; i++) {
|
||||
int w = words[i];
|
||||
if (isLiteral(w)) {
|
||||
// negate the bits and set the most significant bit to 1
|
||||
words[i] = ConciseSetUtils.ALL_ZEROS_LITERAL | ~w;
|
||||
} else {
|
||||
// switch the sequence type
|
||||
words[i] ^= ConciseSetUtils.SEQUENCE_BIT;
|
||||
}
|
||||
}
|
||||
|
||||
// do not complement after the last element
|
||||
if (isLiteral(words[lastWordIndex])) {
|
||||
clearBitsAfterInLastWord(maxLiteralLengthModulus(last));
|
||||
}
|
||||
|
||||
// remove trailing zeros
|
||||
trimZeros();
|
||||
if (isEmpty()) {
|
||||
return;
|
||||
}
|
||||
|
||||
// calculate the maximal element
|
||||
last = 0;
|
||||
int w = 0;
|
||||
for (int i = 0; i <= lastWordIndex; i++) {
|
||||
w = words[i];
|
||||
if (isLiteral(w)) {
|
||||
last += ConciseSetUtils.MAX_LITERAL_LENGTH;
|
||||
} else {
|
||||
last += maxLiteralLengthMultiplication(getSequenceCount(w) + 1);
|
||||
}
|
||||
}
|
||||
|
||||
// manage the last word (that must be a literal or a sequence of 1's)
|
||||
if (isLiteral(w)) {
|
||||
last -= Integer.numberOfLeadingZeros(getLiteralBits(w));
|
||||
} else {
|
||||
last--;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes trailing zeros
|
||||
*/
|
||||
|
@ -1121,24 +817,11 @@ public class ConciseSet extends AbstractIntSet implements Serializable
|
|||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void clear()
|
||||
{
|
||||
reset();
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public int last()
|
||||
{
|
||||
if (isEmpty()) {
|
||||
throw new NoSuchElementException();
|
||||
}
|
||||
return last;
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert a given collection to a {@link ConciseSet} instance
|
||||
*/
|
||||
|
@ -1162,7 +845,6 @@ public class ConciseSet extends AbstractIntSet implements Serializable
|
|||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public ConciseSet convert(int... a)
|
||||
{
|
||||
ConciseSet res = empty();
|
||||
|
@ -1178,30 +860,6 @@ public class ConciseSet extends AbstractIntSet implements Serializable
|
|||
return res;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public ConciseSet convert(Collection<Integer> c)
|
||||
{
|
||||
ConciseSet res = empty();
|
||||
Collection<Integer> sorted;
|
||||
if (c != null) {
|
||||
if (c instanceof SortedSet<?> && ((SortedSet<?>) c).comparator() == null) {
|
||||
sorted = c;
|
||||
} else {
|
||||
sorted = new ArrayList<Integer>(c);
|
||||
Collections.sort((List<Integer>) sorted);
|
||||
}
|
||||
for (int i : sorted) {
|
||||
if (res.last != i) {
|
||||
res.add(i);
|
||||
}
|
||||
}
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
/**
|
||||
* Replace the current instance with another {@link ConciseSet} instance. It
|
||||
* also returns <code>true</code> if the given set is actually different
|
||||
|
@ -1241,7 +899,6 @@ public class ConciseSet extends AbstractIntSet implements Serializable
|
|||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public boolean add(int e)
|
||||
{
|
||||
|
||||
|
@ -1337,7 +994,6 @@ public class ConciseSet extends AbstractIntSet implements Serializable
|
|||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public boolean remove(int o)
|
||||
{
|
||||
|
||||
|
@ -1434,7 +1090,6 @@ public class ConciseSet extends AbstractIntSet implements Serializable
|
|||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public boolean contains(int o)
|
||||
{
|
||||
if (isEmpty() || o > last || o < 0) {
|
||||
|
@ -1497,37 +1152,6 @@ public class ConciseSet extends AbstractIntSet implements Serializable
|
|||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public boolean retainAll(IntSet c)
|
||||
{
|
||||
|
||||
if (isEmpty() || c == this) {
|
||||
return false;
|
||||
}
|
||||
if (c == null || c.isEmpty()) {
|
||||
clear();
|
||||
return true;
|
||||
}
|
||||
|
||||
ConciseSet other = convert(c);
|
||||
if (other.size == 1) {
|
||||
if (contains(other.last)) {
|
||||
if (size == 1) {
|
||||
return false;
|
||||
}
|
||||
return replaceWith(convert(other.last));
|
||||
}
|
||||
clear();
|
||||
return true;
|
||||
}
|
||||
|
||||
return replaceWith(performOperation(other, Operator.AND));
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public boolean addAll(IntSet c)
|
||||
{
|
||||
if (c == null || c.isEmpty() || this == c) {
|
||||
|
@ -1542,29 +1166,6 @@ public class ConciseSet extends AbstractIntSet implements Serializable
|
|||
return replaceWith(performOperation(convert(c), Operator.OR));
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public boolean removeAll(IntSet c)
|
||||
{
|
||||
|
||||
if (c == null || c.isEmpty() || isEmpty()) {
|
||||
return false;
|
||||
}
|
||||
if (c == this) {
|
||||
clear();
|
||||
return true;
|
||||
}
|
||||
|
||||
ConciseSet other = convert(c);
|
||||
if (other.size == 1) {
|
||||
return remove(other.last);
|
||||
}
|
||||
|
||||
return replaceWith(performOperation(convert(c), Operator.ANDNOT));
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
|
@ -1597,7 +1198,6 @@ public class ConciseSet extends AbstractIntSet implements Serializable
|
|||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public ConciseSet empty()
|
||||
{
|
||||
return new ConciseSet(simulateWAH);
|
||||
|
@ -1778,166 +1378,6 @@ public class ConciseSet extends AbstractIntSet implements Serializable
|
|||
return thisIndex >= 0 ? 1 : (otherIndex >= 0 ? -1 : 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void clear(int from, int to)
|
||||
{
|
||||
ConciseSet toRemove = empty();
|
||||
toRemove.fill(from, to);
|
||||
this.removeAll(toRemove);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void fill(int from, int to)
|
||||
{
|
||||
ConciseSet toAdd = empty();
|
||||
toAdd.add(to);
|
||||
toAdd.complement();
|
||||
toAdd.add(to);
|
||||
|
||||
ConciseSet toRemove = empty();
|
||||
toRemove.add(from);
|
||||
toRemove.complement();
|
||||
|
||||
toAdd.removeAll(toRemove);
|
||||
|
||||
this.addAll(toAdd);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void flip(int e)
|
||||
{
|
||||
if (!add(e)) {
|
||||
remove(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public double bitmapCompressionRatio()
|
||||
{
|
||||
if (isEmpty()) {
|
||||
return 0D;
|
||||
}
|
||||
return (lastWordIndex + 1) / Math.ceil((1 + last) / 32D);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public double collectionCompressionRatio()
|
||||
{
|
||||
if (isEmpty()) {
|
||||
return 0D;
|
||||
}
|
||||
return (double) (lastWordIndex + 1) / size();
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public String debugInfo()
|
||||
{
|
||||
final StringBuilder s = new StringBuilder("INTERNAL REPRESENTATION:\n");
|
||||
final Formatter f = new Formatter(s, Locale.ENGLISH);
|
||||
|
||||
if (isEmpty()) {
|
||||
return s.append("null\n").toString();
|
||||
}
|
||||
|
||||
f.format("Elements: %s\n", toString());
|
||||
|
||||
// elements
|
||||
int firstBitInWord = 0;
|
||||
for (int i = 0; i <= lastWordIndex; i++) {
|
||||
// raw representation of words[i]
|
||||
f.format("words[%d] = ", i);
|
||||
String ws = toBinaryString(words[i]);
|
||||
if (isLiteral(words[i])) {
|
||||
s.append(ws.substring(0, 1));
|
||||
s.append("--");
|
||||
s.append(ws.substring(1));
|
||||
} else {
|
||||
s.append(ws.substring(0, 2));
|
||||
s.append('-');
|
||||
if (simulateWAH) {
|
||||
s.append("xxxxx");
|
||||
} else {
|
||||
s.append(ws.substring(2, 7));
|
||||
}
|
||||
s.append('-');
|
||||
s.append(ws.substring(7));
|
||||
}
|
||||
s.append(" --> ");
|
||||
|
||||
// decode words[i]
|
||||
if (isLiteral(words[i])) {
|
||||
// literal
|
||||
s.append("literal: ");
|
||||
s.append(toBinaryString(words[i]).substring(1));
|
||||
f.format(" ---> [from %d to %d] ", firstBitInWord, firstBitInWord + ConciseSetUtils.MAX_LITERAL_LENGTH - 1);
|
||||
firstBitInWord += ConciseSetUtils.MAX_LITERAL_LENGTH;
|
||||
} else {
|
||||
// sequence
|
||||
if (isOneSequence(words[i])) {
|
||||
s.append('1');
|
||||
} else {
|
||||
s.append('0');
|
||||
}
|
||||
s.append(" block: ");
|
||||
s.append(toBinaryString(getLiteralBits(getLiteral(words[i]))).substring(1));
|
||||
if (!simulateWAH) {
|
||||
s.append(" (bit=");
|
||||
int bit = (words[i] & 0x3E000000) >>> 25;
|
||||
if (bit == 0) {
|
||||
s.append("none");
|
||||
} else {
|
||||
s.append(StringUtils.format("%4d", bit - 1));
|
||||
}
|
||||
s.append(')');
|
||||
}
|
||||
int count = getSequenceCount(words[i]);
|
||||
f.format(
|
||||
" followed by %d blocks (%d bits)",
|
||||
getSequenceCount(words[i]),
|
||||
maxLiteralLengthMultiplication(count)
|
||||
);
|
||||
f.format(
|
||||
" ---> [from %d to %d] ",
|
||||
firstBitInWord,
|
||||
firstBitInWord + (count + 1) * ConciseSetUtils.MAX_LITERAL_LENGTH - 1
|
||||
);
|
||||
firstBitInWord += (count + 1) * ConciseSetUtils.MAX_LITERAL_LENGTH;
|
||||
}
|
||||
s.append('\n');
|
||||
}
|
||||
|
||||
// object attributes
|
||||
f.format("simulateWAH: %b\n", simulateWAH);
|
||||
f.format("last: %d\n", last);
|
||||
f.format("size: %s\n", (size == -1 ? "invalid" : Integer.toString(size)));
|
||||
f.format("words.length: %d\n", words.length);
|
||||
f.format("lastWordIndex: %d\n", lastWordIndex);
|
||||
|
||||
// compression
|
||||
f.format("bitmap compression: %.2f%%\n", 100D * bitmapCompressionRatio());
|
||||
f.format("collection compression: %.2f%%\n", 100D * collectionCompressionRatio());
|
||||
|
||||
return s.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Save the state of the instance to a stream
|
||||
*/
|
||||
|
@ -2627,12 +2067,6 @@ public class ConciseSet extends AbstractIntSet implements Serializable
|
|||
return exp.next();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void skipAllBefore(int element)
|
||||
{
|
||||
|
@ -2723,12 +2157,6 @@ public class ConciseSet extends AbstractIntSet implements Serializable
|
|||
return exp.previous();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void skipAllBefore(int element)
|
||||
{
|
||||
|
|
|
@ -191,20 +191,6 @@ public class ConciseSetUtils
|
|||
return getSequenceCount(word) + 1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Clears the (un)set bit in a sequence
|
||||
*
|
||||
* @param word word to check
|
||||
*
|
||||
* @return the sequence corresponding to the given sequence and with no
|
||||
* (un)set bits
|
||||
*/
|
||||
public static int getSequenceWithNoBits(int word)
|
||||
{
|
||||
// clear 29 to 25 LSB bits
|
||||
return (word & 0xC1FFFFFF);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the literal word that represents the first 31 bits of the given the
|
||||
* word (i.e. the first block of a sequence word, or the bits of a literal word).
|
||||
|
|
|
@ -44,12 +44,6 @@ public final class EmptyIntIterator implements IntSet.IntIterator
|
|||
throw new NoSuchElementException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void skipAllBefore(int element)
|
||||
{
|
||||
|
|
|
@ -815,11 +815,6 @@ public class ImmutableConciseSet
|
|||
return buf.array();
|
||||
}
|
||||
|
||||
public int getLastWordIndex()
|
||||
{
|
||||
return lastWordIndex;
|
||||
}
|
||||
|
||||
// Based on the ConciseSet implementation by Alessandro Colantonio
|
||||
private int calcSize()
|
||||
{
|
||||
|
@ -886,80 +881,6 @@ public class ImmutableConciseSet
|
|||
return intIterator.hasNext() && intIterator.next() == integer;
|
||||
}
|
||||
|
||||
// Based on the ConciseSet implementation by Alessandro Colantonio
|
||||
public int get(int i)
|
||||
{
|
||||
if (i < 0) {
|
||||
throw new IndexOutOfBoundsException();
|
||||
}
|
||||
|
||||
// initialize data
|
||||
int firstSetBitInWord = 0;
|
||||
int position = i;
|
||||
int setBitsInCurrentWord = 0;
|
||||
for (int j = 0; j <= lastWordIndex; j++) {
|
||||
int w = words.get(j);
|
||||
if (ConciseSetUtils.isLiteral(w)) {
|
||||
// number of bits in the current word
|
||||
setBitsInCurrentWord = ConciseSetUtils.getLiteralBitCount(w);
|
||||
|
||||
// check if the desired bit is in the current word
|
||||
if (position < setBitsInCurrentWord) {
|
||||
int currSetBitInWord = -1;
|
||||
for (; position >= 0; position--) {
|
||||
currSetBitInWord = Integer.numberOfTrailingZeros(w & (0xFFFFFFFF << (currSetBitInWord + 1)));
|
||||
}
|
||||
return firstSetBitInWord + currSetBitInWord;
|
||||
}
|
||||
|
||||
// skip the 31-bit block
|
||||
firstSetBitInWord += ConciseSetUtils.MAX_LITERAL_LENGTH;
|
||||
} else {
|
||||
// number of involved bits (31 * blocks)
|
||||
int sequenceLength = ConciseSetUtils.maxLiteralLengthMultiplication(ConciseSetUtils.getSequenceCount(w) + 1);
|
||||
|
||||
// check the sequence type
|
||||
if (ConciseSetUtils.isOneSequence(w)) {
|
||||
if (ConciseSetUtils.isSequenceWithNoBits(w)) {
|
||||
setBitsInCurrentWord = sequenceLength;
|
||||
if (position < setBitsInCurrentWord) {
|
||||
return firstSetBitInWord + position;
|
||||
}
|
||||
} else {
|
||||
setBitsInCurrentWord = sequenceLength - 1;
|
||||
if (position < setBitsInCurrentWord) {
|
||||
// check whether the desired set bit is after the
|
||||
// flipped bit (or after the first block)
|
||||
return firstSetBitInWord + position + (position < ConciseSetUtils.getFlippedBit(w) ? 0 : 1);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
if (ConciseSetUtils.isSequenceWithNoBits(w)) {
|
||||
setBitsInCurrentWord = 0;
|
||||
} else {
|
||||
setBitsInCurrentWord = 1;
|
||||
if (position == 0) {
|
||||
return firstSetBitInWord + ConciseSetUtils.getFlippedBit(w);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// skip the 31-bit blocks
|
||||
firstSetBitInWord += sequenceLength;
|
||||
}
|
||||
|
||||
// update the number of found set bits
|
||||
position -= setBitsInCurrentWord;
|
||||
}
|
||||
|
||||
throw new IndexOutOfBoundsException(Integer.toString(i));
|
||||
}
|
||||
|
||||
public int compareTo(ImmutableConciseSet other)
|
||||
{
|
||||
return words.asReadOnlyBuffer().compareTo(other.words.asReadOnlyBuffer());
|
||||
}
|
||||
|
||||
private boolean isEmpty()
|
||||
{
|
||||
return words == null || words.limit() == 0;
|
||||
|
|
|
@ -19,9 +19,6 @@
|
|||
package io.druid.extendedset.intset;
|
||||
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.BitSet;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.NoSuchElementException;
|
||||
|
||||
|
@ -31,55 +28,6 @@ import java.util.NoSuchElementException;
|
|||
*/
|
||||
public interface IntSet extends Cloneable, Comparable<IntSet>
|
||||
{
|
||||
/**
|
||||
* Generates the intersection set
|
||||
*
|
||||
* @param other {@link IntSet} instance that represents the right
|
||||
* operand
|
||||
*
|
||||
* @return the result of the operation
|
||||
*
|
||||
* @see #retainAll(IntSet)
|
||||
*/
|
||||
IntSet intersection(IntSet other);
|
||||
|
||||
/**
|
||||
* Generates the union set
|
||||
*
|
||||
* @param other {@link IntSet} instance that represents the right
|
||||
* operand
|
||||
*
|
||||
* @return the result of the operation
|
||||
*
|
||||
* @see #addAll(IntSet)
|
||||
*/
|
||||
IntSet union(IntSet other);
|
||||
|
||||
/**
|
||||
* Generates the difference set
|
||||
*
|
||||
* @param other {@link IntSet} instance that represents the right
|
||||
* operand
|
||||
*
|
||||
* @return the result of the operation
|
||||
*
|
||||
* @see #removeAll(IntSet)
|
||||
*/
|
||||
IntSet difference(IntSet other);
|
||||
|
||||
/**
|
||||
* Complements the current set. The modified set is represented by all the
|
||||
* elements strictly less than {@link #last()} that do not exist in the
|
||||
* current set.
|
||||
*/
|
||||
void complement();
|
||||
|
||||
/**
|
||||
* Generates an empty set
|
||||
*
|
||||
* @return the empty set
|
||||
*/
|
||||
IntSet empty();
|
||||
|
||||
/**
|
||||
* See the <code>clone()</code> of {@link Object}
|
||||
|
@ -88,24 +36,6 @@ public interface IntSet extends Cloneable, Comparable<IntSet>
|
|||
*/
|
||||
IntSet clone();
|
||||
|
||||
/**
|
||||
* Computes the compression factor of the equivalent bitmap representation
|
||||
* (1 means not compressed, namely a memory footprint similar to
|
||||
* {@link BitSet}, 2 means twice the size of {@link BitSet}, etc.)
|
||||
*
|
||||
* @return the compression factor
|
||||
*/
|
||||
double bitmapCompressionRatio();
|
||||
|
||||
/**
|
||||
* Computes the compression factor of the equivalent integer collection (1
|
||||
* means not compressed, namely a memory footprint similar to
|
||||
* {@link ArrayList}, 2 means twice the size of {@link ArrayList}, etc.)
|
||||
*
|
||||
* @return the compression factor
|
||||
*/
|
||||
double collectionCompressionRatio();
|
||||
|
||||
/**
|
||||
* @return a {@link IntIterator} instance to iterate over the set
|
||||
*/
|
||||
|
@ -117,98 +47,6 @@ public interface IntSet extends Cloneable, Comparable<IntSet>
|
|||
*/
|
||||
IntIterator descendingIterator();
|
||||
|
||||
/**
|
||||
* Prints debug info about the given {@link IntSet} implementation
|
||||
*
|
||||
* @return a string that describes the internal representation of the
|
||||
* instance
|
||||
*/
|
||||
String debugInfo();
|
||||
|
||||
/**
|
||||
* Adds to the set all the elements between <code>first</code> and
|
||||
* <code>last</code>, both included.
|
||||
*
|
||||
* @param from first element
|
||||
* @param to last element
|
||||
*/
|
||||
void fill(int from, int to);
|
||||
|
||||
/**
|
||||
* Removes from the set all the elements between <code>first</code> and
|
||||
* <code>last</code>, both included.
|
||||
*
|
||||
* @param from first element
|
||||
* @param to last element
|
||||
*/
|
||||
void clear(int from, int to);
|
||||
|
||||
/**
|
||||
* Adds the element if it not existing, or removes it if existing
|
||||
*
|
||||
* @param e element to flip
|
||||
*/
|
||||
void flip(int e);
|
||||
|
||||
/**
|
||||
* Gets the <code>i</code><sup>th</sup> element of the set
|
||||
*
|
||||
* @param i position of the element in the sorted set
|
||||
*
|
||||
* @return the <code>i</code><sup>th</sup> element of the set
|
||||
*
|
||||
* @throws IndexOutOfBoundsException if <code>i</code> is less than zero, or greater or equal to
|
||||
* {@link #size()}
|
||||
*/
|
||||
int get(int i);
|
||||
|
||||
/**
|
||||
* Provides position of element within the set.
|
||||
* <p>
|
||||
* It returns -1 if the element does not exist within the set.
|
||||
*
|
||||
* @param e element of the set
|
||||
*
|
||||
* @return the element position
|
||||
*/
|
||||
int indexOf(int e);
|
||||
|
||||
/**
|
||||
* Converts a given array into an instance of the current class.
|
||||
*
|
||||
* @param a array to use to generate the new instance
|
||||
*
|
||||
* @return the converted collection
|
||||
*/
|
||||
IntSet convert(int... a);
|
||||
|
||||
/**
|
||||
* Converts a given collection into an instance of the current class.
|
||||
*
|
||||
* @param c array to use to generate the new instance
|
||||
*
|
||||
* @return the converted collection
|
||||
*/
|
||||
IntSet convert(Collection<Integer> c);
|
||||
|
||||
/**
|
||||
* Returns the first (lowest) element currently in this set.
|
||||
*
|
||||
* @return the first (lowest) element currently in this set
|
||||
*
|
||||
* @throws NoSuchElementException if this set is empty
|
||||
*/
|
||||
int first();
|
||||
|
||||
/**
|
||||
* Returns the last (highest) element currently in this set.
|
||||
*
|
||||
* @return the last (highest) element currently in this set
|
||||
*
|
||||
* @throws NoSuchElementException if this set is empty
|
||||
*/
|
||||
int last();
|
||||
|
||||
/**
|
||||
* @return the number of elements in this set (its cardinality)
|
||||
*/
|
||||
|
@ -219,120 +57,6 @@ public interface IntSet extends Cloneable, Comparable<IntSet>
|
|||
*/
|
||||
boolean isEmpty();
|
||||
|
||||
/**
|
||||
* Returns <tt>true</tt> if this set contains the specified element.
|
||||
*
|
||||
* @param i element whose presence in this set is to be tested
|
||||
*
|
||||
* @return <tt>true</tt> if this set contains the specified element
|
||||
*/
|
||||
boolean contains(int i);
|
||||
|
||||
/**
|
||||
* Adds the specified element to this set if it is not already present. It
|
||||
* ensures that sets never contain duplicate elements.
|
||||
*
|
||||
* @param i element to be added to this set
|
||||
*
|
||||
* @return <tt>true</tt> if this set did not already contain the specified
|
||||
* element
|
||||
*
|
||||
* @throws IllegalArgumentException if some property of the specified element prevents it from
|
||||
* being added to this set
|
||||
*/
|
||||
boolean add(int i);
|
||||
|
||||
/**
|
||||
* Removes the specified element from this set if it is present.
|
||||
*
|
||||
* @param i object to be removed from this set, if present
|
||||
*
|
||||
* @return <tt>true</tt> if this set contained the specified element
|
||||
*
|
||||
* @throws UnsupportedOperationException if the <tt>remove</tt> operation is not supported by this set
|
||||
*/
|
||||
boolean remove(int i);
|
||||
|
||||
/**
|
||||
* Adds all of the elements in the specified collection to this set if
|
||||
* they're not already present.
|
||||
*
|
||||
* @param c collection containing elements to be added to this set
|
||||
*
|
||||
* @return <tt>true</tt> if this set changed as a result of the call
|
||||
*
|
||||
* @throws NullPointerException if the specified collection contains one or more null
|
||||
* elements and this set does not permit null elements, or if
|
||||
* the specified collection is null
|
||||
* @throws IllegalArgumentException if some property of an element of the specified collection
|
||||
* prevents it from being added to this set
|
||||
* @see #add(int)
|
||||
*/
|
||||
boolean addAll(IntSet c);
|
||||
|
||||
/**
|
||||
* Retains only the elements in this set that are contained in the specified
|
||||
* collection. In other words, removes from this set all of its elements
|
||||
* that are not contained in the specified collection.
|
||||
*
|
||||
* @param c collection containing elements to be retained in this set
|
||||
*
|
||||
* @return <tt>true</tt> if this set changed as a result of the call
|
||||
*
|
||||
* @throws NullPointerException if this set contains a null element and the specified
|
||||
* collection does not permit null elements (optional), or if
|
||||
* the specified collection is null
|
||||
* @see #remove(int)
|
||||
*/
|
||||
boolean retainAll(IntSet c);
|
||||
|
||||
/**
|
||||
* Removes from this set all of its elements that are contained in the
|
||||
* specified collection.
|
||||
*
|
||||
* @param c collection containing elements to be removed from this set
|
||||
*
|
||||
* @return <tt>true</tt> if this set changed as a result of the call
|
||||
*
|
||||
* @throws NullPointerException if this set contains a null element and the specified
|
||||
* collection does not permit null elements (optional), or if
|
||||
* the specified collection is null
|
||||
* @see #remove(int)
|
||||
* @see #contains(int)
|
||||
*/
|
||||
boolean removeAll(IntSet c);
|
||||
|
||||
/**
|
||||
* Removes all of the elements from this set. The set will be empty after
|
||||
* this call returns.
|
||||
*
|
||||
* @throws UnsupportedOperationException if the <tt>clear</tt> method is not supported by this set
|
||||
*/
|
||||
void clear();
|
||||
|
||||
/**
|
||||
* @return an array containing all the elements in this set, in the same
|
||||
* order.
|
||||
*/
|
||||
int[] toArray();
|
||||
|
||||
/**
|
||||
* Returns an array containing all of the elements in this set.
|
||||
* <p>
|
||||
* If this set fits in the specified array with room to spare (i.e., the
|
||||
* array has more elements than this set), the element in the array
|
||||
* immediately following the end of the set are left unchanged.
|
||||
*
|
||||
* @param a the array into which the elements of this set are to be
|
||||
* stored.
|
||||
*
|
||||
* @return the array containing all the elements in this set
|
||||
*
|
||||
* @throws NullPointerException if the specified array is null
|
||||
* @throws IllegalArgumentException if this set does not fit in the specified array
|
||||
*/
|
||||
int[] toArray(int[] a);
|
||||
|
||||
/**
|
||||
* An {@link Iterator}-like interface that allows to "skip" some elements of
|
||||
* the set
|
||||
|
@ -353,22 +77,6 @@ public interface IntSet extends Cloneable, Comparable<IntSet>
|
|||
@Override
|
||||
int next();
|
||||
|
||||
/**
|
||||
* Removes from the underlying collection the last element returned by
|
||||
* the iterator (optional operation). This method can be called only
|
||||
* once per call to <tt>next</tt>. The behavior of an iterator is
|
||||
* unspecified if the underlying collection is modified while the
|
||||
* iteration is in progress in any way other than by calling this
|
||||
* method.
|
||||
*
|
||||
* @throws UnsupportedOperationException if the <tt>remove</tt> operation is not supported by
|
||||
* this Iterator.
|
||||
* @throws IllegalStateException if the <tt>next</tt> method has not yet been called,
|
||||
* or the <tt>remove</tt> method has already been called
|
||||
* after the last call to the <tt>next</tt> method.
|
||||
*/
|
||||
void remove();
|
||||
|
||||
/**
|
||||
* Skips all the elements before the the specified element, so that
|
||||
* {@link #next()} gives the given element or, if it does not exist, the
|
||||
|
|
|
@ -1,6 +1,5 @@
|
|||
package io.druid.extendedset.utilities;
|
||||
|
||||
import java.nio.IntBuffer;
|
||||
import java.util.ArrayList;
|
||||
|
||||
/**
|
||||
|
@ -68,27 +67,6 @@ public class IntList
|
|||
return baseList[index % ALLOCATION_SIZE];
|
||||
}
|
||||
|
||||
public int baseListCount()
|
||||
{
|
||||
return baseLists.size();
|
||||
}
|
||||
|
||||
public IntBuffer getBaseList(int index)
|
||||
{
|
||||
final int[] array = baseLists.get(index);
|
||||
if (array == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final IntBuffer retVal = IntBuffer.wrap(array);
|
||||
|
||||
if (index + 1 == baseListCount()) {
|
||||
retVal.limit(maxIndex - (index * ALLOCATION_SIZE));
|
||||
}
|
||||
|
||||
return retVal.asReadOnlyBuffer();
|
||||
}
|
||||
|
||||
public int[] toArray()
|
||||
{
|
||||
int[] retVal = new int[length()];
|
||||
|
|
|
@ -548,7 +548,7 @@ public class ImmutableConciseSetIntersectionTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testIntersectionTerminates() throws Exception
|
||||
public void testIntersectionTerminates()
|
||||
{
|
||||
verifyIntersection(Collections.emptyList(), Arrays.asList(new ImmutableConciseSet(), new ImmutableConciseSet()));
|
||||
}
|
||||
|
|
|
@ -21,17 +21,16 @@ package io.druid.emitter.ambari.metrics;
|
|||
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.java.util.emitter.core.Emitter;
|
||||
import io.druid.java.util.emitter.core.Event;
|
||||
import io.druid.java.util.emitter.service.AlertEvent;
|
||||
import io.druid.java.util.emitter.service.ServiceMetricEvent;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import org.apache.hadoop.metrics2.sink.timeline.AbstractTimelineMetricsSink;
|
||||
import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
|
||||
import org.apache.hadoop.metrics2.sink.timeline.TimelineMetrics;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.Executors;
|
||||
|
@ -202,7 +201,7 @@ public class AmbariMetricsEmitter extends AbstractTimelineMetricsSink implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public void flush() throws IOException
|
||||
public void flush()
|
||||
{
|
||||
synchronized (started) {
|
||||
if (started.get()) {
|
||||
|
@ -220,7 +219,7 @@ public class AmbariMetricsEmitter extends AbstractTimelineMetricsSink implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
public void close()
|
||||
{
|
||||
synchronized (started) {
|
||||
flush();
|
||||
|
|
|
@ -20,9 +20,10 @@
|
|||
package io.druid.emitter.ambari.metrics;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import io.druid.java.util.emitter.service.ServiceMetricEvent;
|
||||
import io.druid.annotations.UsedByJUnitParamsRunner;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.DateTimes;
|
||||
import io.druid.java.util.emitter.service.ServiceMetricEvent;
|
||||
import junitparams.JUnitParamsRunner;
|
||||
import junitparams.Parameters;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
|
@ -44,12 +45,8 @@ import java.io.OutputStream;
|
|||
public class WhiteListBasedDruidToTimelineEventConverterTest
|
||||
{
|
||||
private final String prefix = "druid";
|
||||
private final WhiteListBasedDruidToTimelineEventConverter defaultWhiteListBasedDruidToTimelineEventConverter = new WhiteListBasedDruidToTimelineEventConverter(
|
||||
prefix,
|
||||
"druid",
|
||||
null,
|
||||
new DefaultObjectMapper()
|
||||
);
|
||||
private final WhiteListBasedDruidToTimelineEventConverter defaultWhiteListBasedDruidToTimelineEventConverter =
|
||||
new WhiteListBasedDruidToTimelineEventConverter(prefix, "druid", null, new DefaultObjectMapper());
|
||||
private ServiceMetricEvent event;
|
||||
private final DateTime createdTime = DateTimes.nowUtc();
|
||||
private final String hostname = "testHost:8080";
|
||||
|
@ -143,6 +140,7 @@ public class WhiteListBasedDruidToTimelineEventConverterTest
|
|||
Assert.assertEquals(defaultNamespace + ".g1.jvm/gc/cpu", metric.getMetricName());
|
||||
}
|
||||
|
||||
@UsedByJUnitParamsRunner
|
||||
private Object[] parametersForTestGetName()
|
||||
{
|
||||
return new Object[]{
|
||||
|
|
|
@ -21,14 +21,12 @@ package io.druid.storage.azure;
|
|||
|
||||
import com.google.inject.Inject;
|
||||
import com.microsoft.azure.storage.StorageException;
|
||||
|
||||
import io.druid.java.util.common.MapUtils;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.segment.loading.DataSegmentKiller;
|
||||
import io.druid.segment.loading.SegmentLoadingException;
|
||||
import io.druid.timeline.DataSegment;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URISyntaxException;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.Map;
|
||||
|
@ -69,7 +67,7 @@ public class AzureDataSegmentKiller implements DataSegmentKiller
|
|||
}
|
||||
|
||||
@Override
|
||||
public void killAll() throws IOException
|
||||
public void killAll()
|
||||
{
|
||||
throw new UnsupportedOperationException("not implemented");
|
||||
}
|
||||
|
|
|
@ -19,22 +19,17 @@
|
|||
|
||||
package io.druid.storage.azure;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.io.ByteSource;
|
||||
import com.google.inject.Inject;
|
||||
import io.druid.java.util.common.CompressionUtils;
|
||||
import io.druid.java.util.common.MapUtils;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.segment.loading.DataSegmentPuller;
|
||||
import io.druid.segment.loading.SegmentLoadingException;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
public class AzureDataSegmentPuller implements DataSegmentPuller
|
||||
public class AzureDataSegmentPuller
|
||||
{
|
||||
private static final Logger log = new Logger(AzureDataSegmentPuller.class);
|
||||
|
||||
|
@ -55,7 +50,7 @@ public class AzureDataSegmentPuller implements DataSegmentPuller
|
|||
this.azureStorage = azureStorage;
|
||||
}
|
||||
|
||||
public io.druid.java.util.common.FileUtils.FileCopyResult getSegmentFiles(
|
||||
io.druid.java.util.common.FileUtils.FileCopyResult getSegmentFiles(
|
||||
final String containerName,
|
||||
final String blobPath,
|
||||
final File outDir
|
||||
|
@ -63,7 +58,7 @@ public class AzureDataSegmentPuller implements DataSegmentPuller
|
|||
throws SegmentLoadingException
|
||||
{
|
||||
try {
|
||||
prepareOutDir(outDir);
|
||||
FileUtils.forceMkdir(outDir);
|
||||
|
||||
log.info(
|
||||
"Loading container: [%s], with blobPath: [%s] and outDir: [%s]", containerName, blobPath, outDir
|
||||
|
@ -104,23 +99,5 @@ public class AzureDataSegmentPuller implements DataSegmentPuller
|
|||
}
|
||||
throw new SegmentLoadingException(e, e.getMessage());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getSegmentFiles(DataSegment segment, File outDir) throws SegmentLoadingException
|
||||
{
|
||||
|
||||
final Map<String, Object> loadSpec = segment.getLoadSpec();
|
||||
final String containerName = MapUtils.getString(loadSpec, "containerName");
|
||||
final String blobPath = MapUtils.getString(loadSpec, "blobPath");
|
||||
|
||||
getSegmentFiles(containerName, blobPath, outDir);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
void prepareOutDir(final File outDir) throws IOException
|
||||
{
|
||||
FileUtils.forceMkdir(outDir);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -81,10 +81,6 @@ public class AzureStorageDruidModule implements DruidModule
|
|||
{
|
||||
JsonConfigProvider.bind(binder, "druid.azure", AzureAccountConfig.class);
|
||||
|
||||
Binders.dataSegmentPullerBinder(binder)
|
||||
.addBinding(SCHEME)
|
||||
.to(AzureDataSegmentPuller.class).in(LazySingleton.class);
|
||||
|
||||
Binders.dataSegmentPusherBinder(binder)
|
||||
.addBinding(SCHEME)
|
||||
.to(AzureDataSegmentPusher.class).in(LazySingleton.class);
|
||||
|
|
|
@ -50,7 +50,7 @@ public class AzureTaskLogs implements TaskLogs
|
|||
}
|
||||
|
||||
@Override
|
||||
public void pushTaskLog(final String taskid, final File logFile) throws IOException
|
||||
public void pushTaskLog(final String taskid, final File logFile)
|
||||
{
|
||||
final String taskKey = getTaskLogKey(taskid);
|
||||
log.info("Pushing task log %s to: %s", logFile, taskKey);
|
||||
|
@ -123,13 +123,13 @@ public class AzureTaskLogs implements TaskLogs
|
|||
}
|
||||
|
||||
@Override
|
||||
public void killAll() throws IOException
|
||||
public void killAll()
|
||||
{
|
||||
throw new UnsupportedOperationException("not implemented");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void killOlderThan(long timestamp) throws IOException
|
||||
public void killOlderThan(long timestamp)
|
||||
{
|
||||
throw new UnsupportedOperationException("not implemented");
|
||||
}
|
||||
|
|
|
@ -19,13 +19,9 @@
|
|||
|
||||
package io.druid.storage.azure;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.microsoft.azure.storage.StorageException;
|
||||
import io.druid.java.util.common.FileUtils;
|
||||
import io.druid.java.util.common.Intervals;
|
||||
import io.druid.segment.loading.SegmentLoadingException;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import io.druid.timeline.partition.NoneShardSpec;
|
||||
import org.easymock.EasyMockSupport;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
@ -48,17 +44,6 @@ public class AzureDataSegmentPullerTest extends EasyMockSupport
|
|||
private static final String SEGMENT_FILE_NAME = "segment";
|
||||
private static final String containerName = "container";
|
||||
private static final String blobPath = "/path/to/storage/index.zip";
|
||||
private static final DataSegment dataSegment = new DataSegment(
|
||||
"test",
|
||||
Intervals.of("2015-04-12/2015-04-13"),
|
||||
"1",
|
||||
ImmutableMap.<String, Object>of("containerName", containerName, "blobPath", blobPath),
|
||||
null,
|
||||
null,
|
||||
NoneShardSpec.instance(),
|
||||
0,
|
||||
1
|
||||
);
|
||||
private AzureStorage azureStorage;
|
||||
|
||||
@Before
|
||||
|
@ -129,44 +114,4 @@ public class AzureDataSegmentPullerTest extends EasyMockSupport
|
|||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void getSegmentFilesTest() throws SegmentLoadingException
|
||||
{
|
||||
final File outDir = new File("");
|
||||
try {
|
||||
final FileUtils.FileCopyResult result = createMock(FileUtils.FileCopyResult.class);
|
||||
final AzureDataSegmentPuller puller = createMockBuilder(AzureDataSegmentPuller.class).withConstructor(
|
||||
azureStorage
|
||||
).addMockedMethod("getSegmentFiles", String.class, String.class, File.class).createMock();
|
||||
|
||||
expect(puller.getSegmentFiles(containerName, blobPath, outDir)).andReturn(result);
|
||||
|
||||
replayAll();
|
||||
|
||||
puller.getSegmentFiles(dataSegment, outDir);
|
||||
|
||||
verifyAll();
|
||||
}
|
||||
finally {
|
||||
outDir.delete();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void prepareOutDirTest() throws IOException
|
||||
{
|
||||
File outDir = Files.createTempDirectory("druid").toFile();
|
||||
|
||||
try {
|
||||
AzureDataSegmentPuller puller = new AzureDataSegmentPuller(azureStorage);
|
||||
puller.prepareOutDir(outDir);
|
||||
|
||||
assertTrue(outDir.exists());
|
||||
}
|
||||
finally {
|
||||
outDir.delete();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,12 +24,9 @@ import com.google.inject.Inject;
|
|||
import com.netflix.astyanax.recipes.storage.ChunkedStorage;
|
||||
import io.druid.java.util.common.CompressionUtils;
|
||||
import io.druid.java.util.common.FileUtils;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.RetryUtils;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.segment.loading.DataSegmentPuller;
|
||||
import io.druid.segment.loading.SegmentLoadingException;
|
||||
import io.druid.timeline.DataSegment;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileOutputStream;
|
||||
|
@ -39,7 +36,7 @@ import java.io.OutputStream;
|
|||
/**
|
||||
* Cassandra Segment Puller
|
||||
*/
|
||||
public class CassandraDataSegmentPuller extends CassandraStorage implements DataSegmentPuller
|
||||
public class CassandraDataSegmentPuller extends CassandraStorage
|
||||
{
|
||||
private static final Logger log = new Logger(CassandraDataSegmentPuller.class);
|
||||
private static final int CONCURRENCY = 10;
|
||||
|
@ -51,22 +48,14 @@ public class CassandraDataSegmentPuller extends CassandraStorage implements Data
|
|||
super(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getSegmentFiles(DataSegment segment, File outDir) throws SegmentLoadingException
|
||||
{
|
||||
String key = (String) segment.getLoadSpec().get("key");
|
||||
getSegmentFiles(key, outDir);
|
||||
}
|
||||
public FileUtils.FileCopyResult getSegmentFiles(final String key, final File outDir)
|
||||
throws SegmentLoadingException
|
||||
FileUtils.FileCopyResult getSegmentFiles(final String key, final File outDir) throws SegmentLoadingException
|
||||
{
|
||||
log.info("Pulling index from C* at path[%s] to outDir[%s]", key, outDir);
|
||||
if (!outDir.exists()) {
|
||||
outDir.mkdirs();
|
||||
try {
|
||||
org.apache.commons.io.FileUtils.forceMkdir(outDir);
|
||||
}
|
||||
|
||||
if (!outDir.isDirectory()) {
|
||||
throw new ISE("outDir[%s] must be a directory.", outDir);
|
||||
catch (IOException e) {
|
||||
throw new SegmentLoadingException(e, "");
|
||||
}
|
||||
|
||||
long startTime = System.currentTimeMillis();
|
||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.databind.Module;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.Key;
|
||||
import io.druid.guice.Binders;
|
||||
import io.druid.guice.JsonConfigProvider;
|
||||
import io.druid.guice.LazySingleton;
|
||||
import io.druid.guice.PolyBind;
|
||||
|
@ -42,11 +41,6 @@ public class CassandraDruidModule implements DruidModule
|
|||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
Binders.dataSegmentPullerBinder(binder)
|
||||
.addBinding(SCHEME)
|
||||
.to(CassandraDataSegmentPuller.class)
|
||||
.in(LazySingleton.class);
|
||||
|
||||
PolyBind.optionBinder(binder, Key.get(DataSegmentPusher.class))
|
||||
.addBinding(SCHEME)
|
||||
.to(CassandraDataSegmentPusher.class)
|
||||
|
|
|
@ -20,22 +20,17 @@
|
|||
package io.druid.storage.cloudfiles;
|
||||
|
||||
import com.google.inject.Inject;
|
||||
|
||||
import io.druid.java.util.common.CompressionUtils;
|
||||
import io.druid.java.util.common.FileUtils;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.MapUtils;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.segment.loading.DataSegmentPuller;
|
||||
import io.druid.segment.loading.SegmentLoadingException;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
public class CloudFilesDataSegmentPuller implements DataSegmentPuller
|
||||
public class CloudFilesDataSegmentPuller
|
||||
{
|
||||
|
||||
private static final Logger log = new Logger(CloudFilesDataSegmentPuller.class);
|
||||
|
@ -47,20 +42,7 @@ public class CloudFilesDataSegmentPuller implements DataSegmentPuller
|
|||
this.cloudFilesApi = cloudFilesApi;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getSegmentFiles(final DataSegment segment, final File outDir) throws SegmentLoadingException
|
||||
{
|
||||
final Map<String, Object> loadSpec = segment.getLoadSpec();
|
||||
final String region = MapUtils.getString(loadSpec, "region");
|
||||
final String container = MapUtils.getString(loadSpec, "container");
|
||||
final String path = MapUtils.getString(loadSpec, "path");
|
||||
|
||||
log.info("Pulling index at path[%s] to outDir[%s]", path, outDir);
|
||||
prepareOutDir(outDir);
|
||||
getSegmentFiles(region, container, path, outDir);
|
||||
}
|
||||
|
||||
public FileUtils.FileCopyResult getSegmentFiles(String region, String container, String path, File outDir)
|
||||
FileUtils.FileCopyResult getSegmentFiles(String region, String container, String path, File outDir)
|
||||
throws SegmentLoadingException
|
||||
{
|
||||
CloudFilesObjectApiProxy objectApi = new CloudFilesObjectApiProxy(cloudFilesApi, region, container);
|
||||
|
@ -68,8 +50,10 @@ public class CloudFilesDataSegmentPuller implements DataSegmentPuller
|
|||
|
||||
try {
|
||||
final FileUtils.FileCopyResult result = CompressionUtils.unzip(
|
||||
byteSource, outDir,
|
||||
CloudFilesUtils.CLOUDFILESRETRY, false
|
||||
byteSource,
|
||||
outDir,
|
||||
CloudFilesUtils.CLOUDFILESRETRY,
|
||||
false
|
||||
);
|
||||
log.info("Loaded %d bytes from [%s] to [%s]", result.size(), path, outDir.getAbsolutePath());
|
||||
return result;
|
||||
|
|
|
@ -31,7 +31,6 @@ import io.druid.timeline.DataSegment;
|
|||
import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.nio.file.Files;
|
||||
import java.util.Map;
|
||||
|
@ -74,7 +73,6 @@ public class CloudFilesDataSegmentPusher implements DataSegmentPusher
|
|||
|
||||
@Override
|
||||
public DataSegment push(final File indexFilesDir, final DataSegment inSegment, final boolean replaceExisting)
|
||||
throws IOException
|
||||
{
|
||||
final String segmentPath = CloudFilesUtils.buildCloudFilesPath(this.config.getBasePath(), getStorageDir(inSegment));
|
||||
|
||||
|
|
|
@ -25,13 +25,11 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.Provides;
|
||||
|
||||
import io.druid.guice.Binders;
|
||||
import io.druid.guice.JsonConfigProvider;
|
||||
import io.druid.guice.LazySingleton;
|
||||
import io.druid.initialization.DruidModule;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
|
||||
import org.jclouds.ContextBuilder;
|
||||
import org.jclouds.logging.slf4j.config.SLF4JLoggingModule;
|
||||
import org.jclouds.openstack.v2_0.config.InternalUrlModule;
|
||||
|
@ -86,8 +84,6 @@ public class CloudFilesStorageDruidModule implements DruidModule
|
|||
JsonConfigProvider.bind(binder, "druid.storage", CloudFilesDataSegmentPusherConfig.class);
|
||||
JsonConfigProvider.bind(binder, "druid.cloudfiles", CloudFilesAccountConfig.class);
|
||||
|
||||
Binders.dataSegmentPullerBinder(binder).addBinding(SCHEME).to(CloudFilesDataSegmentPuller.class)
|
||||
.in(LazySingleton.class);
|
||||
Binders.dataSegmentPusherBinder(binder).addBinding(SCHEME).to(CloudFilesDataSegmentPusher.class)
|
||||
.in(LazySingleton.class);
|
||||
|
||||
|
|
|
@ -310,7 +310,7 @@ public class RocketMQFirehoseFactory implements FirehoseFactory<InputRowParser<B
|
|||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
public void close()
|
||||
{
|
||||
defaultMQPullConsumer.shutdown();
|
||||
pullMessageService.shutdown(false);
|
||||
|
|
|
@ -61,7 +61,7 @@ public class GoogleDataSegmentKiller implements DataSegmentKiller
|
|||
}
|
||||
|
||||
@Override
|
||||
public void killAll() throws IOException
|
||||
public void killAll()
|
||||
{
|
||||
throw new UnsupportedOperationException("not implemented");
|
||||
}
|
||||
|
|
|
@ -19,25 +19,20 @@
|
|||
|
||||
package io.druid.storage.google;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.inject.Inject;
|
||||
import io.druid.java.util.common.CompressionUtils;
|
||||
import io.druid.java.util.common.FileUtils;
|
||||
import io.druid.java.util.common.MapUtils;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.segment.loading.DataSegmentPuller;
|
||||
import io.druid.segment.loading.SegmentLoadingException;
|
||||
import io.druid.segment.loading.URIDataPuller;
|
||||
import io.druid.timeline.DataSegment;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.URI;
|
||||
import java.util.Map;
|
||||
|
||||
public class GoogleDataSegmentPuller implements DataSegmentPuller, URIDataPuller
|
||||
public class GoogleDataSegmentPuller implements URIDataPuller
|
||||
{
|
||||
private static final Logger LOG = new Logger(GoogleDataSegmentPuller.class);
|
||||
|
||||
|
@ -49,23 +44,13 @@ public class GoogleDataSegmentPuller implements DataSegmentPuller, URIDataPuller
|
|||
this.storage = storage;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getSegmentFiles(final DataSegment segment, final File outDir) throws SegmentLoadingException
|
||||
{
|
||||
final Map<String, Object> loadSpec = segment.getLoadSpec();
|
||||
final String bucket = MapUtils.getString(loadSpec, "bucket");
|
||||
final String path = MapUtils.getString(loadSpec, "path");
|
||||
|
||||
getSegmentFiles(bucket, path, outDir);
|
||||
}
|
||||
|
||||
public FileUtils.FileCopyResult getSegmentFiles(final String bucket, final String path, File outDir)
|
||||
FileUtils.FileCopyResult getSegmentFiles(final String bucket, final String path, File outDir)
|
||||
throws SegmentLoadingException
|
||||
{
|
||||
LOG.info("Pulling index at bucket[%s] path[%s] to outDir[%s]", bucket, path, outDir.getAbsolutePath());
|
||||
|
||||
try {
|
||||
prepareOutDir(outDir);
|
||||
org.apache.commons.io.FileUtils.forceMkdir(outDir);
|
||||
|
||||
final GoogleByteSource byteSource = new GoogleByteSource(storage, bucket, path);
|
||||
final FileUtils.FileCopyResult result = CompressionUtils.unzip(
|
||||
|
@ -91,12 +76,6 @@ public class GoogleDataSegmentPuller implements DataSegmentPuller, URIDataPuller
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
void prepareOutDir(final File outDir) throws IOException
|
||||
{
|
||||
org.apache.commons.io.FileUtils.forceMkdir(outDir);
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputStream getInputStream(URI uri) throws IOException
|
||||
{
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue