提交 92bc3400 编写于 作者: F fjy

Merge branch 'master' into more-tests

......@@ -28,7 +28,7 @@
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.6.124-SNAPSHOT</version>
<version>0.6.129-SNAPSHOT</version>
</parent>
<dependencies>
......
......@@ -28,7 +28,7 @@
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.6.124-SNAPSHOT</version>
<version>0.6.129-SNAPSHOT</version>
</parent>
<dependencies>
......
......@@ -24,6 +24,7 @@ import com.google.common.base.Throwables;
import com.google.common.collect.Ordering;
import com.google.common.io.Closeables;
import com.metamx.common.guava.Accumulator;
import com.metamx.common.guava.CloseQuietly;
import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Yielder;
import com.metamx.common.guava.Yielders;
......@@ -70,7 +71,7 @@ public class OrderedMergeSequence<T> implements Sequence<T>
return yielder.get();
}
finally {
Closeables.closeQuietly(yielder);
CloseQuietly.close(yielder);
}
}
......
......@@ -46,14 +46,9 @@ public class SerializerUtils
public void writeString(OutputSupplier<? extends OutputStream> supplier, String name) throws IOException
{
OutputStream out = null;
try {
out = supplier.getOutput();
try (OutputStream out = supplier.getOutput()) {
writeString(out, name);
}
finally {
Closeables.closeQuietly(out);
}
}
public void writeString(WritableByteChannel out, String name) throws IOException
......
......@@ -17,7 +17,7 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.server;
package io.druid.common.utils;
import java.lang.reflect.InvocationTargetException;
......
......@@ -89,6 +89,9 @@ The interval is the [ISO8601 interval](http://en.wikipedia.org/wiki/ISO_8601#Tim
"user": "username",
"password": "passmeup",
"segmentTable": "segments"
},
"jobProperties": {
"mapreduce.job.queuename": "default"
}
}
```
......@@ -109,6 +112,7 @@ The interval is the [ISO8601 interval](http://en.wikipedia.org/wiki/ISO_8601#Tim
|leaveIntermediate|leave behind files in the workingPath when job completes or fails (debugging tool).|no|
|partitionsSpec|a specification of how to partition each time bucket into segments, absence of this property means no partitioning will occur.|no|
|metadataUpdateSpec|a specification of how to update the metadata for the druid cluster these segments belong to.|yes|
|jobProperties|a map of properties to add to the Hadoop job configuration.|no|
### Path specification
......
......@@ -154,6 +154,7 @@ Druid storage nodes maintain information about segments they have already downlo
|--------|-----------|-------|
|`druid.segmentCache.locations`|Segments assigned to a Historical node are first stored on the local file system (in a disk cache) and then served by the Historical node. These locations define where that local cache resides. | none (no caching) |
|`druid.segmentCache.deleteOnRemove`|Delete segment files from cache once a node is no longer serving a segment.|true|
|`druid.segmentCache.dropSegmentDelayMillis`|How long a node delays before completely dropping segment.|5 minutes|
|`druid.segmentCache.infoDir`|Historical nodes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the Coordinator to reassign. This path defines where this metadata is kept. Directory will be created if needed.|${first_location}/info_dir|
### Jetty Server Module
......
......@@ -69,10 +69,9 @@ A sample worker setup spec is shown below:
"keyName":"keyName"
},
"userData":{
"classType":"galaxy",
"env":"druid",
"version":"druid_version",
"type":"sample_cluster/worker"
"impl":"string",
"data":"version=:VERSION:",
"versionReplacementString":":VERSION:"
}
}
```
......@@ -81,8 +80,8 @@ Issuing a GET request at the same URL will return the current worker setup spec
|Property|Description|Default|
|--------|-----------|-------|
|`minVersion`|The coordinator only assigns tasks to workers with a version greater than the minVersion. If this is not specified, the minVersion will be the same as the coordinator version.|none|
|`minVersion`|The coordinator only assigns tasks to workers with a version greater than the minVersion. If this is not specified, the minVersion will be druid.indexer.runner.minWorkerVersion.|none|
|`minNumWorkers`|The minimum number of workers that can be in the cluster at any given time.|0|
|`maxNumWorkers`|The maximum number of workers that can be in the cluster at any given time.|0|
|`nodeData`|A JSON object that contains metadata about new nodes to create.|none|
|`userData`|A JSON object that contains metadata about how the node should register itself on startup. This data is sent with node creation requests.|none|
|`nodeData`|A JSON object that describes how to launch new nodes. Currently, only EC2 is supported.|none; required|
|`userData`|A JSON object that describes how to configure new nodes. Currently, only EC2 is supported. If you have set druid.indexer.autoscale.workerVersion, this must have a versionReplacementString. Otherwise, a versionReplacementString is not necessary.|none; optional|
......@@ -22,6 +22,7 @@ Additional peon configs include:
|`druid.indexer.task.baseTaskDir`|Base temporary working directory for tasks.|/tmp/persistent/tasks|
|`druid.indexer.task.hadoopWorkingPath`|Temporary working directory for Hadoop tasks.|/tmp/druid-indexing|
|`druid.indexer.task.defaultRowFlushBoundary`|Highest row count before persisting to disk. Used for indexing generating tasks.|50000|
|`druid.indexer.task.defaultHadoopCoordinates`|Hadoop version to use with HadoopIndexTasks that do not request a particular version.|org.apache.hadoop:hadoop-client:2.3.0|
|`druid.indexer.task.chathandler.type`|Choices are "noop" and "announce". Certain tasks will use service discovery to announce an HTTP endpoint that events can be posted to.|noop|
If the peon is running in remote mode, there must be an overlord up and running. Running peons in remote mode require the following configurations:
......
......@@ -153,7 +153,7 @@ druid.indexer.logs.s3Bucket=#{LOGS_BUCKET}
druid.indexer.logs.s3Prefix=prod/logs/v1
# Dedicate more resources to peons
druid.indexer.runner.javaOpts=-server -Xmx6g -Xms6g -XX:NewSize=256m -XX:MaxNewSize=256m -XX:+PrintGCDetails -XX:+PrintGCTimeStamps
druid.indexer.runner.javaOpts=-server -Xmx3g -XX:+UseG1GC -XX:MaxGCPauseMillis=100 -XX:+PrintGCDetails -XX:+PrintGCTimeStamps
druid.indexer.runner.taskDir=/mnt/persistent/task/
druid.indexer.task.taskDir=/mnt/persistent/task/
druid.indexer.task.chathandler.type=announce
......
......@@ -69,6 +69,20 @@ You should see a bunch of files:
* run_example_client.sh
* LICENSE, config, examples, lib directories
Setting up Zookeeper
--------------------
Before we get started, we need to start Apache Zookeeper.
```bash
curl http://apache.osuosl.org/zookeeper/zookeeper-3.4.5/zookeeper-3.4.5.tar.gz -o zookeeper-3.4.5.tar.gz
tar xzf zookeeper-3.4.5.tar.gz
cd zookeeper-3.4.5
cp conf/zoo_sample.cfg conf/zoo.cfg
./bin/zkServer.sh start
cd ..
```
Running Example Scripts
-----------------------
......
......@@ -85,28 +85,23 @@ config/overlord/runtime.properties
The configurations for the overlord node are as follows:
```bash
-server
-Xmx256m
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
druid.host=localhost
druid.port=8087
druid.service=overlord
-Ddruid.host=localhost
-Ddruid.port=8080
-Ddruid.service=overlord
druid.zk.service.host=localhost
-Ddruid.zk.service.host=localhost
druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.121"]
-Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.121"]
druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid
druid.db.connector.user=druid
druid.db.connector.password=diurd
-Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid
-Ddruid.db.connector.user=druid
-Ddruid.db.connector.password=diurd
-Ddruid.selectors.indexing.serviceName=overlord
-Ddruid.indexer.queue.startDelay=PT0M
-Ddruid.indexer.runner.javaOpts="-server -Xmx1g"
-Ddruid.indexer.fork.property.druid.processing.numThreads=1
-Ddruid.indexer.fork.property.druid.computation.buffer.size=100000000
druid.selectors.indexing.serviceName=overlord
druid.indexer.queue.startDelay=PT0M
druid.indexer.runner.javaOpts="-server -Xmx256m"
druid.indexer.fork.property.druid.processing.numThreads=1
druid.indexer.fork.property.druid.computation.buffer.size=100000000
```
If you are interested in reading more about these configurations, see [here](Indexing-Service.html).
......
......@@ -48,7 +48,7 @@ CREATE database druid;
#### Setting up Zookeeper
```bash
curl http://www.motorlogy.com/apache/zookeeper/zookeeper-3.4.5/zookeeper-3.4.5.tar.gz -o zookeeper-3.4.5.tar.gz
curl http://apache.osuosl.org/zookeeper/zookeeper-3.4.5/zookeeper-3.4.5.tar.gz -o zookeeper-3.4.5.tar.gz
tar xzf zookeeper-3.4.5.tar.gz
cd zookeeper-3.4.5
cp conf/zoo_sample.cfg conf/zoo.cfg
......@@ -120,7 +120,7 @@ druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
druid.db.connector.user=druid
druid.db.connector.password=diurd
druid.coordinator.startDelay=PT60s
druid.coordinator.startDelay=PT70s
```
To start the coordinator node:
......
-server
-Xmx256m
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
druid.host=localhost
druid.port=8087
druid.service=overlord
-Ddruid.host=localhost
-Ddruid.port=8080
-Ddruid.service=overlord
druid.zk.service.host=localhost
-Ddruid.zk.service.host=localhost
druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.121"]
-Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.121"]
druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid
druid.db.connector.user=druid
druid.db.connector.password=diurd
-Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid
-Ddruid.db.connector.user=druid
-Ddruid.db.connector.password=diurd
-Ddruid.selectors.indexing.serviceName=overlord
-Ddruid.indexer.queue.startDelay=PT0M
-Ddruid.indexer.runner.javaOpts="-server -Xmx256m"
-Ddruid.indexer.fork.property.druid.processing.numThreads=1
-Ddruid.indexer.fork.property.druid.computation.buffer.size=100000000
\ No newline at end of file
druid.selectors.indexing.serviceName=overlord
druid.indexer.queue.startDelay=PT0M
druid.indexer.runner.javaOpts="-server -Xmx256m"
druid.indexer.fork.property.druid.processing.numThreads=1
druid.indexer.fork.property.druid.computation.buffer.size=100000000
\ No newline at end of file
......@@ -28,7 +28,7 @@
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.6.124-SNAPSHOT</version>
<version>0.6.129-SNAPSHOT</version>
</parent>
<dependencies>
......
......@@ -28,7 +28,7 @@
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.6.124-SNAPSHOT</version>
<version>0.6.129-SNAPSHOT</version>
</parent>
<dependencies>
......
......@@ -21,6 +21,7 @@ package io.druid.storage.hdfs;
import com.google.common.io.Closeables;
import com.google.inject.Inject;
import com.metamx.common.guava.CloseQuietly;
import io.druid.segment.loading.DataSegmentPuller;
import io.druid.segment.loading.SegmentLoadingException;
import io.druid.timeline.DataSegment;
......@@ -52,22 +53,17 @@ public class HdfsDataSegmentPuller implements DataSegmentPuller
final FileSystem fs = checkPathAndGetFilesystem(path);
FSDataInputStream in = null;
try {
if (path.getName().endsWith(".zip")) {
in = fs.open(path);
CompressionUtils.unzip(in, dir);
in.close();
if (path.getName().endsWith(".zip")) {
try {
try (FSDataInputStream in = fs.open(path)) {
CompressionUtils.unzip(in, dir);
}
}
else {
throw new SegmentLoadingException("Unknown file type[%s]", path);
catch (IOException e) {
throw new SegmentLoadingException(e, "Some IOException");
}
}
catch (IOException e) {
throw new SegmentLoadingException(e, "Some IOException");
}
finally {
Closeables.closeQuietly(in);
} else {
throw new SegmentLoadingException("Unknown file type[%s]", path);
}
}
......@@ -85,7 +81,8 @@ public class HdfsDataSegmentPuller implements DataSegmentPuller
}
}
private Path getPath(DataSegment segment) {
private Path getPath(DataSegment segment)
{
return new Path(String.valueOf(segment.getLoadSpec().get("path")));
}
......
......@@ -26,6 +26,7 @@ import com.google.common.io.ByteStreams;
import com.google.common.io.Closeables;
import com.google.common.io.OutputSupplier;
import com.google.inject.Inject;
import com.metamx.common.guava.CloseQuietly;
import com.metamx.common.logger.Logger;
import io.druid.segment.SegmentUtils;
import io.druid.segment.loading.DataSegmentPusher;
......@@ -78,17 +79,10 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher
fs.mkdirs(outFile.getParent());
log.info("Compressing files from[%s] to [%s]", inDir, outFile);
FSDataOutputStream out = null;
long size;
try {
out = fs.create(outFile);
long size;
try (FSDataOutputStream out = fs.create(outFile)) {
size = CompressionUtils.zip(inDir, out);
out.close();
}
finally {
Closeables.closeQuietly(out);
}
return createDescriptorFile(
......
......@@ -28,7 +28,7 @@
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.6.124-SNAPSHOT</version>
<version>0.6.129-SNAPSHOT</version>
</parent>
<dependencies>
......
......@@ -44,7 +44,9 @@ import io.druid.guice.JsonConfigProvider;
import io.druid.guice.annotations.Self;
import io.druid.indexer.partitions.PartitionsSpec;
import io.druid.indexer.path.PathSpec;
import io.druid.guice.GuiceInjectors;
import io.druid.initialization.Initialization;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.indexing.granularity.GranularitySpec;
import io.druid.server.DruidNode;
import io.druid.timeline.DataSegment;
......@@ -81,7 +83,7 @@ public class HadoopDruidIndexerConfig
static {
injector = Initialization.makeInjectorWithModules(
Initialization.makeStartupInjector(),
GuiceInjectors.makeStartupInjector(),
ImmutableList.<Object>of(
new Module()
{
......@@ -166,12 +168,14 @@ public class HadoopDruidIndexerConfig
private volatile HadoopIngestionSpec schema;
private volatile PathSpec pathSpec;
private volatile ColumnConfig columnConfig;
@JsonCreator
public HadoopDruidIndexerConfig(
final @JsonProperty("schema") HadoopIngestionSpec schema
)
{
this.columnConfig = columnConfig;
this.schema = schema;
this.pathSpec = jsonMapper.convertValue(schema.getIOConfig().getPathSpec(), PathSpec.class);
}
......@@ -182,6 +186,11 @@ public class HadoopDruidIndexerConfig
return schema;
}
public ColumnConfig getColumnConfig()
{
return columnConfig;
}
public String getDataSource()
{
return schema.getDataSchema().getDataSource();
......
......@@ -31,6 +31,7 @@ import com.google.common.io.Closeables;
import com.google.common.primitives.Longs;
import com.metamx.common.IAE;
import com.metamx.common.ISE;
import com.metamx.common.guava.CloseQuietly;
import com.metamx.common.logger.Logger;
import io.druid.data.input.InputRow;
import io.druid.data.input.impl.StringInputRowParser;
......@@ -425,7 +426,7 @@ public class IndexGeneratorJob implements Jobby
if (caughtException == null) {
Closeables.close(out, false);
} else {
Closeables.closeQuietly(out);
CloseQuietly.close(out);
throw Throwables.propagate(caughtException);
}
}
......@@ -605,7 +606,7 @@ public class IndexGeneratorJob implements Jobby
}
}
finally {
Closeables.closeQuietly(in);
CloseQuietly.close(in);
}
out.closeEntry();
context.progress();
......
......@@ -45,7 +45,6 @@ public class HadoopDruidIndexerConfigTest
}
}
@Test
public void shouldMakeHDFSCompliantSegmentOutputPath()
{
......
......@@ -28,7 +28,7 @@
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.6.124-SNAPSHOT</version>
<version>0.6.129-SNAPSHOT</version>
</parent>
<dependencies>
......
......@@ -49,7 +49,6 @@ import java.io.IOException;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ExecutorService;
/**
......@@ -187,7 +186,8 @@ public class TaskToolbox
return retVal;
}
public void pushSegments(Iterable<DataSegment> segments) throws IOException {
public void pushSegments(Iterable<DataSegment> segments) throws IOException
{
// Request segment pushes for each set
final Multimap<Interval, DataSegment> segmentMultimap = Multimaps.index(
segments,
......
......@@ -29,6 +29,7 @@ import io.druid.indexing.common.actions.TaskActionClientFactory;
import io.druid.indexing.common.config.TaskConfig;
import io.druid.indexing.common.task.Task;
import io.druid.query.QueryRunnerFactoryConglomerate;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.loading.DataSegmentArchiver;
import io.druid.segment.loading.DataSegmentKiller;
import io.druid.segment.loading.DataSegmentMover;
......
......@@ -21,11 +21,17 @@ package io.druid.indexing.common.config;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableList;
import java.io.File;
import java.util.List;
public class TaskConfig
{
public static List<String> DEFAULT_DEFAULT_HADOOP_COORDINATES = ImmutableList.of(
"org.apache.hadoop:hadoop-client:2.3.0"
);
@JsonProperty
private final String baseDir;
......@@ -38,40 +44,57 @@ public class TaskConfig
@JsonProperty
private final int defaultRowFlushBoundary;
@JsonProperty
private final List<String> defaultHadoopCoordinates;
@JsonCreator
public TaskConfig(
@JsonProperty("baseDir") String baseDir,
@JsonProperty("baseTaskDir") String baseTaskDir,
@JsonProperty("hadoopWorkingPath") String hadoopWorkingPath,
@JsonProperty("defaultRowFlushBoundary") Integer defaultRowFlushBoundary
@JsonProperty("defaultRowFlushBoundary") Integer defaultRowFlushBoundary,
@JsonProperty("defaultHadoopCoordinates") List<String> defaultHadoopCoordinates
)
{
this.baseDir = baseDir == null ? "/tmp" : baseDir;
this.baseTaskDir = new File(defaultDir(baseTaskDir, "persistent/task"));
this.hadoopWorkingPath = defaultDir(hadoopWorkingPath, "druid-indexing");
this.defaultRowFlushBoundary = defaultRowFlushBoundary == null ? 500000 : defaultRowFlushBoundary;
this.defaultHadoopCoordinates = defaultHadoopCoordinates == null
? DEFAULT_DEFAULT_HADOOP_COORDINATES
: defaultHadoopCoordinates;
}
@JsonProperty
public String getBaseDir()
{
return baseDir;
}
@JsonProperty
public File getBaseTaskDir()
{
return baseTaskDir;
}
@JsonProperty
public String getHadoopWorkingPath()
{
return hadoopWorkingPath;
}
@JsonProperty
public int getDefaultRowFlushBoundary()
{
return defaultRowFlushBoundary;
}
@JsonProperty
public List<String> getDefaultHadoopCoordinates()
{
return defaultHadoopCoordinates;
}
private String defaultDir(String configParameter, final String defaultVal)
{
if (configParameter == null) {
......@@ -80,4 +103,4 @@ public class TaskConfig
return configParameter;
}
}
\ No newline at end of file
}
......@@ -27,6 +27,7 @@ import com.google.api.client.util.Lists;
import com.google.common.base.Joiner;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.metamx.common.logger.Logger;
import io.druid.common.utils.JodaUtils;
......@@ -41,8 +42,9 @@ import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.LockAcquireAction;
import io.druid.indexing.common.actions.LockTryAcquireAction;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.guice.GuiceInjectors;
import io.druid.initialization.Initialization;
import io.druid.server.initialization.ExtensionsConfig;
import io.druid.guice.ExtensionsConfig;
import io.druid.timeline.DataSegment;
import io.tesla.aether.internal.DefaultTeslaAether;
import org.joda.time.DateTime;
......@@ -62,11 +64,9 @@ public class HadoopIndexTask extends AbstractTask
private static final ExtensionsConfig extensionsConfig;
static {
extensionsConfig = Initialization.makeStartupInjector().getInstance(ExtensionsConfig.class);
extensionsConfig = GuiceInjectors.makeStartupInjector().getInstance(ExtensionsConfig.class);
}
public static String DEFAULT_HADOOP_COORDINATES = "org.apache.hadoop:hadoop-client:2.3.0";
private static String getTheDataSource(HadoopIngestionSpec spec, HadoopIngestionSpec config)
{
if (spec != null) {
......@@ -115,9 +115,14 @@ public class HadoopIndexTask extends AbstractTask
Preconditions.checkArgument(this.spec.getTuningConfig().getWorkingPath() == null, "workingPath must be absent");
Preconditions.checkArgument(this.spec.getIOConfig().getMetadataUpdateSpec() == null, "updaterJobSpec must be absent");
this.hadoopDependencyCoordinates = hadoopDependencyCoordinates == null ? Arrays.<String>asList(
hadoopCoordinates == null ? DEFAULT_HADOOP_COORDINATES : hadoopCoordinates
) : hadoopDependencyCoordinates;
if (hadoopDependencyCoordinates != null) {
this.hadoopDependencyCoordinates = hadoopDependencyCoordinates;
} else if (hadoopCoordinates != null) {
this.hadoopDependencyCoordinates = ImmutableList.of(hadoopCoordinates);
} else {
// Will be defaulted to something at runtime, based on taskConfig.
this.hadoopDependencyCoordinates = null;
}
}
@Override
......@@ -158,6 +163,10 @@ public class HadoopIndexTask extends AbstractTask
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
final List<String> finalHadoopDependencyCoordinates = hadoopDependencyCoordinates != null
? hadoopDependencyCoordinates
: toolbox.getConfig().getDefaultHadoopCoordinates();
final DefaultTeslaAether aetherClient = Initialization.getAetherClient(extensionsConfig);
final List<URL> extensionURLs = Lists.newArrayList();
......@@ -174,7 +183,7 @@ public class HadoopIndexTask extends AbstractTask
final List<URL> driverURLs = Lists.newArrayList();
driverURLs.addAll(nonHadoopURLs);
// put hadoop dependencies last to avoid jets3t & apache.httpcore version conflicts
for (String hadoopDependencyCoordinate : hadoopDependencyCoordinates) {
for (String hadoopDependencyCoordinate : finalHadoopDependencyCoordinates) {
final ClassLoader hadoopLoader = Initialization.getClassLoaderForCoordinates(
aetherClient, hadoopDependencyCoordinate
);
......
......@@ -19,14 +19,15 @@
package io.druid.indexing.common.task;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.io.Closeables;
import com.metamx.common.Granularity;
import com.metamx.common.exception.FormattedException;
import com.metamx.common.guava.CloseQuietly;
import com.metamx.common.parsers.ParseException;
import com.metamx.emitter.EmittingLogger;
import io.druid.data.input.Firehose;
import io.druid.data.input.FirehoseFactory;
......@@ -43,9 +44,10 @@ import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryRunnerFactoryConglomerate;
import io.druid.query.QueryToolChest;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.RealtimeTuningConfig;
import io.druid.segment.indexing.RealtimeIOConfig;
import io.druid.segment.indexing.RealtimeTuningConfig;
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
import io.druid.segment.realtime.FireDepartment;
import io.druid.segment.realtime.FireDepartmentConfig;
......@@ -353,7 +355,7 @@ public class RealtimeIndexTask extends AbstractTask
nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis();
}
}
catch (FormattedException e) {
catch (ParseException e) {
log.warn(e, "unparseable line");
fireDepartment.getMetrics().incrementUnparseable();
}
......@@ -375,7 +377,7 @@ public class RealtimeIndexTask extends AbstractTask
log.makeAlert(e, "Failed to finish realtime task").emit();
}
finally {
Closeables.closeQuietly(firehose);
CloseQuietly.close(firehose);
toolbox.getMonitorScheduler().removeMonitor(metricsMonitor);
}
}
......
......@@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Sets;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.common.logger.Logger;
......
......@@ -585,7 +585,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
// Syncing state with Zookeeper - don't assign new tasks until the task we just assigned is actually running
// on a worker - this avoids overflowing a worker with tasks
Stopwatch timeoutStopwatch = new Stopwatch();
Stopwatch timeoutStopwatch = Stopwatch.createUnstarted();
timeoutStopwatch.start();
synchronized (statusLock) {
while (!isWorkerRunningTask(theWorker, task.getId())) {
......
......@@ -29,19 +29,14 @@ import com.amazonaws.services.ec2.model.Reservation;
import com.amazonaws.services.ec2.model.RunInstancesRequest;
import com.amazonaws.services.ec2.model.RunInstancesResult;
import com.amazonaws.services.ec2.model.TerminateInstancesRequest;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Supplier;
import com.google.common.collect.Lists;
import com.google.inject.Inject;
import com.metamx.emitter.EmittingLogger;
import io.druid.guice.annotations.Json;
import io.druid.indexing.overlord.setup.EC2NodeData;
import io.druid.indexing.overlord.setup.GalaxyUserData;
import io.druid.indexing.overlord.setup.WorkerSetupData;
import org.apache.commons.codec.binary.Base64;
import javax.annotation.Nullable;
import java.util.List;
/**
......@@ -50,20 +45,17 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy
{
private static final EmittingLogger log = new EmittingLogger(EC2AutoScalingStrategy.class);
private final ObjectMapper jsonMapper;
private final AmazonEC2 amazonEC2Client;
private final SimpleResourceManagementConfig config;
private final Supplier<WorkerSetupData> workerSetupDataRef;
@Inject
public EC2AutoScalingStrategy(
@Json ObjectMapper jsonMapper,
AmazonEC2 amazonEC2Client,
SimpleResourceManagementConfig config,
Supplier<WorkerSetupData> workerSetupDataRef
)
{
this.jsonMapper = jsonMapper;
this.amazonEC2Client = amazonEC2Client;
this.config = config;
this.workerSetupDataRef = workerSetupDataRef;
......@@ -73,15 +65,21 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy
public AutoScalingData provision()
{
try {
WorkerSetupData setupData = workerSetupDataRef.get();
EC2NodeData workerConfig = setupData.getNodeData();
GalaxyUserData userData = setupData.getUserData();
if (config.getWorkerVersion() != null) {
userData = userData.withVersion(config.getWorkerVersion());
final WorkerSetupData setupData = workerSetupDataRef.get();
final EC2NodeData workerConfig = setupData.getNodeData();
final String userDataBase64;
if (setupData.getUserData() == null) {
userDataBase64 = null;
} else {
if (config.getWorkerVersion() == null) {
userDataBase64 = setupData.getUserData().getUserDataBase64();
} else {
userDataBase64 = setupData.getUserData().withVersion(config.getWorkerVersion()).getUserDataBase64();
}
}
RunInstancesResult result = amazonEC2Client.runInstances(
final RunInstancesResult result = amazonEC2Client.runInstances(
new RunInstancesRequest(
workerConfig.getAmiId(),
workerConfig.getMinInstances(),
......@@ -91,16 +89,10 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy
.withSecurityGroupIds(workerConfig.getSecurityGroupIds())
.withPlacement(new Placement(setupData.getAvailabilityZone()))
.withKeyName(workerConfig.getKeyName())
.withUserData(
Base64.encodeBase64String(
jsonMapper.writeValueAsBytes(
userData
)
)
)
.withUserData(userDataBase64)
);
List<String> instanceIds = Lists.transform(
final List<String> instanceIds = Lists.transform(
result.getReservation().getInstances(),
new Function<Instance, String>()
{
......
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.indexing.overlord.setup;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
/**
* Represents any user data that may be needed to launch EC2 instances.
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "impl", defaultImpl = GalaxyEC2UserData.class)
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "galaxy", value = GalaxyEC2UserData.class),
@JsonSubTypes.Type(name = "string", value = StringEC2UserData.class)
})
public interface EC2UserData<T extends EC2UserData>
{
/**
* Return a copy of this instance with a different worker version. If no changes are needed (possibly because the
* user data does not depend on the worker version) then it is OK to return "this".
*/
public EC2UserData<T> withVersion(String version);
public String getUserDataBase64();
}
......@@ -19,24 +19,32 @@
package io.druid.indexing.overlord.setup;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.api.client.repackaged.com.google.common.base.Throwables;
import io.druid.guice.annotations.Json;
import org.apache.commons.codec.binary.Base64;
/**
*/
public class GalaxyUserData
public class GalaxyEC2UserData implements EC2UserData<GalaxyEC2UserData>
{
public final String env;
public final String version;
public final String type;
private final ObjectMapper jsonMapper;
private final String env;
private final String version;
private final String type;
@JsonCreator
public GalaxyUserData(
public GalaxyEC2UserData(
@JacksonInject @Json ObjectMapper jsonMapper,
@JsonProperty("env") String env,
@JsonProperty("version") String version,
@JsonProperty("type") String type
)
{
this.jsonMapper = jsonMapper;
this.env = env;
this.version = version;
this.type = type;
......@@ -60,9 +68,21 @@ public class GalaxyUserData
return type;
}
public GalaxyUserData withVersion(String ver)
@Override
public GalaxyEC2UserData withVersion(String ver)
{
return new GalaxyEC2UserData(jsonMapper, env, ver, type);
}
@Override
public String getUserDataBase64()
{
return new GalaxyUserData(env, ver, type);
try {
return Base64.encodeBase64String(jsonMapper.writeValueAsBytes(this));
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
@Override
......
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.indexing.overlord.setup;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.api.client.util.Charsets;
import org.apache.commons.codec.binary.Base64;
public class StringEC2UserData implements EC2UserData<StringEC2UserData>
{
private final String data;
private final String versionReplacementString;
private final String version;
@JsonCreator
public StringEC2UserData(
@JsonProperty("data") String data,
@JsonProperty("versionReplacementString") String versionReplacementString,
@JsonProperty("version") String version
)
{
this.data = data;
this.versionReplacementString = versionReplacementString;
this.version = version;
}
@JsonProperty
public String getData()
{
return data;
}
@JsonProperty
public String getVersionReplacementString()
{
return versionReplacementString;
}
@JsonProperty
public String getVersion()
{
return version;
}
@Override
public StringEC2UserData withVersion(final String _version)
{
return new StringEC2UserData(data, versionReplacementString, _version);
}
@Override
public String getUserDataBase64()
{
final String finalData;
if (versionReplacementString != null && version != null) {
finalData = data.replace(versionReplacementString, version);
} else {
finalData = data;
}
return Base64.encodeBase64String(finalData.getBytes(Charsets.UTF_8));
}
@Override
public String toString()
{
return "StringEC2UserData{" +
"data='" + data + '\'' +
", versionReplacementString='" + versionReplacementString + '\'' +
", version='" + version + '\'' +
'}';
}
}
......@@ -33,7 +33,7 @@ public class WorkerSetupData
private final int maxNumWorkers;
private final String availabilityZone;
private final EC2NodeData nodeData;
private final GalaxyUserData userData;
private final EC2UserData userData;
@JsonCreator
public WorkerSetupData(
......@@ -42,7 +42,7 @@ public class WorkerSetupData
@JsonProperty("maxNumWorkers") int maxNumWorkers,
@JsonProperty("availabilityZone") String availabilityZone,
@JsonProperty("nodeData") EC2NodeData nodeData,
@JsonProperty("userData") GalaxyUserData userData
@JsonProperty("userData") EC2UserData userData
)
{
this.minVersion = minVersion;
......@@ -84,7 +84,7 @@ public class WorkerSetupData
}
@JsonProperty
public GalaxyUserData getUserData()
public EC2UserData getUserData()
{
return userData;
}
......
......@@ -19,19 +19,51 @@
package io.druid.indexing.common;
import com.fasterxml.jackson.databind.BeanProperty;
import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Stopwatch;
import com.metamx.common.ISE;
import io.druid.guice.ServerModule;
import io.druid.jackson.DefaultObjectMapper;
import java.util.List;
import java.util.concurrent.TimeUnit;
/**
*/
public class TestUtils
{
public static final ObjectMapper MAPPER = new DefaultObjectMapper();
static {
final List<? extends Module> list = new ServerModule().getJacksonModules();
for (Module module : list) {
MAPPER.registerModule(module);
}
MAPPER.setInjectableValues(
new InjectableValues()
{
@Override
public Object findInjectableValue(
Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance
)
{
if (valueId.equals("com.fasterxml.jackson.databind.ObjectMapper")) {
return TestUtils.MAPPER;
}
throw new ISE("No Injectable value found");
}
}
);
}
public static boolean conditionValid(IndexingServiceCondition condition)
{
try {
Stopwatch stopwatch = new Stopwatch();
Stopwatch stopwatch = Stopwatch.createUnstarted();
stopwatch.start();
while (!condition.isValid()) {
Thread.sleep(100);
......
......@@ -48,6 +48,8 @@ import java.io.File;
public class TaskSerdeTest
{
private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
@Test
public void testIndexTaskSerde() throws Exception
{
......@@ -68,7 +70,6 @@ public class TaskSerdeTest
-1
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
for (final Module jacksonModule : new FirehoseModule().getJacksonModules()) {
jsonMapper.registerModule(jacksonModule);
}
......@@ -102,7 +103,6 @@ public class TaskSerdeTest
)
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
......@@ -131,7 +131,6 @@ public class TaskSerdeTest
new Interval("2010-01-01/P1D")
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
......@@ -153,7 +152,6 @@ public class TaskSerdeTest
DataSegment.builder().dataSource("foo").interval(new Interval("2010-01-01/P1D")).version("1234").build()
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
......@@ -177,7 +175,6 @@ public class TaskSerdeTest
DataSegment.builder().dataSource("foo").interval(new Interval("2010-01-01/P1D")).version("1234").build()
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
......@@ -208,7 +205,6 @@ public class TaskSerdeTest
null
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
......@@ -251,7 +247,6 @@ public class TaskSerdeTest
new Interval("2010-01-01/P1D")
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
......@@ -269,7 +264,6 @@ public class TaskSerdeTest
@Test
public void testDeleteTaskFromJson() throws Exception
{
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final DeleteTask task = (DeleteTask) jsonMapper.readValue(
"{\"type\":\"delete\",\"dataSource\":\"foo\",\"interval\":\"2010-01-01/P1D\"}",
Task.class
......@@ -300,7 +294,6 @@ public class TaskSerdeTest
)
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
......@@ -325,7 +318,6 @@ public class TaskSerdeTest
new Interval("2010-01-01/P1D")
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
......@@ -350,7 +342,6 @@ public class TaskSerdeTest
new Interval("2010-01-01/P1D")
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
......@@ -375,7 +366,6 @@ public class TaskSerdeTest
ImmutableMap.<String, Object>of("bucket", "hey", "baseKey", "what")
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
......@@ -434,7 +424,6 @@ public class TaskSerdeTest
null
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
final HadoopIndexTask task2 = (HadoopIndexTask) jsonMapper.readValue(json, Task.class);
......
......@@ -43,6 +43,7 @@ import io.druid.data.input.InputRow;
import io.druid.data.input.MapBasedInputRow;
import io.druid.data.input.impl.InputRowParser;
import io.druid.granularity.QueryGranularity;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
import io.druid.indexing.common.SegmentLoaderFactory;
import io.druid.indexing.common.TaskLock;
......@@ -135,7 +136,7 @@ public class TaskLifecycleTest
mdc = newMockMDC();
tac = new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tl, mdc, newMockEmitter()));
tb = new TaskToolboxFactory(
new TaskConfig(tmp.toString(), null, null, 50000),
new TaskConfig(tmp.toString(), null, null, 50000, null),
tac,
newMockEmitter(),
new DataSegmentPusher()
......
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.indexing.overlord;
import com.google.common.base.Charsets;
import io.druid.indexing.common.TestUtils;
import io.druid.indexing.overlord.setup.EC2UserData;
import io.druid.indexing.overlord.setup.GalaxyEC2UserData;
import io.druid.indexing.overlord.setup.StringEC2UserData;
import org.apache.commons.codec.binary.Base64;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
public class WorkerSetupDataTest
{
@Test
public void testGalaxyEC2UserDataSerde() throws IOException
{
final String json = "{\"env\":\"druid\",\"version\":null,\"type\":\"typical\"}";
final GalaxyEC2UserData userData = (GalaxyEC2UserData) TestUtils.MAPPER.readValue(json, EC2UserData.class);
Assert.assertEquals("druid", userData.getEnv());
Assert.assertEquals("typical", userData.getType());
Assert.assertNull(userData.getVersion());
Assert.assertEquals("1234", userData.withVersion("1234").getVersion());
}
@Test
public void testStringEC2UserDataSerde() throws IOException
{
final String json = "{\"impl\":\"string\",\"data\":\"hey :ver:\",\"versionReplacementString\":\":ver:\",\"version\":\"1234\"}";
final StringEC2UserData userData = (StringEC2UserData) TestUtils.MAPPER.readValue(json, EC2UserData.class);
Assert.assertEquals("hey :ver:", userData.getData());
Assert.assertEquals("1234", userData.getVersion());
Assert.assertEquals(
Base64.encodeBase64String("hey 1234".getBytes(Charsets.UTF_8)),
userData.getUserDataBase64()
);
Assert.assertEquals(
Base64.encodeBase64String("hey xyz".getBytes(Charsets.UTF_8)),
userData.withVersion("xyz").getUserDataBase64()
);
}
}
......@@ -30,7 +30,7 @@ import com.amazonaws.services.ec2.model.TerminateInstancesRequest;
import com.google.common.collect.Lists;
import io.druid.common.guava.DSuppliers;
import io.druid.indexing.overlord.setup.EC2NodeData;
import io.druid.indexing.overlord.setup.GalaxyUserData;
import io.druid.indexing.overlord.setup.GalaxyEC2UserData;
import io.druid.indexing.overlord.setup.WorkerSetupData;
import io.druid.jackson.DefaultObjectMapper;
import org.easymock.EasyMock;
......@@ -75,7 +75,6 @@ public class EC2AutoScalingStrategyTest
.withPrivateIpAddress(IP);
strategy = new EC2AutoScalingStrategy(
new DefaultObjectMapper(),
amazonEC2Client,
new SimpleResourceManagementConfig().setWorkerPort(8080).setWorkerVersion(""),
DSuppliers.of(workerSetupData)
......@@ -101,7 +100,7 @@ public class EC2AutoScalingStrategyTest
1,
"",
new EC2NodeData(AMI_ID, INSTANCE_ID, 1, 1, Lists.<String>newArrayList(), "foo"),
new GalaxyUserData("env", "version", "type")
new GalaxyEC2UserData(new DefaultObjectMapper(), "env", "version", "type")
)
);
......
......@@ -20,11 +20,17 @@
package io.druid.indexing.worker;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.introspect.AnnotationIntrospectorPair;
import com.fasterxml.jackson.databind.introspect.GuiceAnnotationIntrospector;
import com.fasterxml.jackson.databind.introspect.GuiceInjectableValues;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.io.Files;
import com.google.inject.Binder;
import com.google.inject.Guice;
import com.google.inject.Injector;
import io.druid.curator.PotentiallyGzippedCompressionProvider;
import io.druid.indexing.common.IndexingServiceCondition;
import io.druid.indexing.common.SegmentLoaderFactory;
......@@ -38,6 +44,7 @@ import io.druid.indexing.overlord.TestRemoteTaskRunnerConfig;
import io.druid.indexing.overlord.ThreadPoolTaskRunner;
import io.druid.indexing.worker.config.WorkerConfig;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.loading.DataSegmentPuller;
import io.druid.segment.loading.LocalDataSegmentPuller;
import io.druid.segment.loading.OmniSegmentLoader;
......@@ -61,6 +68,40 @@ import java.util.List;
public class WorkerTaskMonitorTest
{
private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
private static final Injector injector = Guice.createInjector(
new com.google.inject.Module()
{
@Override
public void configure(Binder binder)
{
binder.bind(ColumnConfig.class).toInstance(
new ColumnConfig()
{
@Override
public int columnCacheSizeBytes()
{
return 1024 * 1024;
}
}
);
}
}
);
static {
final GuiceAnnotationIntrospector guiceIntrospector = new GuiceAnnotationIntrospector();
jsonMapper.setInjectableValues(new GuiceInjectableValues(injector));
jsonMapper.setAnnotationIntrospectors(
new AnnotationIntrospectorPair(
guiceIntrospector, jsonMapper.getSerializationConfig().getAnnotationIntrospector()
),
new AnnotationIntrospectorPair(
guiceIntrospector, jsonMapper.getDeserializationConfig().getAnnotationIntrospector()
)
);
}
private static final Joiner joiner = Joiner.on("/");
private static final String basePath = "/test/druid";
private static final String tasksPath = String.format("%s/indexer/tasks/worker", basePath);
......@@ -121,7 +162,7 @@ public class WorkerTaskMonitorTest
workerCuratorCoordinator,
new ThreadPoolTaskRunner(
new TaskToolboxFactory(
new TaskConfig(tmp.toString(), null, null, 0),
new TaskConfig(tmp.toString(), null, null, 0, null),
null, null, null, null, null, null, null, null, null, null, null, new SegmentLoaderFactory(
new OmniSegmentLoader(
ImmutableMap.<String, DataSegmentPuller>of(
......
......@@ -28,7 +28,7 @@
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.6.124-SNAPSHOT</version>
<version>0.6.129-SNAPSHOT</version>
</parent>
<dependencies>
......
......@@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Sets;
import com.metamx.common.exception.FormattedException;
import com.metamx.common.logger.Logger;
import io.druid.data.input.ByteBufferInputRowParser;
import io.druid.data.input.Firehose;
......@@ -115,7 +114,7 @@ public class KafkaEightFirehoseFactory implements FirehoseFactory<ByteBufferInpu
}
@Override
public InputRow nextRow() throws FormattedException
public InputRow nextRow()
{
final byte[] message = iter.next().message();
......@@ -123,15 +122,7 @@ public class KafkaEightFirehoseFactory implements FirehoseFactory<ByteBufferInpu
return null;
}
try {
return theParser.parse(ByteBuffer.wrap(message));
}
catch (Exception e) {
throw new FormattedException.Builder()
.withErrorCode(FormattedException.ErrorCode.UNPARSABLE_ROW)
.withMessage(String.format("Error parsing[%s], got [%s]", ByteBuffer.wrap(message), e.toString()))
.build();
}
return theParser.parse(ByteBuffer.wrap(message));
}
@Override
......
......@@ -28,7 +28,7 @@
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.6.124-SNAPSHOT</version>
<version>0.6.129-SNAPSHOT</version>
</parent>
<dependencies>
......
......@@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Sets;
import com.metamx.common.exception.FormattedException;
import com.metamx.common.logger.Logger;
import io.druid.data.input.ByteBufferInputRowParser;
import io.druid.data.input.Firehose;
......@@ -123,7 +122,7 @@ public class KafkaSevenFirehoseFactory implements FirehoseFactory<ByteBufferInpu
}
@Override
public InputRow nextRow() throws FormattedException
public InputRow nextRow()
{
final Message message = iter.next().message();
......@@ -134,17 +133,9 @@ public class KafkaSevenFirehoseFactory implements FirehoseFactory<ByteBufferInpu
return parseMessage(message);
}
public InputRow parseMessage(Message message) throws FormattedException
public InputRow parseMessage(Message message)
{
try {
return theParser.parse(message.payload());
}
catch (Exception e) {
throw new FormattedException.Builder()
.withErrorCode(FormattedException.ErrorCode.UNPARSABLE_ROW)
.withMessage(String.format("Error parsing[%s], got [%s]", message.payload(), e.toString()))
.build();
}
return theParser.parse(message.payload());
}
@Override
......
......@@ -23,14 +23,14 @@
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<packaging>pom</packaging>
<version>0.6.124-SNAPSHOT</version>
<version>0.6.129-SNAPSHOT</version>
<name>druid</name>
<description>druid</description>
<scm>
<connection>scm:git:ssh://git@github.com/metamx/druid.git</connection>
<developerConnection>scm:git:ssh://git@github.com/metamx/druid.git</developerConnection>
<url>http://www.github.com/metamx/druid</url>
<tag>druid-0.6.107-SNAPSHOT</tag>
<tag>druid-0.6.117-SNAPSHOT</tag>
</scm>
<prerequisites>
......@@ -39,9 +39,9 @@
<properties>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<metamx.java-util.version>0.25.6</metamx.java-util.version>
<apache.curator.version>2.4.0</apache.curator.version>
<druid.api.version>0.2.3</druid.api.version>
<metamx.java-util.version>0.26.5</metamx.java-util.version>
<apache.curator.version>2.5.0</apache.curator.version>
<druid.api.version>0.2.4</druid.api.version>
</properties>
<modules>
......@@ -198,22 +198,22 @@
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
<version>14.0.1</version>
<version>17.0</version>
</dependency>
<dependency>
<groupId>com.google.inject</groupId>
<artifactId>guice</artifactId>
<version>4.0-beta4</version>
<version>4.0-beta</version>
</dependency>
<dependency>
<groupId>com.google.inject.extensions</groupId>
<artifactId>guice-servlet</artifactId>
<version>4.0-beta4</version>
<version>4.0-beta</version>
</dependency>
<dependency>
<groupId>com.google.inject.extensions</groupId>
<artifactId>guice-multibindings</artifactId>
<version>4.0-beta4</version>
<version>4.0-beta</version>
</dependency>
<dependency>
<groupId>com.ibm.icu</groupId>
......@@ -561,15 +561,7 @@
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-release-plugin</artifactId>
<version>2.4.2</version>
<dependencies>
<dependency>
<groupId>org.apache.maven.scm</groupId>
<artifactId>maven-scm-provider-gitexe</artifactId>
<!-- This version is necessary for use with git version 1.8.5 and above -->
<version>1.8.1</version>
</dependency>
</dependencies>
<version>2.5</version>
</plugin>
</plugins>
</pluginManagement>
......
......@@ -28,7 +28,7 @@
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.6.124-SNAPSHOT</version>
<version>0.6.129-SNAPSHOT</version>
</parent>
<dependencies>
......
......@@ -28,10 +28,8 @@ import com.google.protobuf.ByteString;
import com.google.protobuf.Descriptors;
import com.google.protobuf.DynamicMessage;
import com.google.protobuf.InvalidProtocolBufferException;
import com.metamx.common.exception.FormattedException;
import com.metamx.common.logger.Logger;
import io.druid.data.input.impl.DimensionsSpec;
import io.druid.data.input.impl.InputRowParser;
import io.druid.data.input.impl.JSONParseSpec;
import io.druid.data.input.impl.MapInputRowParser;
import io.druid.data.input.impl.ParseSpec;
......@@ -94,7 +92,7 @@ public class ProtoBufInputRowParser implements ByteBufferInputRowParser
}
@Override
public InputRow parse(ByteBuffer input) throws FormattedException
public InputRow parse(ByteBuffer input)
{
// We should really create a ProtoBufBasedInputRow that does not need an intermediate map but accesses
// the DynamicMessage directly...
......
......@@ -17,7 +17,7 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.server.initialization;
package io.druid.guice;
import com.google.inject.Binder;
import com.google.inject.Module;
......
......@@ -17,7 +17,7 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.server.initialization;
package io.druid.guice;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableList;
......
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.guice;
import com.google.common.collect.Lists;
import com.google.inject.Binder;
import com.google.inject.Guice;
import com.google.inject.Injector;
import com.google.inject.Module;
import com.google.inject.util.Modules;
import io.druid.guice.ConfigModule;
import io.druid.guice.DruidGuiceExtensions;
import io.druid.guice.DruidSecondaryModule;
import io.druid.guice.ExtensionsConfig;
import io.druid.guice.JsonConfigProvider;
import io.druid.guice.PropertiesModule;
import io.druid.jackson.JacksonModule;
import java.util.List;
/**
*/
public class GuiceInjectors
{
public static Injector makeStartupInjector()
{
return Guice.createInjector(
new DruidGuiceExtensions(),
new JacksonModule(),
new PropertiesModule("runtime.properties"),
new ConfigModule(),
new Module()
{
@Override
public void configure(Binder binder)
{
binder.bind(DruidSecondaryModule.class);
JsonConfigProvider.bind(binder, "druid.extensions", ExtensionsConfig.class);
}
}
);
}
public static Injector makeStartupInjectorWithModules(Iterable<Module> modules)
{
List<Module> theModules = Lists.newArrayList();
theModules.add(new DruidGuiceExtensions());
theModules.add(new JacksonModule());
theModules.add(new PropertiesModule("runtime.properties"));
theModules.add(new ConfigModule());
theModules.add(
new Module()
{
@Override
public void configure(Binder binder)
{
binder.bind(DruidSecondaryModule.class);
JsonConfigProvider.bind(binder, "druid.extensions", ExtensionsConfig.class);
}
}
);
for (Module theModule : modules) {
theModules.add(theModule);
}
return Guice.createInjector(theModules);
}
}
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.guice;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Lists;
import com.google.inject.Injector;
import com.google.inject.Key;
import com.google.inject.Module;
import com.metamx.common.ISE;
import io.druid.guice.annotations.Json;
import io.druid.guice.annotations.Smile;
import io.druid.initialization.DruidModule;
import java.util.Collections;
import java.util.List;
/**
*/
public class ModuleList
{
private final Injector baseInjector;
private final ObjectMapper jsonMapper;
private final ObjectMapper smileMapper;
private final List<Module> modules;
public ModuleList(Injector baseInjector)
{
this.baseInjector = baseInjector;
this.jsonMapper = baseInjector.getInstance(Key.get(ObjectMapper.class, Json.class));
this.smileMapper = baseInjector.getInstance(Key.get(ObjectMapper.class, Smile.class));
this.modules = Lists.newArrayList();
}
public List<Module> getModules()
{
return Collections.unmodifiableList(modules);
}
public void addModule(Object input)
{
if (input instanceof DruidModule) {
baseInjector.injectMembers(input);
modules.add(registerJacksonModules(((DruidModule) input)));
} else if (input instanceof Module) {
baseInjector.injectMembers(input);
modules.add((Module) input);
} else if (input instanceof Class) {
if (DruidModule.class.isAssignableFrom((Class) input)) {
modules.add(registerJacksonModules(baseInjector.getInstance((Class<? extends DruidModule>) input)));
} else if (Module.class.isAssignableFrom((Class) input)) {
modules.add(baseInjector.getInstance((Class<? extends Module>) input));
return;
} else {
throw new ISE("Class[%s] does not implement %s", input.getClass(), Module.class);
}
} else {
throw new ISE("Unknown module type[%s]", input.getClass());
}
}
public void addModules(Object... object)
{
for (Object o : object) {
addModule(o);
}
}
private DruidModule registerJacksonModules(DruidModule module)
{
for (com.fasterxml.jackson.databind.Module jacksonModule : module.getJacksonModules()) {
jsonMapper.registerModule(jacksonModule);
smileMapper.registerModule(jacksonModule);
}
return module;
}
}
......@@ -17,13 +17,13 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.server.initialization;
package io.druid.guice;
import com.google.common.base.Charsets;
import com.google.common.base.Throwables;
import com.google.common.io.Closeables;
import com.google.inject.Binder;
import com.google.inject.Module;
import com.metamx.common.guava.CloseQuietly;
import com.metamx.common.logger.Logger;
import java.io.BufferedInputStream;
......@@ -80,7 +80,7 @@ public class PropertiesModule implements Module
log.wtf(e, "This can only happen if the .exists() call lied. That's f'd up.");
}
finally {
Closeables.closeQuietly(stream);
CloseQuietly.close(stream);
}
binder.bind(Properties.class).toInstance(props);
......
......@@ -17,14 +17,13 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.server;
package io.druid.query;
import com.metamx.common.concurrent.ExecutorServiceConfig;
import io.druid.segment.column.ColumnConfig;
import org.skife.config.Config;
/**
*/
public abstract class DruidProcessingConfig extends ExecutorServiceConfig
public abstract class DruidProcessingConfig extends ExecutorServiceConfig implements ColumnConfig
{
@Config({"druid.computation.buffer.size", "${base_path}.buffer.sizeBytes"})
public int intermediateComputeSizeBytes()
......@@ -39,4 +38,10 @@ public abstract class DruidProcessingConfig extends ExecutorServiceConfig
final int processors = Runtime.getRuntime().availableProcessors();
return processors > 1 ? processors - 1 : processors;
}
@Config(value = "${base_path}.columnCache.sizeBytes")
public int columnCacheSizeBytes()
{
return 1024 * 1024;
}
}
......@@ -19,7 +19,7 @@
package io.druid.query;
import com.google.common.io.Closeables;
import com.metamx.common.guava.CloseQuietly;
import com.metamx.common.guava.ResourceClosingSequence;
import com.metamx.common.guava.Sequence;
import io.druid.segment.ReferenceCountingSegment;
......@@ -52,7 +52,7 @@ public class ReferenceCountingSegmentQueryRunner<T> implements QueryRunner<T>
return new ResourceClosingSequence<T>(baseSequence, closeable);
}
catch (RuntimeException e) {
Closeables.closeQuietly(closeable);
CloseQuietly.close(closeable);
throw e;
}
}
......
......@@ -54,7 +54,7 @@ public class CardinalityAggregator implements Aggregator
// nothing to add to hasher if size == 0, only handle size == 1 and size != 0 cases.
if (size == 1) {
final String value = selector.lookupName(row.get(0));
hasher.putString(value != null ? value : NULL_STRING);
hasher.putUnencodedChars(value != null ? value : NULL_STRING);
} else if (size != 0) {
final String[] values = new String[size];
for (int i = 0; i < size; ++i) {
......@@ -67,7 +67,7 @@ public class CardinalityAggregator implements Aggregator
if (i != 0) {
hasher.putChar(SEPARATOR);
}
hasher.putString(values[i]);
hasher.putUnencodedChars(values[i]);
}
}
}
......@@ -79,7 +79,7 @@ public class CardinalityAggregator implements Aggregator
for (final DimensionSelector selector : selectors) {
for (final Integer index : selector.getRow()) {
final String value = selector.lookupName(index);
collector.add(hashFn.hashString(value == null ? NULL_STRING : value).asBytes());
collector.add(hashFn.hashUnencodedChars(value == null ? NULL_STRING : value).asBytes());
}
}
}
......
......@@ -128,8 +128,9 @@ public class HyperUniquesSerde extends ComplexMetricSerde
@Override
public HyperLogLogCollector fromByteBuffer(ByteBuffer buffer, int numBytes)
{
buffer.limit(buffer.position() + numBytes);
return HyperLogLogCollector.makeCollector(buffer);
final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer();
readOnlyBuffer.limit(readOnlyBuffer.position() + numBytes);
return HyperLogLogCollector.makeCollector(readOnlyBuffer);
}
@Override
......
......@@ -98,4 +98,12 @@ public class JavascriptDimExtractionFn implements DimExtractionFn
{
return false;
}
@Override
public String toString()
{
return "JavascriptDimExtractionFn{" +
"function='" + function + '\'' +
'}';
}
}
......@@ -25,12 +25,12 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterators;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.io.Closeables;
import com.google.common.primitives.Ints;
import com.google.inject.Inject;
import com.metamx.common.IAE;
import com.metamx.common.ISE;
import com.metamx.common.guava.BaseSequence;
import com.metamx.common.guava.CloseQuietly;
import com.metamx.common.guava.FunctionalIterator;
import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences;
......@@ -123,7 +123,7 @@ public class GroupByQueryEngine
@Override
public void cleanup(RowIterator iterFromMake)
{
Closeables.closeQuietly(iterFromMake);
CloseQuietly.close(iterFromMake);
}
}
);
......@@ -135,7 +135,7 @@ public class GroupByQueryEngine
@Override
public void close() throws IOException
{
Closeables.closeQuietly(bufferHolder);
CloseQuietly.close(bufferHolder);
}
}
)
......
......@@ -52,24 +52,6 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
private static final byte CACHE_TYPE_ID = 0x0;
public static Iterable<Result<TimeBoundaryResultValue>> buildResult(DateTime timestamp, DateTime min, DateTime max)
{
List<Result<TimeBoundaryResultValue>> results = Lists.newArrayList();
Map<String, Object> result = Maps.newHashMap();
if (min != null) {
result.put(MIN_TIME, min);
}
if (max != null) {
result.put(MAX_TIME, max);
}
if (!result.isEmpty()) {
results.add(new Result<>(timestamp, new TimeBoundaryResultValue(result)));
}
return results;
}
private final String bound;
@JsonCreator
......@@ -150,6 +132,24 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
.array();
}
public Iterable<Result<TimeBoundaryResultValue>> buildResult(DateTime timestamp, DateTime min, DateTime max)
{
List<Result<TimeBoundaryResultValue>> results = Lists.newArrayList();
Map<String, Object> result = Maps.newHashMap();
if (min != null) {
result.put(MIN_TIME, min);
}
if (max != null) {
result.put(MAX_TIME, max);
}
if (!result.isEmpty()) {
results.add(new Result<>(timestamp, new TimeBoundaryResultValue(result)));
}
return results;
}
public Iterable<Result<TimeBoundaryResultValue>> mergeResults(List<Result<TimeBoundaryResultValue>> results)
{
if (results == null || results.isEmpty()) {
......
......@@ -19,8 +19,8 @@
package io.druid.query.topn;
import com.google.common.io.Closeables;
import com.metamx.common.Pair;
import com.metamx.common.guava.CloseQuietly;
import io.druid.collections.ResourceHolder;
import io.druid.collections.StupidPool;
import io.druid.query.aggregation.BufferAggregator;
......@@ -233,7 +233,7 @@ public class PooledTopNAlgorithm
if (resultsBufHolder != null) {
resultsBufHolder.get().clear();
}
Closeables.closeQuietly(resultsBufHolder);
CloseQuietly.close(resultsBufHolder);
}
public static class PooledTopNParams extends TopNParams
......
......@@ -19,8 +19,8 @@
package io.druid.segment;
import com.google.common.io.Closeables;
import com.metamx.collections.spatial.ImmutableRTree;
import com.metamx.common.guava.CloseQuietly;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.segment.column.Column;
import io.druid.segment.column.DictionaryEncodedColumn;
......@@ -95,7 +95,7 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
return column.length();
}
finally {
Closeables.closeQuietly(column);
CloseQuietly.close(column);
}
}
......
......@@ -22,6 +22,7 @@ package io.druid.segment;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
......@@ -31,6 +32,9 @@ import com.google.common.io.ByteStreams;
import com.google.common.io.Closeables;
import com.google.common.io.Files;
import com.google.common.primitives.Ints;
import com.google.inject.Binder;
import com.google.inject.Injector;
import com.google.inject.Module;
import com.metamx.collections.spatial.ImmutableRTree;
import com.metamx.common.IAE;
import com.metamx.common.ISE;
......@@ -41,9 +45,13 @@ import com.metamx.common.io.smoosh.SmooshedWriter;
import com.metamx.common.logger.Logger;
import com.metamx.emitter.EmittingLogger;
import io.druid.common.utils.SerializerUtils;
import io.druid.guice.ConfigProvider;
import io.druid.guice.GuiceInjectors;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.DruidProcessingConfig;
import io.druid.segment.column.Column;
import io.druid.segment.column.ColumnBuilder;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.column.ColumnDescriptor;
import io.druid.segment.column.ValueType;
import io.druid.segment.data.ArrayIndexed;
......@@ -90,6 +98,9 @@ public class IndexIO
{
public static final byte V8_VERSION = 0x8;
public static final byte V9_VERSION = 0x9;
public static final int CURRENT_VERSION_ID = V9_VERSION;
public static final ByteOrder BYTE_ORDER = ByteOrder.nativeOrder();
private static final Map<Integer, IndexLoader> indexLoaders =
ImmutableMap.<Integer, IndexLoader>builder()
......@@ -107,13 +118,33 @@ public class IndexIO
private static final EmittingLogger log = new EmittingLogger(IndexIO.class);
private static final SerializerUtils serializerUtils = new SerializerUtils();
public static final ByteOrder BYTE_ORDER = ByteOrder.nativeOrder();
// This should really be provided by DI, should be changed once we switch around to using a DI framework
private static final ObjectMapper mapper = new DefaultObjectMapper();
private static final ObjectMapper mapper;
protected static final ColumnConfig columnConfig;
static {
final Injector injector = GuiceInjectors.makeStartupInjectorWithModules(
ImmutableList.<Module>of(
new Module()
{
@Override
public void configure(Binder binder)
{
ConfigProvider.bind(
binder,
DruidProcessingConfig.class,
ImmutableMap.of("base_path", "druid.processing")
);
binder.bind(ColumnConfig.class).to(DruidProcessingConfig.class);
}
}
)
);
mapper = injector.getInstance(ObjectMapper.class);
columnConfig = injector.getInstance(ColumnConfig.class);
}
private static volatile IndexIOHandler handler = null;
public static final int CURRENT_VERSION_ID = V9_VERSION;
@Deprecated
public static MMappedIndex mapDir(final File inDir) throws IOException
......@@ -165,15 +196,10 @@ public class IndexIO
}
final File indexFile = new File(inDir, "index.drd");
InputStream in = null;
int version;
try {
in = new FileInputStream(indexFile);
try (InputStream in = new FileInputStream(indexFile)) {
version = in.read();
}
finally {
Closeables.closeQuietly(in);
}
return version;
}
......@@ -194,8 +220,8 @@ public class IndexIO
case 2:
case 3:
log.makeAlert("Attempt to load segment of version <= 3.")
.addData("version", version)
.emit();
.addData("version", version)
.emit();
return false;
case 4:
case 5:
......@@ -631,7 +657,10 @@ public class IndexIO
.setHasMultipleValues(true)
.setDictionaryEncodedColumn(
new DictionaryEncodedColumnSupplier(
index.getDimValueLookup(dimension), null, (index.getDimColumn(dimension))
index.getDimValueLookup(dimension),
null,
index.getDimColumn(dimension),
columnConfig.columnCacheSizeBytes()
)
)
.setBitmapIndex(
......@@ -743,7 +772,7 @@ public class IndexIO
ColumnDescriptor serde = mapper.readValue(
serializerUtils.readString(byteBuffer), ColumnDescriptor.class
);
return serde.read(byteBuffer);
return serde.read(byteBuffer, columnConfig);
}
}
......
......@@ -38,6 +38,7 @@ import com.metamx.collections.spatial.RTree;
import com.metamx.collections.spatial.split.LinearGutmanSplitStrategy;
import com.metamx.common.IAE;
import com.metamx.common.ISE;
import com.metamx.common.guava.CloseQuietly;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.common.guava.MergeIterable;
import com.metamx.common.guava.nary.BinaryFn;
......@@ -50,6 +51,7 @@ import io.druid.common.utils.JodaUtils;
import io.druid.common.utils.SerializerUtils;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.ToLowerCaseAggregatorFactory;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.data.ByteBufferWriter;
import io.druid.segment.data.CompressedLongsSupplierSerializer;
import io.druid.segment.data.ConciseCompressedIndexedInts;
......@@ -432,9 +434,9 @@ public class IndexMerger
serializerUtils.writeString(channel, String.format("%s/%s", minTime, maxTime));
}
finally {
Closeables.closeQuietly(channel);
CloseQuietly.close(channel);
channel = null;
Closeables.closeQuietly(fileOutputStream);
CloseQuietly.close(fileOutputStream);
fileOutputStream = null;
}
IndexIO.checkFileSize(indexFile);
......@@ -875,7 +877,7 @@ public class IndexMerger
);
}
finally {
Closeables.closeQuietly(channel);
CloseQuietly.close(channel);
channel = null;
}
IndexIO.checkFileSize(indexFile);
......
......@@ -20,7 +20,7 @@
package io.druid.segment;
import com.google.common.collect.Maps;
import com.google.common.io.Closeables;
import com.metamx.common.guava.CloseQuietly;
import io.druid.segment.data.ConciseCompressedIndexedInts;
import io.druid.segment.data.Indexed;
import io.druid.segment.data.IndexedFloats;
......@@ -118,9 +118,9 @@ public class MMappedIndexAdapter implements IndexableAdapter
{
final boolean hasNext = currRow < numRows;
if (!hasNext && !done) {
Closeables.closeQuietly(timestamps);
CloseQuietly.close(timestamps);
for (IndexedFloats floatMetric : floatMetrics) {
Closeables.closeQuietly(floatMetric);
CloseQuietly.close(floatMetric);
}
done = true;
}
......
......@@ -20,11 +20,11 @@
package io.druid.segment;
import com.google.common.io.ByteStreams;
import com.google.common.io.Closeables;
import com.google.common.io.InputSupplier;
import com.google.common.io.OutputSupplier;
import com.metamx.common.IAE;
import com.metamx.common.ISE;
import com.metamx.common.guava.CloseQuietly;
import io.druid.common.utils.SerializerUtils;
import io.druid.segment.data.CompressedFloatsIndexedSupplier;
import io.druid.segment.data.CompressedFloatsSupplierSerializer;
......@@ -84,8 +84,8 @@ public class MetricHolder
ByteStreams.copy(in, out);
}
finally {
Closeables.closeQuietly(out);
Closeables.closeQuietly(in);
CloseQuietly.close(out);
CloseQuietly.close(in);
}
}
......
......@@ -22,8 +22,8 @@ package io.druid.segment;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.google.common.io.Closeables;
import com.metamx.common.ISE;
import com.metamx.common.guava.CloseQuietly;
import com.metamx.common.logger.Logger;
import io.druid.segment.column.BitmapIndex;
import io.druid.segment.column.Column;
......@@ -208,10 +208,10 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
{
final boolean hasNext = currRow < numRows;
if (!hasNext && !done) {
Closeables.closeQuietly(timestamps);
CloseQuietly.close(timestamps);
for (Object metric : metrics) {
if (metric instanceof Closeable) {
Closeables.closeQuietly((Closeable) metric);
CloseQuietly.close((Closeable) metric);
}
}
done = true;
......
......@@ -21,9 +21,10 @@ package io.druid.segment;
import com.google.common.base.Function;
import com.google.common.base.Predicates;
import com.google.common.collect.Iterators;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.google.common.io.Closeables;
import com.metamx.common.guava.CloseQuietly;
import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences;
import io.druid.granularity.QueryGranularity;
......@@ -38,12 +39,12 @@ import io.druid.segment.column.ValueType;
import io.druid.segment.data.Indexed;
import io.druid.segment.data.IndexedInts;
import io.druid.segment.data.Offset;
import io.druid.segment.data.SingleIndexedInts;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import java.io.Closeable;
import java.io.IOException;
import java.util.Iterator;
import java.util.Map;
/**
......@@ -109,7 +110,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
return new DateTime(column.getLongSingleValueRow(0));
}
finally {
Closeables.closeQuietly(column);
CloseQuietly.close(column);
}
}
......@@ -122,7 +123,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
return new DateTime(column.getLongSingleValueRow(column.length() - 1));
}
finally {
Closeables.closeQuietly(column);
CloseQuietly.close(column);
}
}
......@@ -186,6 +187,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
{
final Offset baseOffset = offset.clone();
final Map<String, DictionaryEncodedColumn> dictionaryColumnCache = Maps.newHashMap();
final Map<String, GenericColumn> genericColumnCache = Maps.newHashMap();
final Map<String, ComplexColumn> complexColumnCache = Maps.newHashMap();
final Map<String, Object> objectColumnCache = Maps.newHashMap();
......@@ -270,12 +272,16 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
public DimensionSelector makeDimensionSelector(String dimension)
{
final String dimensionName = dimension.toLowerCase();
DictionaryEncodedColumn cachedColumn = dictionaryColumnCache.get(dimensionName);
final Column columnDesc = index.getColumn(dimensionName);
if (columnDesc == null) {
return null;
if (cachedColumn == null && columnDesc != null) {
cachedColumn = columnDesc.getDictionaryEncoding();
dictionaryColumnCache.put(dimensionName, cachedColumn);
}
final DictionaryEncodedColumn column = columnDesc.getDictionaryEncoding();
final DictionaryEncodedColumn column = cachedColumn;
if (column == null) {
return null;
......@@ -313,7 +319,27 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
@Override
public IndexedInts getRow()
{
return new SingleIndexedInts(column.getSingleValueRow(cursorOffset.getOffset()));
// using an anonymous class is faster than creating a class that stores a copy of the value
return new IndexedInts()
{
@Override
public int size()
{
return 1;
}
@Override
public int get(int index)
{
return column.getSingleValueRow(cursorOffset.getOffset());
}
@Override
public Iterator<Integer> iterator()
{
return Iterators.singletonIterator(column.getSingleValueRow(cursorOffset.getOffset()));
}
};
}
@Override
......@@ -535,16 +561,19 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
@Override
public void close() throws IOException
{
Closeables.closeQuietly(timestamps);
CloseQuietly.close(timestamps);
for (DictionaryEncodedColumn column : dictionaryColumnCache.values()) {
CloseQuietly.close(column);
}
for (GenericColumn column : genericColumnCache.values()) {
Closeables.closeQuietly(column);
CloseQuietly.close(column);
}
for (ComplexColumn complexColumn : complexColumnCache.values()) {
Closeables.closeQuietly(complexColumn);
CloseQuietly.close(complexColumn);
}
for (Object column : objectColumnCache.values()) {
if(column instanceof Closeable) {
Closeables.closeQuietly((Closeable) column);
CloseQuietly.close((Closeable) column);
}
}
}
......@@ -620,6 +649,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
*/
public Sequence<Cursor> build()
{
final Map<String, DictionaryEncodedColumn> dictionaryColumnCache = Maps.newHashMap();
final Map<String, GenericColumn> genericColumnCache = Maps.newHashMap();
final Map<String, ComplexColumn> complexColumnCache = Maps.newHashMap();
final Map<String, Object> objectColumnCache = Maps.newHashMap();
......@@ -697,41 +727,45 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
public DimensionSelector makeDimensionSelector(String dimension)
{
final String dimensionName = dimension.toLowerCase();
final Column column = index.getColumn(dimensionName);
if (column == null) {
return null;
DictionaryEncodedColumn cachedColumn = dictionaryColumnCache.get(dimensionName);
final Column columnDesc = index.getColumn(dimensionName);
if (cachedColumn == null && columnDesc != null) {
cachedColumn = columnDesc.getDictionaryEncoding();
dictionaryColumnCache.put(dimensionName, cachedColumn);
}
final DictionaryEncodedColumn dict = column.getDictionaryEncoding();
final DictionaryEncodedColumn column = cachedColumn;
if (dict == null) {
if (column == null) {
return null;
} else if (column.getCapabilities().hasMultipleValues()) {
} else if (columnDesc.getCapabilities().hasMultipleValues()) {
return new DimensionSelector()
{
@Override
public IndexedInts getRow()
{
return dict.getMultiValueRow(currRow);
return column.getMultiValueRow(currRow);
}
@Override
public int getValueCardinality()
{
return dict.getCardinality();
return column.getCardinality();
}
@Override
public String lookupName(int id)
{
final String retVal = dict.lookupName(id);
final String retVal = column.lookupName(id);
return retVal == null ? "" : retVal;
}
@Override
public int lookupId(String name)
{
return dict.lookupId(name);
return column.lookupId(name);
}
};
} else {
......@@ -740,25 +774,45 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
@Override
public IndexedInts getRow()
{
return new SingleIndexedInts(dict.getSingleValueRow(currRow));
// using an anonymous class is faster than creating a class that stores a copy of the value
return new IndexedInts()
{
@Override
public int size()
{
return 1;
}
@Override
public int get(int index)
{
return column.getSingleValueRow(currRow);
}
@Override
public Iterator<Integer> iterator()
{
return Iterators.singletonIterator(column.getSingleValueRow(currRow));
}
};
}
@Override
public int getValueCardinality()
{
return dict.getCardinality();
return column.getCardinality();
}
@Override
public String lookupName(int id)
{
return dict.lookupName(id);
return column.lookupName(id);
}
@Override
public int lookupId(String name)
{
return dict.lookupId(name);
return column.lookupId(name);
}
};
}
......@@ -962,16 +1016,19 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
@Override
public void close() throws IOException
{
Closeables.closeQuietly(timestamps);
CloseQuietly.close(timestamps);
for (DictionaryEncodedColumn column : dictionaryColumnCache.values()) {
CloseQuietly.close(column);
}
for (GenericColumn column : genericColumnCache.values()) {
Closeables.closeQuietly(column);
CloseQuietly.close(column);
}
for (ComplexColumn complexColumn : complexColumnCache.values()) {
Closeables.closeQuietly(complexColumn);
CloseQuietly.close(complexColumn);
}
for (Object column : objectColumnCache.values()) {
if (column instanceof Closeable) {
Closeables.closeQuietly((Closeable) column);
CloseQuietly.close((Closeable) column);
}
}
}
......@@ -979,31 +1036,4 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
);
}
}
private static class NullDimensionSelector implements DimensionSelector
{
@Override
public IndexedInts getRow()
{
return new SingleIndexedInts(0);
}
@Override
public int getValueCardinality()
{
return 1;
}
@Override
public String lookupName(int id)
{
return "";
}
@Override
public int lookupId(String name)
{
return 0;
}
}
}
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.segment.column;
public interface ColumnConfig
{
public int columnCacheSizeBytes();
}
......@@ -92,14 +92,14 @@ public class ColumnDescriptor
}
}
public Column read(ByteBuffer buffer)
public Column read(ByteBuffer buffer, ColumnConfig columnConfig)
{
final ColumnBuilder builder = new ColumnBuilder()
.setType(valueType)
.setHasMultipleValues(hasMultipleValues);
for (ColumnPartSerde part : parts) {
part.read(buffer, builder);
part.read(buffer, builder, columnConfig);
}
return builder.build();
......
......@@ -21,9 +21,11 @@ package io.druid.segment.column;
import io.druid.segment.data.IndexedInts;
import java.io.Closeable;
/**
*/
public interface DictionaryEncodedColumn
public interface DictionaryEncodedColumn extends Closeable
{
public int length();
public boolean hasMultipleValues();
......
......@@ -20,7 +20,7 @@
package io.druid.segment.column;
import com.google.common.base.Supplier;
import com.google.common.io.Closeables;
import com.metamx.common.guava.CloseQuietly;
/**
*/
......@@ -68,7 +68,7 @@ class SimpleColumn implements Column
return column.length();
}
finally {
Closeables.closeQuietly(column);
CloseQuietly.close(column);
}
}
......
......@@ -24,6 +24,8 @@ import io.druid.segment.data.IndexedInts;
import io.druid.segment.data.VSizeIndexed;
import io.druid.segment.data.VSizeIndexedInts;
import java.io.IOException;
/**
*/
public class SimpleDictionaryEncodedColumn implements DictionaryEncodedColumn
......@@ -84,4 +86,10 @@ public class SimpleDictionaryEncodedColumn implements DictionaryEncodedColumn
{
return lookups.size();
}
@Override
public void close() throws IOException
{
lookups.close();
}
}
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
* Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
......@@ -19,35 +19,8 @@
package io.druid.segment.data;
import com.google.common.collect.Iterators;
import java.util.Iterator;
/**
*/
public class SingleIndexedInts implements IndexedInts
{
private final int value;
public SingleIndexedInts(int value) {
this.value = value;
}
@Override
public int size()
{
return 1;
}
@Override
public int get(int index)
{
return value;
}
@Override
public Iterator<Integer> iterator()
{
return Iterators.singletonIterator(value);
}
}
* Implementing CacheableObjectStrategy instead of ObjectSrategy indicates
* that a column scan may cache the results of fromByteBuffer
*/
public interface CacheableObjectStrategy<T> extends ObjectStrategy<T> {}
......@@ -25,6 +25,7 @@ import com.google.common.io.Closeables;
import com.google.common.primitives.Floats;
import com.google.common.primitives.Ints;
import com.metamx.common.IAE;
import com.metamx.common.guava.CloseQuietly;
import io.druid.collections.ResourceHolder;
import io.druid.collections.StupidResourceHolder;
......@@ -123,7 +124,7 @@ public class CompressedFloatsIndexedSupplier implements Supplier<IndexedFloats>
private void loadBuffer(int bufferNum)
{
Closeables.closeQuietly(holder);
CloseQuietly.close(holder);
holder = baseFloatBuffers.get(bufferNum);
buffer = holder.get();
currIndex = bufferNum;
......
......@@ -106,17 +106,11 @@ public class CompressedFloatsSupplierSerializer
flattener.close();
OutputStream out = null;
try {
out = consolidatedOut.getOutput();
try (OutputStream out = consolidatedOut.getOutput()) {
out.write(CompressedFloatsIndexedSupplier.version);
out.write(Ints.toByteArray(numInserted));
out.write(Ints.toByteArray(sizePer));
ByteStreams.copy(flattener.combineStreams(), out);
}
finally {
Closeables.closeQuietly(out);
}
}
}
......@@ -25,6 +25,7 @@ import com.google.common.io.Closeables;
import com.google.common.primitives.Ints;
import com.google.common.primitives.Longs;
import com.metamx.common.IAE;
import com.metamx.common.guava.CloseQuietly;
import io.druid.collections.ResourceHolder;
import io.druid.collections.StupidResourceHolder;
......@@ -122,7 +123,7 @@ public class CompressedLongsIndexedSupplier implements Supplier<IndexedLongs>
private void loadBuffer(int bufferNum)
{
Closeables.closeQuietly(holder);
CloseQuietly.close(holder);
holder = baseLongBuffers.get(bufferNum);
buffer = holder.get();
currIndex = bufferNum;
......
......@@ -100,17 +100,11 @@ public class CompressedLongsSupplierSerializer
flattener.close();
OutputStream out = null;
try {
out = consolidatedOut.getOutput();
try (OutputStream out = consolidatedOut.getOutput()) {
out.write(CompressedLongsIndexedSupplier.version);
out.write(Ints.toByteArray(numInserted));
out.write(Ints.toByteArray(sizePer));
ByteStreams.copy(flattener.combineStreams(), out);
}
finally {
Closeables.closeQuietly(out);
}
}
}
......@@ -20,7 +20,7 @@
package io.druid.segment.data;
import com.google.common.base.Throwables;
import com.google.common.io.Closeables;
import com.metamx.common.guava.CloseQuietly;
import com.ning.compress.lzf.ChunkEncoder;
import com.ning.compress.lzf.LZFChunk;
import com.ning.compress.lzf.LZFDecoder;
......@@ -74,7 +74,7 @@ public class CompressedObjectStrategy<T extends Buffer> implements ObjectStrateg
buf.put(outputBytes, 0, numDecompressedBytes);
buf.flip();
Closeables.closeQuietly(outputBytesHolder);
CloseQuietly.close(outputBytesHolder);
return new ResourceHolder<T>()
{
......@@ -105,7 +105,7 @@ public class CompressedObjectStrategy<T extends Buffer> implements ObjectStrateg
final ResourceHolder<ChunkEncoder> encoder = CompressedPools.getChunkEncoder();
LZFChunk chunk = encoder.get().encodeChunk(buf.array(), 0, buf.array().length);
Closeables.closeQuietly(encoder);
CloseQuietly.close(encoder);
return chunk.getData();
}
......
......@@ -123,8 +123,9 @@ public class ConciseCompressedIndexedInts implements IndexedInts, Comparable<Con
@Override
public ImmutableConciseSet fromByteBuffer(ByteBuffer buffer, int numBytes)
{
buffer.limit(buffer.position() + numBytes);
return new ImmutableConciseSet(buffer.asReadOnlyBuffer());
final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer();
readOnlyBuffer.limit(readOnlyBuffer.position() + numBytes);
return new ImmutableConciseSet(readOnlyBuffer);
}
@Override
......
......@@ -20,10 +20,11 @@
package io.druid.segment.data;
import com.google.common.base.Charsets;
import com.google.common.collect.Maps;
import com.google.common.collect.Ordering;
import com.google.common.io.Closeables;
import com.google.common.primitives.Ints;
import com.metamx.common.IAE;
import com.metamx.common.guava.CloseQuietly;
import java.io.ByteArrayOutputStream;
import java.io.Closeable;
......@@ -32,6 +33,8 @@ import java.nio.ByteBuffer;
import java.nio.channels.WritableByteChannel;
import java.util.Arrays;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.Map;
/**
* A generic, flat storage mechanism. Use static methods fromArray() or fromIterable() to construct. If input
......@@ -46,10 +49,14 @@ import java.util.Iterator;
* bytes 10-((numElements * 4) + 10): integers representing *end* offsets of byte serialized values
* bytes ((numElements * 4) + 10)-(numBytesUsed + 2): 4-byte integer representing length of value, followed by bytes for value
*/
public class GenericIndexed<T> implements Indexed<T>
public class GenericIndexed<T> implements Indexed<T>, Closeable
{
private static final byte version = 0x1;
public static final int INITIAL_CACHE_CAPACITY = 16384;
private int indexOffset;
public static <T> GenericIndexed<T> fromArray(T[] objects, ObjectStrategy<T> strategy)
{
return fromIterable(Arrays.asList(objects), strategy);
......@@ -73,14 +80,14 @@ public class GenericIndexed<T> implements Indexed<T>
allowReverseLookup = false;
}
if (prevVal instanceof Closeable) {
Closeables.closeQuietly((Closeable) prevVal);
CloseQuietly.close((Closeable) prevVal);
}
prevVal = next;
++count;
}
if (prevVal instanceof Closeable) {
Closeables.closeQuietly((Closeable) prevVal);
CloseQuietly.close((Closeable) prevVal);
}
ByteArrayOutputStream headerBytes = new ByteArrayOutputStream(4 + (count * 4));
......@@ -98,7 +105,7 @@ public class GenericIndexed<T> implements Indexed<T>
valueBytes.write(bytes);
if (object instanceof Closeable) {
Closeables.closeQuietly((Closeable) object);
CloseQuietly.close((Closeable) object);
}
}
}
......@@ -114,11 +121,44 @@ public class GenericIndexed<T> implements Indexed<T>
return new GenericIndexed<T>(theBuffer.asReadOnlyBuffer(), strategy, allowReverseLookup);
}
private static class SizedLRUMap<K, V> extends LinkedHashMap<K, V>
{
final Map<K, Integer> sizes = Maps.newHashMap();
int numBytes = 0;
int maxBytes = 0;
public SizedLRUMap(int initialCapacity, int maxBytes)
{
super(initialCapacity, 0.75f, true);
this.maxBytes = maxBytes;
}
@Override
protected boolean removeEldestEntry(Map.Entry<K, V> eldest)
{
if (numBytes > maxBytes) {
numBytes -= sizes.remove(eldest.getKey());
return true;
}
return false;
}
public V put(K key, V value, int size)
{
numBytes += size;
sizes.put(key, size);
return super.put(key, value);
}
}
private final ByteBuffer theBuffer;
private final ObjectStrategy<T> strategy;
private final boolean allowReverseLookup;
private final int size;
private final boolean cacheable;
private final ThreadLocal<ByteBuffer> cachedBuffer;
private final ThreadLocal<SizedLRUMap<Integer, T>> cachedValues;
private final int valuesOffset;
GenericIndexed(
......@@ -132,7 +172,45 @@ public class GenericIndexed<T> implements Indexed<T>
this.allowReverseLookup = allowReverseLookup;
size = theBuffer.getInt();
indexOffset = theBuffer.position();
valuesOffset = theBuffer.position() + (size << 2);
this.cachedBuffer = new ThreadLocal<ByteBuffer>()
{
@Override
protected ByteBuffer initialValue()
{
return theBuffer.asReadOnlyBuffer();
}
};
this.cacheable = false;
this.cachedValues = new ThreadLocal<>();
}
/**
* Creates a copy of the given indexed with the given cache size
* The resulting copy must be closed to release resources used by the cache
*/
GenericIndexed(GenericIndexed<T> other, final int maxBytes)
{
this.theBuffer = other.theBuffer;
this.strategy = other.strategy;
this.allowReverseLookup = other.allowReverseLookup;
this.size = other.size;
this.indexOffset = other.indexOffset;
this.valuesOffset = other.valuesOffset;
this.cachedBuffer = other.cachedBuffer;
this.cachedValues = new ThreadLocal<SizedLRUMap<Integer, T>>() {
@Override
protected SizedLRUMap<Integer, T> initialValue()
{
return new SizedLRUMap<>(INITIAL_CACHE_CAPACITY, maxBytes);
}
};
this.cacheable = strategy instanceof CacheableObjectStrategy;
}
@Override
......@@ -157,24 +235,41 @@ public class GenericIndexed<T> implements Indexed<T>
throw new IAE(String.format("Index[%s] >= size[%s]", index, size));
}
ByteBuffer myBuffer = theBuffer.asReadOnlyBuffer();
int startOffset = 4;
int endOffset;
if(cacheable) {
final T cached = cachedValues.get().get(index);
if (cached != null) {
return cached;
}
}
// using a cached copy of the buffer instead of making a read-only copy every time get() is called is faster
final ByteBuffer copyBuffer = this.cachedBuffer.get();
final int startOffset;
final int endOffset;
if (index == 0) {
endOffset = myBuffer.getInt();
startOffset = 4;
endOffset = copyBuffer.getInt(indexOffset);
} else {
myBuffer.position(myBuffer.position() + ((index - 1) * 4));
startOffset = myBuffer.getInt() + 4;
endOffset = myBuffer.getInt();
copyBuffer.position(indexOffset + ((index - 1) * 4));
startOffset = copyBuffer.getInt() + 4;
endOffset = copyBuffer.getInt();
}
if (startOffset == endOffset) {
return null;
}
myBuffer.position(valuesOffset + startOffset);
return strategy.fromByteBuffer(myBuffer, endOffset - startOffset);
copyBuffer.position(valuesOffset + startOffset);
final int size = endOffset - startOffset;
// fromByteBuffer must not modify the buffer limit
final T value = strategy.fromByteBuffer(copyBuffer, size);
if(cacheable) {
cachedValues.get().put(index, value, size);
}
return value;
}
@Override
......@@ -220,6 +315,25 @@ public class GenericIndexed<T> implements Indexed<T>
channel.write(theBuffer.asReadOnlyBuffer());
}
/**
* The returned GenericIndexed must be closed to release the underlying memory
* @param maxBytes
* @return
*/
public GenericIndexed<T> withCache(int maxBytes)
{
return new GenericIndexed<>(this, maxBytes);
}
@Override
public void close() throws IOException
{
if(cacheable) {
cachedValues.get().clear();
cachedValues.remove();
}
}
public static <T> GenericIndexed<T> read(ByteBuffer buffer, ObjectStrategy<T> strategy)
{
byte versionFromBuffer = buffer.get();
......@@ -241,7 +355,7 @@ public class GenericIndexed<T> implements Indexed<T>
throw new IAE("Unknown version[%s]", versionFromBuffer);
}
public static ObjectStrategy<String> stringStrategy = new ObjectStrategy<String>()
public static ObjectStrategy<String> stringStrategy = new CacheableObjectStrategy<String>()
{
@Override
public Class<? extends String> getClazz()
......@@ -250,9 +364,9 @@ public class GenericIndexed<T> implements Indexed<T>
}
@Override
public String fromByteBuffer(ByteBuffer buffer, int numBytes)
public String fromByteBuffer(final ByteBuffer buffer, final int numBytes)
{
byte[] bytes = new byte[numBytes];
final byte[] bytes = new byte[numBytes];
buffer.get(bytes);
return new String(bytes, Charsets.UTF_8);
}
......
......@@ -22,7 +22,7 @@ package io.druid.segment.data;
import com.google.common.base.Function;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.io.Closeables;
import com.metamx.common.guava.CloseQuietly;
import io.druid.collections.ResourceHolder;
import io.druid.collections.StupidResourceHolder;
......@@ -153,7 +153,7 @@ public class InMemoryCompressedFloats implements IndexedFloats
private void loadBuffer(int bufferNum)
{
loadBuffer = null;
Closeables.closeQuietly(holder);
CloseQuietly.close(holder);
final byte[] compressedBytes = compressedBuffers.get(bufferNum);
holder = strategy.fromByteBuffer(ByteBuffer.wrap(compressedBytes), compressedBytes.length);
loadBuffer = holder.get();
......@@ -191,6 +191,6 @@ public class InMemoryCompressedFloats implements IndexedFloats
@Override
public void close() throws IOException
{
Closeables.closeQuietly(holder);
CloseQuietly.close(holder);
}
}
......@@ -23,6 +23,7 @@ import com.google.common.base.Function;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.io.Closeables;
import com.metamx.common.guava.CloseQuietly;
import io.druid.collections.ResourceHolder;
import io.druid.collections.StupidResourceHolder;
......@@ -163,7 +164,7 @@ public class InMemoryCompressedLongs implements IndexedLongs
private void loadBuffer(int bufferNum)
{
loadBuffer = null;
Closeables.closeQuietly(holder);
CloseQuietly.close(holder);
final byte[] compressedBytes = compressedBuffers.get(bufferNum);
holder = strategy.fromByteBuffer(ByteBuffer.wrap(compressedBytes), compressedBytes.length);
loadBuffer = holder.get();
......
......@@ -81,8 +81,10 @@ public class IndexedRTree implements Comparable<IndexedRTree>
@Override
public ImmutableRTree fromByteBuffer(ByteBuffer buffer, int numBytes)
{
buffer.limit(buffer.position() + numBytes);
return new ImmutableRTree(buffer.asReadOnlyBuffer());
final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer();
readOnlyBuffer.limit(readOnlyBuffer.position() + numBytes);
return new ImmutableRTree(readOnlyBuffer);
}
@Override
......
......@@ -101,8 +101,9 @@ public class IntBufferIndexedInts implements IndexedInts, Comparable<IntBufferIn
@Override
public IntBufferIndexedInts fromByteBuffer(ByteBuffer buffer, int numBytes)
{
buffer.limit(buffer.position() + numBytes);
return new IntBufferIndexedInts(buffer);
final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer();
readOnlyBuffer.limit(readOnlyBuffer.position() + numBytes);
return new IntBufferIndexedInts(readOnlyBuffer);
}
@Override
......
......@@ -22,11 +22,20 @@ package io.druid.segment.data;
import java.nio.ByteBuffer;
import java.util.Comparator;
/**
*/
public interface ObjectStrategy<T> extends Comparator<T>
{
public Class<? extends T> getClazz();
/**
* Convert values from their underlying byte representation.
*
* Implementations of this method must not change the given buffer mark, or limit, but may modify its position.
* Use buffer.asReadOnlyBuffer() or buffer.duplicate() if mark or limit need to be set.
*
* @param buffer buffer to read value from
* @param numBytes number of bytes used to store the value, starting at buffer.position()
* @return
*/
public T fromByteBuffer(ByteBuffer buffer, int numBytes);
public byte[] toBytes(T val);
}
......@@ -23,12 +23,14 @@ import com.google.common.base.Function;
import com.google.common.base.Joiner;
import com.google.common.base.Predicate;
import com.google.common.base.Splitter;
import com.google.common.base.Throwables;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.google.common.primitives.Floats;
import com.metamx.common.ISE;
import com.metamx.common.parsers.ParseException;
import io.druid.data.input.InputRow;
import io.druid.data.input.impl.SpatialDimensionSchema;
......@@ -134,14 +136,20 @@ public class SpatialDimensionRowFormatter
}
@Override
public Object getRaw(String dimension) {
public Object getRaw(String dimension)
{
return row.getRaw(dimension);
}
@Override
public float getFloatMetric(String metric)
{
return row.getFloatMetric(metric);
try {
return row.getFloatMetric(metric);
}
catch (ParseException e) {
throw Throwables.propagate(e);
}
}
@Override
......
......@@ -22,6 +22,7 @@ package io.druid.segment.serde;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import io.druid.segment.column.ColumnBuilder;
import io.druid.segment.column.ColumnConfig;
import java.io.IOException;
import java.nio.ByteBuffer;
......@@ -40,5 +41,5 @@ public interface ColumnPartSerde
{
public long numBytes();
public void write(WritableByteChannel channel) throws IOException;
public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder);
public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig);
}
......@@ -22,6 +22,7 @@ package io.druid.segment.serde;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.segment.column.ColumnBuilder;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.data.GenericIndexed;
import java.io.IOException;
......@@ -71,7 +72,7 @@ public class ComplexColumnPartSerde implements ColumnPartSerde
}
@Override
public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder)
public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig)
{
return serde == null ? this : serde.deserializeColumn(buffer, builder);
}
......
......@@ -25,6 +25,7 @@ import com.google.common.primitives.Ints;
import com.metamx.collections.spatial.ImmutableRTree;
import com.metamx.common.IAE;
import io.druid.segment.column.ColumnBuilder;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.column.ValueType;
import io.druid.segment.data.ByteBufferSerializer;
import io.druid.segment.data.ConciseCompressedIndexedInts;
......@@ -43,7 +44,9 @@ import java.nio.channels.WritableByteChannel;
public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
{
@JsonCreator
public static DictionaryEncodedColumnPartSerde createDeserializer(boolean singleValued)
public static DictionaryEncodedColumnPartSerde createDeserializer(
boolean singleValued
)
{
return new DictionaryEncodedColumnPartSerde();
}
......@@ -125,7 +128,7 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
}
@Override
public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder)
public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig)
{
final boolean isSingleValued = buffer.get() == 0x0;
final GenericIndexed<String> dictionary = GenericIndexed.read(buffer, GenericIndexed.stringStrategy);
......@@ -138,12 +141,12 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
singleValuedColumn = VSizeIndexedInts.readFromByteBuffer(buffer);
multiValuedColumn = null;
builder.setHasMultipleValues(false)
.setDictionaryEncodedColumn(new DictionaryEncodedColumnSupplier(dictionary, singleValuedColumn, null));
.setDictionaryEncodedColumn(new DictionaryEncodedColumnSupplier(dictionary, singleValuedColumn, null, columnConfig.columnCacheSizeBytes()));
} else {
singleValuedColumn = null;
multiValuedColumn = VSizeIndexed.readFromByteBuffer(buffer);
builder.setHasMultipleValues(true)
.setDictionaryEncodedColumn(new DictionaryEncodedColumnSupplier(dictionary, null, multiValuedColumn));
.setDictionaryEncodedColumn(new DictionaryEncodedColumnSupplier(dictionary, null, multiValuedColumn, columnConfig.columnCacheSizeBytes()));
}
GenericIndexed<ImmutableConciseSet> bitmaps = GenericIndexed.read(
......
......@@ -33,21 +33,28 @@ public class DictionaryEncodedColumnSupplier implements Supplier<DictionaryEncod
private final GenericIndexed<String> dictionary;
private final VSizeIndexedInts singleValuedColumn;
private final VSizeIndexed multiValuedColumn;
private final int lookupCacheSize;
public DictionaryEncodedColumnSupplier(
GenericIndexed<String> dictionary,
VSizeIndexedInts singleValuedColumn,
VSizeIndexed multiValuedColumn
VSizeIndexed multiValuedColumn,
int lookupCacheSize
)
{
this.dictionary = dictionary;
this.singleValuedColumn = singleValuedColumn;
this.multiValuedColumn = multiValuedColumn;
this.lookupCacheSize = lookupCacheSize;
}
@Override
public DictionaryEncodedColumn get()
{
return new SimpleDictionaryEncodedColumn(singleValuedColumn, multiValuedColumn, dictionary);
return new SimpleDictionaryEncodedColumn(
singleValuedColumn,
multiValuedColumn,
lookupCacheSize > 0 ? dictionary.withCache(lookupCacheSize) : dictionary
);
}
}
......@@ -22,6 +22,7 @@ package io.druid.segment.serde;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.segment.column.ColumnBuilder;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.column.ValueType;
import io.druid.segment.data.CompressedFloatsIndexedSupplier;
......@@ -70,7 +71,7 @@ public class FloatGenericColumnPartSerde implements ColumnPartSerde
}
@Override
public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder)
public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig)
{
final CompressedFloatsIndexedSupplier column = CompressedFloatsIndexedSupplier.fromByteBuffer(buffer, byteOrder);
......
......@@ -22,6 +22,7 @@ package io.druid.segment.serde;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.segment.column.ColumnBuilder;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.column.ValueType;
import io.druid.segment.data.CompressedLongsIndexedSupplier;
......@@ -70,7 +71,7 @@ public class LongGenericColumnPartSerde implements ColumnPartSerde
}
@Override
public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder)
public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig)
{
final CompressedLongsIndexedSupplier column = CompressedLongsIndexedSupplier.fromByteBuffer(buffer, byteOrder);
......
......@@ -65,6 +65,7 @@ public class ChainedExecutionQueryRunnerTest
);
final CountDownLatch queriesStarted = new CountDownLatch(2);
final CountDownLatch queriesInterrupted = new CountDownLatch(2);
final CountDownLatch queryIsRegistered = new CountDownLatch(1);
Capture<ListenableFuture> capturedFuture = new Capture<>();
......@@ -86,9 +87,9 @@ public class ChainedExecutionQueryRunnerTest
EasyMock.replay(watcher);
DyingQueryRunner runner1 = new DyingQueryRunner(queriesStarted);
DyingQueryRunner runner2 = new DyingQueryRunner(queriesStarted);
DyingQueryRunner runner3 = new DyingQueryRunner(queriesStarted);
DyingQueryRunner runner1 = new DyingQueryRunner(queriesStarted, queriesInterrupted);
DyingQueryRunner runner2 = new DyingQueryRunner(queriesStarted, queriesInterrupted);
DyingQueryRunner runner3 = new DyingQueryRunner(queriesStarted, queriesInterrupted);
ChainedExecutionQueryRunner chainedRunner = new ChainedExecutionQueryRunner<>(
exec,
Ordering.<Integer>natural(),
......@@ -135,6 +136,7 @@ public class ChainedExecutionQueryRunnerTest
Assert.assertTrue(e.getCause() instanceof QueryInterruptedException);
cause = (QueryInterruptedException)e.getCause();
}
Assert.assertTrue(queriesInterrupted.await(500, TimeUnit.MILLISECONDS));
Assert.assertNotNull(cause);
Assert.assertTrue(future.isCancelled());
Assert.assertTrue(runner1.hasStarted);
......@@ -170,6 +172,7 @@ public class ChainedExecutionQueryRunnerTest
);
final CountDownLatch queriesStarted = new CountDownLatch(2);
final CountDownLatch queriesInterrupted = new CountDownLatch(2);
final CountDownLatch queryIsRegistered = new CountDownLatch(1);
Capture<ListenableFuture> capturedFuture = new Capture<>();
......@@ -191,9 +194,9 @@ public class ChainedExecutionQueryRunnerTest
EasyMock.replay(watcher);
DyingQueryRunner runner1 = new DyingQueryRunner(queriesStarted);
DyingQueryRunner runner2 = new DyingQueryRunner(queriesStarted);
DyingQueryRunner runner3 = new DyingQueryRunner(queriesStarted);
DyingQueryRunner runner1 = new DyingQueryRunner(queriesStarted, queriesInterrupted);
DyingQueryRunner runner2 = new DyingQueryRunner(queriesStarted, queriesInterrupted);
DyingQueryRunner runner3 = new DyingQueryRunner(queriesStarted, queriesInterrupted);
ChainedExecutionQueryRunner chainedRunner = new ChainedExecutionQueryRunner<>(
exec,
Ordering.<Integer>natural(),
......@@ -210,7 +213,7 @@ public class ChainedExecutionQueryRunnerTest
.dataSource("test")
.intervals("2014/2015")
.aggregators(Lists.<AggregatorFactory>newArrayList(new CountAggregatorFactory("count")))
.context(ImmutableMap.<String, Object>of("timeout", (100), "queryId", "test"))
.context(ImmutableMap.<String, Object>of("timeout", 100, "queryId", "test"))
.build()
);
......@@ -229,10 +232,10 @@ public class ChainedExecutionQueryRunnerTest
Assert.assertTrue(queryIsRegistered.await(1, TimeUnit.SECONDS));
Assert.assertTrue(queriesStarted.await(1, TimeUnit.SECONDS));
// cancel the query
Assert.assertTrue(capturedFuture.hasCaptured());
ListenableFuture future = capturedFuture.getValue();
// wait for query to time out
QueryInterruptedException cause = null;
try {
resultFuture.get();
......@@ -241,6 +244,7 @@ public class ChainedExecutionQueryRunnerTest
Assert.assertEquals("Query timeout", e.getCause().getMessage());
cause = (QueryInterruptedException)e.getCause();
}
Assert.assertTrue(queriesInterrupted.await(500, TimeUnit.MILLISECONDS));
Assert.assertNotNull(cause);
Assert.assertTrue(future.isCancelled());
Assert.assertTrue(runner1.hasStarted);
......@@ -257,23 +261,27 @@ public class ChainedExecutionQueryRunnerTest
private static class DyingQueryRunner implements QueryRunner<Integer>
{
private final CountDownLatch latch;
private final CountDownLatch start;
private final CountDownLatch stop;
private boolean hasStarted = false;
private boolean hasCompleted = false;
private boolean interrupted = false;
public DyingQueryRunner(CountDownLatch latch)
public DyingQueryRunner(CountDownLatch start, CountDownLatch stop)
{
this.latch = latch;
this.start = start;
this.stop = stop;
}
@Override
public Sequence<Integer> run(Query<Integer> query)
{
hasStarted = true;
latch.countDown();
start.countDown();
if (Thread.interrupted()) {
interrupted = true;
stop.countDown();
throw new QueryInterruptedException("I got killed");
}
......@@ -283,10 +291,12 @@ public class ChainedExecutionQueryRunnerTest
}
catch (InterruptedException e) {
interrupted = true;
stop.countDown();
throw new QueryInterruptedException("I got killed");
}
hasCompleted = true;
stop.countDown();
return Sequences.simple(Lists.newArrayList(123));
}
}
......
......@@ -23,6 +23,7 @@ import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import io.druid.granularity.QueryGranularity;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.incremental.IncrementalIndexAdapter;
import org.joda.time.Interval;
......
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.segment;
import org.junit.Assert;
import org.junit.Ignore;
import org.junit.Test;
public class IndexIOTest
{
@Test @Ignore // this test depends on static fields, so it has to be tested independently
public void testInjector() throws Exception
{
System.setProperty("druid.processing.columnCache.sizeBytes", "1234");
Assert.assertEquals(1234, IndexIO.columnConfig.columnCacheSizeBytes());
}
}
......@@ -26,6 +26,7 @@ import com.google.common.io.Files;
import io.druid.data.input.MapBasedInputRow;
import io.druid.granularity.QueryGranularity;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.data.IncrementalIndexTest;
import io.druid.segment.incremental.IncrementalIndex;
import junit.framework.Assert;
......
......@@ -35,6 +35,7 @@ import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.serde.ComplexMetrics;
import org.joda.time.DateTime;
......@@ -180,6 +181,7 @@ public class TestIndex
new TimestampSpec("ts", "iso"),
new DimensionsSpec(Arrays.asList(DIMENSIONS), null, null),
"\t",
"\u0001",
Arrays.asList(COLUMNS)
),
null, null, null, null
......
......@@ -19,8 +19,8 @@
package io.druid.segment.data;
import com.google.common.io.Closeables;
import com.google.common.primitives.Longs;
import com.metamx.common.guava.CloseQuietly;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
......@@ -47,7 +47,7 @@ public class CompressedLongsIndexedSupplierTest
@Before
public void setUp() throws Exception
{
Closeables.closeQuietly(indexed);
CloseQuietly.close(indexed);
indexed = null;
supplier = null;
vals = null;
......@@ -56,7 +56,7 @@ public class CompressedLongsIndexedSupplierTest
@After
public void tearDown() throws Exception
{
Closeables.closeQuietly(indexed);
CloseQuietly.close(indexed);
}
private void setupSimple()
......@@ -247,7 +247,7 @@ public class CompressedLongsIndexedSupplierTest
stopLatch.await();
}
finally {
Closeables.closeQuietly(indexed2);
CloseQuietly.close(indexed2);
}
if (failureHappened.get()) {
......
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册