Merge pull request #791 from metamx/fix-rollup-hashed-partitions

Fix rollup hashed partitions
This commit is contained in:
Fangjin Yang 2014-10-15 12:26:03 -06:00
commit 8ec8952993
10 changed files with 112 additions and 23 deletions

View File

@ -41,6 +41,7 @@ import com.metamx.common.logger.Logger;
import io.druid.common.utils.JodaUtils;
import io.druid.data.input.InputRow;
import io.druid.data.input.impl.StringInputRowParser;
import io.druid.granularity.QueryGranularity;
import io.druid.guice.GuiceInjectors;
import io.druid.guice.JsonConfigProvider;
import io.druid.guice.annotations.Self;
@ -172,6 +173,7 @@ public class HadoopDruidIndexerConfig
private volatile PathSpec pathSpec;
private volatile Map<DateTime,ShardSpecLookup> shardSpecLookups = Maps.newHashMap();
private volatile Map<ShardSpec, HadoopyShardSpec> hadoopShardSpecLookup = Maps.newHashMap();
private final QueryGranularity rollupGran;
@JsonCreator
public HadoopDruidIndexerConfig(
@ -203,6 +205,7 @@ public class HadoopDruidIndexerConfig
hadoopShardSpecLookup.put(hadoopyShardSpec.getActualSpec(), hadoopyShardSpec);
}
}
this.rollupGran = schema.getDataSchema().getGranularitySpec().getQueryGranularity();
}
@JsonProperty
@ -326,7 +329,7 @@ public class HadoopDruidIndexerConfig
return Optional.absent();
}
final ShardSpec actualSpec = shardSpecLookups.get(timeBucket.get().getStart()).getShardSpec(inputRow);
final ShardSpec actualSpec = shardSpecLookups.get(timeBucket.get().getStart()).getShardSpec(rollupGran.truncate(inputRow.getTimestampFromEpoch()), inputRow);
final HadoopyShardSpec hadoopyShardSpec = hadoopShardSpecLookup.get(actualSpec);
return Optional.of(

View File

@ -20,15 +20,34 @@
package io.druid.indexer;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.api.client.util.Lists;
import com.google.common.base.Optional;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.metamx.common.Granularity;
import io.druid.data.input.InputRow;
import io.druid.data.input.MapBasedInputRow;
import io.druid.data.input.impl.JSONDataSpec;
import io.druid.data.input.impl.TimestampSpec;
import io.druid.granularity.QueryGranularity;
import io.druid.indexer.rollup.DataRollupSpec;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
import io.druid.timeline.partition.HashBasedNumberedShardSpec;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
import java.util.Arrays;
import java.util.List;
/**
*/
public class HadoopDruidIndexerConfigTest
@ -125,4 +144,68 @@ public class HadoopDruidIndexerConfigTest
);
}
@Test
public void testHashedBucketSelection() {
List<HadoopyShardSpec> specs = Lists.newArrayList();
final int partitionCount = 10;
for (int i = 0; i < partitionCount; i++) {
specs.add(new HadoopyShardSpec(new HashBasedNumberedShardSpec(i, partitionCount, new DefaultObjectMapper()), i));
}
HadoopIngestionSpec spec = new HadoopIngestionSpec(
null, null, null,
"foo",
new TimestampSpec("timestamp", "auto"),
new JSONDataSpec(ImmutableList.of("foo"), null),
new UniformGranularitySpec(
Granularity.HOUR,
QueryGranularity.MINUTE,
ImmutableList.of(new Interval("2010-01-01/P1D")),
Granularity.HOUR
),
null,
null,
null,
null,
null,
false,
true,
ImmutableMap.of(new DateTime("2010-01-01T01:00:00"), specs),
false,
new DataRollupSpec(ImmutableList.<AggregatorFactory>of(), QueryGranularity.MINUTE),
null,
false,
ImmutableMap.of("foo", "bar"),
false,
null,
null,
null,
null,
null,
null
);
HadoopDruidIndexerConfig config = HadoopDruidIndexerConfig.fromSchema(spec);
final List<String> dims = Arrays.asList("diM1", "dIM2");
final ImmutableMap<String, Object> values = ImmutableMap.<String, Object>of(
"Dim1",
"1",
"DiM2",
"2",
"dim1",
"3",
"dim2",
"4"
);
final long timestamp = new DateTime("2010-01-01T01:00:01").getMillis();
final Bucket expectedBucket = config.getBucket(new MapBasedInputRow(timestamp, dims, values)).get();
final long nextBucketTimestamp = QueryGranularity.MINUTE.next(QueryGranularity.MINUTE.truncate(timestamp));
// check that all rows having same set of dims and truncated timestamp hash to same bucket
for (int i = 0; timestamp + i < nextBucketTimestamp; i++) {
Assert.assertEquals(
expectedBucket.partitionNum,
config.getBucket(new MapBasedInputRow(timestamp + i, dims, values)).get().partitionNum
);
}
}
}

View File

