mirror of https://github.com/apache/druid.git
Segment pruning for multi-dim partitioning given query domain (#12046)
Segment pruning for multi-dim partitioning for a given query DimensionRangeShardSpec#possibleInDomain has been modified to enhance pruning when multi-dim partitioning is used. Idea While iterating through each dimension, If query domain doesn't overlap with the set of permissible values in the segment, the segment is pruned. If the overlap happens on a boundary, consider the next dimensions. If there is an overlap within the segment boundaries, the segment cannot be pruned.
This commit is contained in:
parent
5d043cefbc
commit
c0b1514177
|
@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Range;
|
||||
import com.google.common.collect.RangeSet;
|
||||
import com.google.common.collect.TreeRangeSet;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.StringTuple;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
|
@ -49,9 +50,6 @@ public class DimensionRangeShardSpec implements ShardSpec
|
|||
private final int partitionNum;
|
||||
private final int numCorePartitions;
|
||||
|
||||
private final String firstDimStart;
|
||||
private final String firstDimEnd;
|
||||
|
||||
/**
|
||||
* @param dimensions partition dimensions
|
||||
* @param start inclusive start of this range
|
||||
|
@ -78,8 +76,6 @@ public class DimensionRangeShardSpec implements ShardSpec
|
|||
this.end = end;
|
||||
this.partitionNum = partitionNum;
|
||||
this.numCorePartitions = numCorePartitions == null ? UNKNOWN_NUM_CORE_PARTITIONS : numCorePartitions;
|
||||
this.firstDimStart = getFirstValueOrNull(start);
|
||||
this.firstDimEnd = getFirstValueOrNull(end);
|
||||
}
|
||||
|
||||
@JsonProperty("dimensions")
|
||||
|
@ -145,29 +141,132 @@ public class DimensionRangeShardSpec implements ShardSpec
|
|||
return Collections.unmodifiableList(dimensions);
|
||||
}
|
||||
|
||||
private Range<String> getFirstDimRange()
|
||||
/**
|
||||
* Check if a given domain of Strings is a singleton set containing the given value
|
||||
* @param rangeSet Domain of Strings
|
||||
* @param val Value of String
|
||||
* @return rangeSet == {val}
|
||||
*/
|
||||
private boolean isRangeSetSingletonWithVal(RangeSet<String> rangeSet, String val)
|
||||
{
|
||||
Range<String> range;
|
||||
if (firstDimStart == null && firstDimEnd == null) {
|
||||
range = Range.all();
|
||||
} else if (firstDimStart == null) {
|
||||
range = Range.atMost(firstDimEnd);
|
||||
} else if (firstDimEnd == null) {
|
||||
range = Range.atLeast(firstDimStart);
|
||||
} else {
|
||||
range = Range.closed(firstDimStart, firstDimEnd);
|
||||
if (val == null) {
|
||||
return false;
|
||||
}
|
||||
return range;
|
||||
return rangeSet.asRanges().equals(
|
||||
Collections.singleton(Range.singleton(val))
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set[:i] is the cartesian product of Set[0],...,Set[i - 1]
|
||||
* EffectiveDomain[:i] is defined as QueryDomain[:i] INTERSECTION SegmentRange[:i]
|
||||
*
|
||||
* i = 1
|
||||
* If EffectiveDomain[:i] == {start[:i]} || EffectiveDomain == {end[:i]}:
|
||||
* if i == index.dimensions.size:
|
||||
* ACCEPT segment
|
||||
* else:
|
||||
* REPEAT with i = i + 1
|
||||
*else if EffectiveDomain[:i] == {}:
|
||||
* PRUNE segment
|
||||
*else:
|
||||
* ACCEPT segment
|
||||
*
|
||||
*
|
||||
* Example: Index on (Hour, Minute, Second). Index.size is 3
|
||||
* I)
|
||||
* start = (3, 25, 10)
|
||||
* end = (5, 10, 30)
|
||||
* query domain = {3} * [0, 10] * {10, 20, 30, 40}
|
||||
* EffectiveDomain[:1] == {3} == start[:1]
|
||||
* EffectiveDomain[:2] == {3} * ([0, 10] INTERSECTION [25, INF))
|
||||
* == {} -> PRUNE
|
||||
*
|
||||
* II)
|
||||
* start = (3, 25, 10)
|
||||
* end = (5, 15, 30)
|
||||
* query domain = {4} * [0, 10] * {10, 20, 30, 40}
|
||||
* EffectiveDomain[:1] == {4} (!= {} && != start[:1] && != {end[:1]}) -> ACCEPT
|
||||
*
|
||||
* III)
|
||||
* start = (3, 25, 10)
|
||||
* end = (5, 15, 30)
|
||||
* query domain = {5} * [0, 10] * {10, 20, 30, 40}
|
||||
* EffectiveDomain[:1] == {5} == end[:1]
|
||||
* EffectiveDomain[:2] == {5} * ([0, 10] INTERSECTION (-INF, 15])
|
||||
* == {5} * [0, 10] (! ={} && != {end[:2]}) -> ACCEPT
|
||||
*
|
||||
* IV)
|
||||
* start = (3, 25, 10)
|
||||
* end = (5, 15, 30)
|
||||
* query domain = {5} * [15, 40] * {10, 20, 30, 40}
|
||||
* EffectiveDomain[:1] == {5} == end[:1]
|
||||
* EffectiveDomain[:2] == {5} * ([15, 40] INTERSECTION (-INF, 15])
|
||||
* == {5} * {15} == {end[:2]}
|
||||
* EffectiveDomain[:3] == {5} * {15} * ({10, 20, 30, 40} * (-INF, 30])
|
||||
* == {5} * {15} * {10, 20, 30} != {} -> ACCEPT
|
||||
*
|
||||
* V)
|
||||
* start = (3, 25, 10)
|
||||
* end = (5, 15, 30)
|
||||
* query domain = {5} * [15, 40] * {50}
|
||||
* EffectiveDomain[:1] == {5} == end[:1]
|
||||
* EffectiveDomain[:2] == {5} * ([15, 40] INTERSECTION (-INF, 15])
|
||||
* == {5} * {15} == {end[:2]}
|
||||
* EffectiveDomain[:3] == {5} * {15} * ({40} * (-INF, 30])
|
||||
* == {5} * {15} * {}
|
||||
* == {} -> PRUNE
|
||||
*
|
||||
* @param domain The domain inferred from the query. Assumed to be non-emtpy
|
||||
* @return true if segment needs to be considered for query, false if it can be pruned
|
||||
*/
|
||||
@Override
|
||||
public boolean possibleInDomain(Map<String, RangeSet<String>> domain)
|
||||
{
|
||||
RangeSet<String> rangeSet = domain.get(dimensions.get(0));
|
||||
if (rangeSet == null) {
|
||||
return true;
|
||||
final StringTuple segmentStart = start == null ? new StringTuple(new String[dimensions.size()]) : start;
|
||||
final StringTuple segmentEnd = end == null ? new StringTuple(new String[dimensions.size()]) : end;
|
||||
|
||||
// Indicates if the effective domain is equivalent to {start} till the previous dimension
|
||||
boolean effectiveDomainIsStart = true;
|
||||
// Indicates if the effective domain is equivalent to {end} till the previous dimension
|
||||
boolean effectiveDomainIsEnd = true;
|
||||
|
||||
for (int i = 0; i < dimensions.size(); i++) {
|
||||
String dimension = dimensions.get(i);
|
||||
RangeSet<String> queryDomainForDimension = domain.get(dimension);
|
||||
if (queryDomainForDimension == null) {
|
||||
queryDomainForDimension = TreeRangeSet.create();
|
||||
queryDomainForDimension.add(Range.all());
|
||||
}
|
||||
|
||||
// Compute the segment's range for given dimension based on its start, end and boundary conditions
|
||||
Range<String> rangeTillSegmentBoundary = Range.all();
|
||||
if (effectiveDomainIsStart && segmentStart.get(i) != null) {
|
||||
rangeTillSegmentBoundary = rangeTillSegmentBoundary.intersection(Range.atLeast(segmentStart.get(i)));
|
||||
}
|
||||
if (effectiveDomainIsEnd && segmentEnd.get(i) != null) {
|
||||
rangeTillSegmentBoundary = rangeTillSegmentBoundary.intersection(Range.atMost(segmentEnd.get(i)));
|
||||
}
|
||||
|
||||
// EffectiveDomain[i] = QueryDomain[i] INTERSECTION SegmentRange[i]
|
||||
RangeSet<String> effectiveDomainForDimension = queryDomainForDimension.subRangeSet(rangeTillSegmentBoundary);
|
||||
// Prune segment because query domain is out of segment range
|
||||
if (effectiveDomainForDimension.isEmpty()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
// EffectiveDomain is singleton and lies only on the boundaries -> consider next dimensions
|
||||
effectiveDomainIsStart = effectiveDomainIsStart
|
||||
&& isRangeSetSingletonWithVal(effectiveDomainForDimension, segmentStart.get(i));
|
||||
effectiveDomainIsEnd = effectiveDomainIsEnd
|
||||
&& isRangeSetSingletonWithVal(effectiveDomainForDimension, segmentEnd.get(i));
|
||||
|
||||
// EffectiveDomain lies within the boundaries as well -> cannot prune based on next dimensions
|
||||
if (!effectiveDomainIsStart && !effectiveDomainIsEnd) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return !rangeSet.subRangeSet(getFirstDimRange()).isEmpty();
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -207,11 +306,6 @@ public class DimensionRangeShardSpec implements ShardSpec
|
|||
&& (inputVsEnd < 0 || end == null);
|
||||
}
|
||||
|
||||
private static String getFirstValueOrNull(StringTuple values)
|
||||
{
|
||||
return values != null && values.size() > 0 ? values.get(0) : null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType()
|
||||
{
|
||||
|
|
|
@ -19,6 +19,9 @@
|
|||
|
||||
package org.apache.druid.timeline.partition;
|
||||
|
||||
import com.google.common.collect.Range;
|
||||
import com.google.common.collect.RangeSet;
|
||||
import com.google.common.collect.TreeRangeSet;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.MapBasedInputRow;
|
||||
import org.apache.druid.data.input.StringTuple;
|
||||
|
@ -297,6 +300,313 @@ public class DimensionRangeShardSpecTest
|
|||
));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPossibleInDomain_withNullStart()
|
||||
{
|
||||
setDimensions("planet", "country", "city");
|
||||
|
||||
final StringTuple start = null; // considered to be (-INF, -INF, -INF)
|
||||
final StringTuple end = StringTuple.create("Saturn", "Foo", "Bar");
|
||||
|
||||
final RangeSet<String> universalSet = TreeRangeSet.create();
|
||||
universalSet.add(Range.all());
|
||||
|
||||
ShardSpec shard = new DimensionRangeShardSpec(dimensions, start, end, 0, null);
|
||||
Map<String, RangeSet<String>> domain = new HashMap<>();
|
||||
|
||||
// {Mars} * {Zoo, Zuu} * {Blah, Random}
|
||||
populateDomain(
|
||||
domain,
|
||||
getRangeSet(Range.singleton("Mars")),
|
||||
// EffectiveDomain[:1].size > 1 -> ACCEPT
|
||||
getUnion(
|
||||
getRangeSet(Range.singleton("Zoo")),
|
||||
getRangeSet(Range.singleton("Zuu"))
|
||||
),
|
||||
getUnion(
|
||||
getRangeSet(Range.singleton("Blah")),
|
||||
getRangeSet(Range.singleton("Random"))
|
||||
)
|
||||
);
|
||||
assertTrue(shard.possibleInDomain(domain));
|
||||
|
||||
// {Saturn} * (-INF, INF) * (-INF, INF)
|
||||
populateDomain(
|
||||
domain,
|
||||
getRangeSet(Range.singleton("Saturn")),
|
||||
// EffectiveDomain[:1] == {end[:1]}
|
||||
universalSet,
|
||||
// EffectiveDomain[:2].size > 1 -> ACCEPT
|
||||
universalSet
|
||||
);
|
||||
assertTrue(shard.possibleInDomain(domain));
|
||||
|
||||
// {Saturn} * {Zoo} * (-INF, INF)
|
||||
populateDomain(
|
||||
domain,
|
||||
getRangeSet(Range.singleton("Saturn")),
|
||||
// EffectiveDomain[:1] == {end[:1]}
|
||||
getRangeSet(Range.singleton("Zoo")),
|
||||
// EffectiveDomain[:2] == {} -> PRUNE
|
||||
universalSet
|
||||
);
|
||||
assertFalse(shard.possibleInDomain(domain));
|
||||
|
||||
// (Xeon) * (-INF, INF) * (-INF, INF)
|
||||
populateDomain(
|
||||
domain,
|
||||
getRangeSet(Range.singleton("Xeon")),
|
||||
// EffectiveDomain[:1] == {} -> PRUNE
|
||||
universalSet,
|
||||
universalSet
|
||||
);
|
||||
assertFalse(shard.possibleInDomain(domain));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPossibleInDomain_withNullValues()
|
||||
{
|
||||
setDimensions("planet", "country", "city");
|
||||
|
||||
final StringTuple start = StringTuple.create("Earth", "India", "Delhi");
|
||||
final StringTuple end = StringTuple.create("Krypton", null, "Kryptonopolis"); // null in end translates to INF
|
||||
|
||||
final RangeSet<String> universalSet = TreeRangeSet.create();
|
||||
universalSet.add(Range.all());
|
||||
|
||||
ShardSpec shard = new DimensionRangeShardSpec(dimensions, start, end, 0, null);
|
||||
Map<String, RangeSet<String>> domain = new HashMap<>();
|
||||
|
||||
// (-INF, INF) * (-INF, INF) * (-INF, INF)
|
||||
populateDomain(
|
||||
domain,
|
||||
universalSet,
|
||||
// EffectiveDomain[:1].size > 1 -> ACCEPT
|
||||
universalSet,
|
||||
universalSet
|
||||
);
|
||||
assertTrue(shard.possibleInDomain(domain));
|
||||
|
||||
// {Earth} * (-INF, INF) * (-INF, INF)
|
||||
populateDomain(
|
||||
domain,
|
||||
getRangeSet(Range.singleton("Earth")),
|
||||
// EffectiveDomain[:1] == {start[:1]}
|
||||
universalSet,
|
||||
// EffectiveDomain[:2].size > 1 -> ACCEPT
|
||||
universalSet
|
||||
);
|
||||
assertTrue(shard.possibleInDomain(domain));
|
||||
|
||||
// (-INF, Earth) U (Krypton, INF) * (-INF, INF) * (-INF, INF)
|
||||
populateDomain(
|
||||
domain,
|
||||
getUnion(
|
||||
getRangeSet(Range.lessThan("Earth")),
|
||||
getRangeSet(Range.greaterThan("Krypton"))
|
||||
),
|
||||
// EffectiveDomain[:1] = {} -> PRUNE
|
||||
universalSet,
|
||||
universalSet
|
||||
);
|
||||
assertFalse(shard.possibleInDomain(domain));
|
||||
|
||||
// (-INF, INF) * (-INF, France) * (-INF, INF)
|
||||
populateDomain(
|
||||
domain,
|
||||
universalSet,
|
||||
// EffectiveDomain[:1].size > 2 -> ACCEPT
|
||||
getRangeSet(Range.lessThan("France")),
|
||||
universalSet
|
||||
);
|
||||
assertTrue(shard.possibleInDomain(domain));
|
||||
|
||||
// {Jupiter} * (Foo) * {Bar}
|
||||
populateDomain(
|
||||
domain,
|
||||
getRangeSet(Range.singleton("Jupiter")),
|
||||
// EffectiveDomain[:1] != {} OR {start[:1]} OR {end[:1]} -> ACCEPT
|
||||
getRangeSet(Range.singleton("Foo")),
|
||||
getRangeSet(Range.singleton("Bar"))
|
||||
);
|
||||
assertTrue(shard.possibleInDomain(domain));
|
||||
|
||||
// {Krypton} * (-INF, France] * {Paris}
|
||||
populateDomain(
|
||||
domain,
|
||||
getRangeSet(Range.singleton("Krypton")),
|
||||
// EffectiveDomain[:1] == {end[:1]}
|
||||
getRangeSet(Range.atMost("France")),
|
||||
// EffectiveDomain[:2].size > 1 -> ACCEPT
|
||||
universalSet
|
||||
);
|
||||
assertTrue(shard.possibleInDomain(domain));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPossibleInDomain_nonNullValues_acceptanceScenarios()
|
||||
{
|
||||
setDimensions("planet", "country", "city");
|
||||
|
||||
final StringTuple start = StringTuple.create("Earth", "France", "Paris");
|
||||
final StringTuple end = StringTuple.create("Earth", "USA", "New York");
|
||||
|
||||
final RangeSet<String> universalSet = TreeRangeSet.create();
|
||||
universalSet.add(Range.all());
|
||||
|
||||
ShardSpec shard = new DimensionRangeShardSpec(dimensions, start, end, 0, null);
|
||||
Map<String, RangeSet<String>> domain = new HashMap<>();
|
||||
|
||||
// (-INF, INF) * (-INF, INF) * (-INF, INF)
|
||||
populateDomain(
|
||||
domain,
|
||||
universalSet,
|
||||
// EffectiveDomain[:1] == {Earth} == {start[:1]} == {end[:1]}
|
||||
universalSet,
|
||||
// EffectiveDomain[:2].size > 1 -> ACCEPT
|
||||
universalSet
|
||||
);
|
||||
assertTrue(shard.possibleInDomain(domain));
|
||||
|
||||
// {Earth} * (-INF, INF) * (-INF, INF)
|
||||
populateDomain(
|
||||
domain,
|
||||
getRangeSet(Range.singleton("Earth")),
|
||||
// EffectiveDomain[:1] == {Earth} == {start[:1]} == {end[:1]}
|
||||
universalSet,
|
||||
// EffectiveDomain[:2].size > 1 -> ACCEPT
|
||||
universalSet
|
||||
);
|
||||
assertTrue(shard.possibleInDomain(domain));
|
||||
|
||||
// (-INF, INF) * [USA, INF) * {New York}
|
||||
populateDomain(
|
||||
domain,
|
||||
universalSet,
|
||||
// EffectiveDomain[:1] == {Earth} == {start[:1]} == {end[:1]}
|
||||
getRangeSet(Range.atLeast("USA")),
|
||||
// EffectiveDomain[:2] == {end[:2]}
|
||||
getRangeSet(Range.singleton("New York"))
|
||||
// EffectiveDomain[:3] == {end[:3]}
|
||||
);
|
||||
//EffectiveDomain[:].size > 0 -> ACCEPT
|
||||
assertTrue(shard.possibleInDomain(domain));
|
||||
|
||||
// (-INF, INF) * (-INF, "France"] * (Paris, INF)
|
||||
populateDomain(
|
||||
domain,
|
||||
universalSet,
|
||||
// EffectiveDomain[:1] == {Earth} == {start[:1]} == {end[:1]}
|
||||
getRangeSet(Range.atMost("France")),
|
||||
// EffectiveDomain[:2] == {<Earth, France>} == {start[:2]}
|
||||
getRangeSet(Range.greaterThan("Paris"))
|
||||
// EffectiveDomain[:3].size > 1 -> ACCEPT
|
||||
);
|
||||
assertTrue(shard.possibleInDomain(domain));
|
||||
|
||||
// {Earth} * {India} * Any Non-empty set
|
||||
populateDomain(
|
||||
domain,
|
||||
getRangeSet(Range.singleton("Earth")),
|
||||
// EffectiveDomain[:1] == {Earth} == {start[:1]} == {end[:1]}
|
||||
getRangeSet(Range.singleton("India")),
|
||||
// EffectiveDomain[:2] == {<Earth, India>} != {start[:2]} OR {end[:2]}
|
||||
getRangeSet(Range.greaterThan("New York"))
|
||||
// EffectiveDomain[:3].size > 1 -> ACCEPT
|
||||
);
|
||||
assertTrue(shard.possibleInDomain(domain));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPossibleInDomain_nonNullValues_pruningScenarios()
|
||||
{
|
||||
setDimensions("planet", "country", "city");
|
||||
|
||||
final StringTuple start = StringTuple.create("Earth", "France", "Paris");
|
||||
final StringTuple end = StringTuple.create("Earth", "USA", "New York");
|
||||
|
||||
final RangeSet<String> universalSet = TreeRangeSet.create();
|
||||
universalSet.add(Range.all());
|
||||
|
||||
ShardSpec shard = new DimensionRangeShardSpec(dimensions, start, end, 0, null);
|
||||
Map<String, RangeSet<String>> domain = new HashMap<>();
|
||||
|
||||
// (-INF, Earth) U (Earth, INF) * (-INF, INF) * (-INF, INF)
|
||||
populateDomain(
|
||||
domain,
|
||||
getUnion(
|
||||
getRangeSet(Range.lessThan("Earth")),
|
||||
getRangeSet(Range.greaterThan("Earth"))
|
||||
),
|
||||
// EffectiveDomain[:1] == {} -> PRUNE
|
||||
universalSet,
|
||||
universalSet
|
||||
);
|
||||
assertFalse(shard.possibleInDomain(domain));
|
||||
|
||||
// (-INF, INF) * (-INF, "France") * (-INF, INF)
|
||||
populateDomain(
|
||||
domain,
|
||||
universalSet,
|
||||
// EffectiveDomain[:1] == {Earth} == {start[:1]} == {end[:1]}
|
||||
getRangeSet(Range.lessThan("France")),
|
||||
// EffectiveDomain[:2] == {} -> PRUNE
|
||||
universalSet
|
||||
);
|
||||
assertFalse(shard.possibleInDomain(domain));
|
||||
|
||||
// (-INF, INF) * (-INF, "France] * (-INF, Paris)
|
||||
populateDomain(
|
||||
domain,
|
||||
universalSet,
|
||||
// EffectiveDomain[:1] == {Earth} == {start[:1]} == {end[:1]}
|
||||
getRangeSet(Range.atMost("France")),
|
||||
// EffectiveDomain[:2] == {<Earth, France>} == {start[:2]}
|
||||
getRangeSet(Range.lessThan("Paris"))
|
||||
// EffectiveDomain[:3] == {} -> PRUNE
|
||||
);
|
||||
assertFalse(shard.possibleInDomain(domain));
|
||||
|
||||
// {Earth} * {USA} * (New York, INF)
|
||||
populateDomain(
|
||||
domain,
|
||||
getRangeSet(Range.singleton("Earth")),
|
||||
// EffectiveDomain[:1] == {Earth} == {start[:1]} == {end[:1]}
|
||||
getRangeSet(Range.singleton("USA")),
|
||||
// EffectiveDomain[:2] == {<Earth, USA>} == {end[:2]}
|
||||
getRangeSet(Range.greaterThan("New York"))
|
||||
// EffectiveDomain[:3] == {} -> PRUNE
|
||||
);
|
||||
assertFalse(shard.possibleInDomain(domain));
|
||||
}
|
||||
|
||||
private RangeSet<String> getRangeSet(Range range)
|
||||
{
|
||||
RangeSet<String> rangeSet = TreeRangeSet.create();
|
||||
rangeSet.add(range);
|
||||
return rangeSet;
|
||||
}
|
||||
|
||||
private RangeSet<String> getUnion(RangeSet<String>... rangeSets)
|
||||
{
|
||||
RangeSet<String> unionSet = TreeRangeSet.create();
|
||||
for (RangeSet<String> range : rangeSets) {
|
||||
unionSet.addAll(range);
|
||||
}
|
||||
return unionSet;
|
||||
}
|
||||
|
||||
private void populateDomain(Map<String, RangeSet<String>> domain,
|
||||
RangeSet<String> planetSet,
|
||||
RangeSet<String> countrySet,
|
||||
RangeSet<String> citySet)
|
||||
{
|
||||
domain.clear();
|
||||
domain.put("planet", planetSet);
|
||||
domain.put("country", countrySet);
|
||||
domain.put("city", citySet);
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if the given InputRow is in the chunk represented by the given shard spec.
|
||||
*/
|
||||
|
|
Loading…
Reference in New Issue