@ -407,14 +407,14 @@ public class IndexTask extends AbstractFixedIntervalTask
final int myRowFlushBoundary = rowFlushBoundary > 0
? rowFlushBoundary
: toolbox.getConfig().getDefaultRowFlushBoundary();
final QueryGranularity rollupGran = ingestionSchema.getDataSchema().getGranularitySpec().getQueryGranularity();
try {
plumber.startJob();
while (firehose.hasMore()) {
final InputRow inputRow = firehose.nextRow();
if (shouldIndex(shardSpec, interval, inputRow)) {
if (shouldIndex(shardSpec, interval, inputRow, rollupGran)) {
int numRows = plumber.add(inputRow);
if (numRows == -1) {
throw new ISE(
@ -469,13 +469,15 @@ public class IndexTask extends AbstractFixedIntervalTask
*
* @return true or false
*/
private boolean shouldIndex(
private static boolean shouldIndex(
final ShardSpec shardSpec,
final Interval interval,
final InputRow inputRow
final InputRow inputRow,
final QueryGranularity rollupGran
)
{
return interval.contains(inputRow.getTimestampFromEpoch()) && shardSpec.isInChunk(inputRow);
return interval.contains(inputRow.getTimestampFromEpoch())
&& shardSpec.isInChunk(rollupGran.truncate(inputRow.getTimestampFromEpoch()), inputRow);
}
public static class IndexIngestionSpec extends IngestionSpec<IndexIOConfig, IndexTuningConfig>

View File

@ -41,7 +41,7 @@
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<metamx.java-util.version>0.26.9</metamx.java-util.version>
<apache.curator.version>2.6.0</apache.curator.version>
<druid.api.version>0.2.14</druid.api.version>
<druid.api.version>0.2.14.1</druid.api.version>
</properties>
<modules>

View File

@ -49,14 +49,14 @@ public class HashBasedNumberedShardSpec extends NumberedShardSpec
}
@Override
public boolean isInChunk(InputRow inputRow)
public boolean isInChunk(long timestamp, InputRow inputRow)
{
return (((long) hash(inputRow)) - getPartitionNum()) % getPartitions() == 0;
return (((long) hash(timestamp, inputRow)) - getPartitionNum()) % getPartitions() == 0;
}
protected int hash(InputRow inputRow)
protected int hash(long timestamp, InputRow inputRow)
{
final List<Object> groupKey = Rows.toGroupKey(inputRow.getTimestampFromEpoch(), inputRow);
final List<Object> groupKey = Rows.toGroupKey(timestamp, inputRow);
try {
return hashFunction.hashBytes(jsonMapper.writeValueAsBytes(groupKey)).asInt();
}
@ -80,9 +80,9 @@ public class HashBasedNumberedShardSpec extends NumberedShardSpec
return new ShardSpecLookup()
{
@Override
public ShardSpec getShardSpec(InputRow row)
public ShardSpec getShardSpec(long timestamp, InputRow row)
{
int index = Math.abs(hash(row) % getPartitions());
int index = Math.abs(hash(timestamp, row) % getPartitions());
return shardSpecs.get(index);
}
};

View File

@ -50,7 +50,7 @@ public class LinearShardSpec implements ShardSpec
return new ShardSpecLookup()
{
@Override
public ShardSpec getShardSpec(InputRow row)
public ShardSpec getShardSpec(long timestamp, InputRow row)
{
return shardSpecs.get(0);
}
@ -63,7 +63,7 @@ public class LinearShardSpec implements ShardSpec
}
@Override
public boolean isInChunk(InputRow inputRow) {
public boolean isInChunk(long timestamp, InputRow inputRow) {
return true;
}

View File

@ -60,7 +60,7 @@ public class NumberedShardSpec implements ShardSpec
return new ShardSpecLookup()
{
@Override
public ShardSpec getShardSpec(InputRow row)
public ShardSpec getShardSpec(long timestamp, InputRow row)
{
return shardSpecs.get(0);
}
@ -80,7 +80,7 @@ public class NumberedShardSpec implements ShardSpec
}
@Override
public boolean isInChunk(InputRow inputRow)
public boolean isInChunk(long timestamp, InputRow inputRow)
{
return true;
}

View File

@ -100,10 +100,10 @@ public class SingleDimensionShardSpec implements ShardSpec
return new ShardSpecLookup()
{
@Override
public ShardSpec getShardSpec(InputRow row)
public ShardSpec getShardSpec(long timestamp, InputRow row)
{
for (ShardSpec spec : shardSpecs) {
if (spec.isInChunk(row)) {
if (spec.isInChunk(timestamp, row)) {
return spec;
}
}
@ -124,7 +124,7 @@ public class SingleDimensionShardSpec implements ShardSpec
}
@Override
public boolean isInChunk(InputRow inputRow)
public boolean isInChunk(long timestamp, InputRow inputRow)
{
final List<String> values = inputRow.getDimension(dimension);

View File

@ -127,7 +127,7 @@ public class HashBasedNumberedShardSpecTest
public boolean assertExistsInOneSpec(List<ShardSpec> specs, InputRow row)
{
for (ShardSpec spec : specs) {
if (spec.isInChunk(row)) {
if (spec.isInChunk(row.getTimestampFromEpoch(), row)) {
return true;
}
}
@ -145,7 +145,7 @@ public class HashBasedNumberedShardSpecTest
}
@Override
protected int hash(InputRow inputRow)
protected int hash(long timestamp, InputRow inputRow)
{
return inputRow.hashCode();
}
@ -208,4 +208,5 @@ public class HashBasedNumberedShardSpecTest
return 0;
}
}
}

View File

@ -111,7 +111,7 @@ public class SingleDimensionShardSpecTest
}
)
);
Assert.assertEquals(String.format("spec[%s], row[%s]", spec, inputRow), pair.lhs, spec.isInChunk(inputRow));
Assert.assertEquals(String.format("spec[%s], row[%s]", spec, inputRow), pair.lhs, spec.isInChunk(inputRow.getTimestampFromEpoch(), inputRow));
}
}
}