mirror of https://github.com/apache/druid.git
address cr
This commit is contained in:
parent
d4a47fe6e8
commit
d2cf7d3f0a
|
@ -692,14 +692,14 @@ public class Druids
|
|||
{
|
||||
private DataSource dataSource;
|
||||
private QuerySegmentSpec querySegmentSpec;
|
||||
private String exclude;
|
||||
private String bound;
|
||||
private Map<String, Object> context;
|
||||
|
||||
public TimeBoundaryQueryBuilder()
|
||||
{
|
||||
dataSource = null;
|
||||
querySegmentSpec = null;
|
||||
exclude = null;
|
||||
bound = null;
|
||||
context = null;
|
||||
}
|
||||
|
||||
|
@ -708,7 +708,7 @@ public class Druids
|
|||
return new TimeBoundaryQuery(
|
||||
dataSource,
|
||||
querySegmentSpec,
|
||||
exclude,
|
||||
bound,
|
||||
context
|
||||
);
|
||||
}
|
||||
|
@ -718,7 +718,7 @@ public class Druids
|
|||
return new TimeBoundaryQueryBuilder()
|
||||
.dataSource(builder.dataSource)
|
||||
.intervals(builder.querySegmentSpec)
|
||||
.exclude(builder.exclude)
|
||||
.bound(builder.bound)
|
||||
.context(builder.context);
|
||||
}
|
||||
|
||||
|
@ -752,9 +752,9 @@ public class Druids
|
|||
return this;
|
||||
}
|
||||
|
||||
public TimeBoundaryQueryBuilder exclude(String ex)
|
||||
public TimeBoundaryQueryBuilder bound(String b)
|
||||
{
|
||||
exclude = ex;
|
||||
bound = b;
|
||||
return this;
|
||||
}
|
||||
|
||||
|
|
|
@ -50,13 +50,13 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
|
|||
public static final String MIN_TIME = "minTime";
|
||||
private static final byte CACHE_TYPE_ID = 0x0;
|
||||
|
||||
private final String exclude;
|
||||
private final String bound;
|
||||
|
||||
@JsonCreator
|
||||
public TimeBoundaryQuery(
|
||||
@JsonProperty("dataSource") DataSource dataSource,
|
||||
@JsonProperty("intervals") QuerySegmentSpec querySegmentSpec,
|
||||
@JsonProperty("exclude") String exclude,
|
||||
@JsonProperty("bound") String bound,
|
||||
@JsonProperty("context") Map<String, Object> context
|
||||
)
|
||||
{
|
||||
|
@ -67,7 +67,7 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
|
|||
context
|
||||
);
|
||||
|
||||
this.exclude = exclude == null ? "" : exclude;
|
||||
this.bound = bound == null ? "" : bound;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -83,9 +83,9 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
|
|||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getExclude()
|
||||
public String getBound()
|
||||
{
|
||||
return exclude;
|
||||
return bound;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -94,7 +94,7 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
|
|||
return new TimeBoundaryQuery(
|
||||
getDataSource(),
|
||||
getQuerySegmentSpec(),
|
||||
exclude,
|
||||
bound,
|
||||
computeOverridenContext(contextOverrides)
|
||||
);
|
||||
}
|
||||
|
@ -105,7 +105,7 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
|
|||
return new TimeBoundaryQuery(
|
||||
getDataSource(),
|
||||
spec,
|
||||
exclude,
|
||||
bound,
|
||||
getContext()
|
||||
);
|
||||
}
|
||||
|
@ -116,14 +116,14 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
|
|||
return new TimeBoundaryQuery(
|
||||
dataSource,
|
||||
getQuerySegmentSpec(),
|
||||
exclude,
|
||||
bound,
|
||||
getContext()
|
||||
);
|
||||
}
|
||||
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
final byte[] excludeBytes = exclude.getBytes();
|
||||
final byte[] excludeBytes = bound.getBytes();
|
||||
return ByteBuffer.allocate(1 + excludeBytes.length)
|
||||
.put(CACHE_TYPE_ID)
|
||||
.put(excludeBytes)
|
||||
|
@ -137,7 +137,7 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
|
|||
"dataSource='" + getDataSource() + '\'' +
|
||||
", querySegmentSpec=" + getQuerySegmentSpec() +
|
||||
", duration=" + getDuration() +
|
||||
", exclude" + exclude +
|
||||
", bound" + bound +
|
||||
'}';
|
||||
}
|
||||
|
||||
|
@ -146,10 +146,10 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
|
|||
List<Result<TimeBoundaryResultValue>> results = Lists.newArrayList();
|
||||
Map<String, Object> result = Maps.newHashMap();
|
||||
|
||||
if (min != null && !exclude.equalsIgnoreCase(MIN_TIME)) {
|
||||
if (min != null && !bound.equalsIgnoreCase(MAX_TIME)) {
|
||||
result.put(MIN_TIME, min);
|
||||
}
|
||||
if (max != null && !exclude.equalsIgnoreCase(MAX_TIME)) {
|
||||
if (max != null && !bound.equalsIgnoreCase(MIN_TIME)) {
|
||||
result.put(MAX_TIME, max);
|
||||
}
|
||||
if (!result.isEmpty()) {
|
||||
|
@ -170,13 +170,13 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
|
|||
for (Result<TimeBoundaryResultValue> result : results) {
|
||||
TimeBoundaryResultValue val = result.getValue();
|
||||
|
||||
if (!exclude.equalsIgnoreCase(MIN_TIME)) {
|
||||
if (!bound.equalsIgnoreCase(MAX_TIME)) {
|
||||
DateTime currMinTime = val.getMinTime();
|
||||
if (currMinTime.isBefore(min)) {
|
||||
min = currMinTime;
|
||||
}
|
||||
}
|
||||
if (!exclude.equalsIgnoreCase(MAX_TIME)) {
|
||||
if (!bound.equalsIgnoreCase(MIN_TIME)) {
|
||||
DateTime currMaxTime = val.getMaxTime();
|
||||
if (currMaxTime.isAfter(max)) {
|
||||
max = currMaxTime;
|
||||
|
@ -187,16 +187,16 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
|
|||
final ImmutableMap.Builder<String, Object> builder = new ImmutableMap.Builder<>();
|
||||
final DateTime ts;
|
||||
|
||||
if (exclude.equalsIgnoreCase(MIN_TIME)) {
|
||||
ts = max;
|
||||
builder.put(MAX_TIME, max);
|
||||
} else if (exclude.equalsIgnoreCase(MAX_TIME)) {
|
||||
if (bound.equalsIgnoreCase(MIN_TIME)) {
|
||||
ts = min;
|
||||
builder.put(MIN_TIME, min);
|
||||
builder.put(MIN_TIME, max);
|
||||
} else if (bound.equalsIgnoreCase(MAX_TIME)) {
|
||||
ts = max;
|
||||
builder.put(MAX_TIME, min);
|
||||
} else {
|
||||
ts = min;
|
||||
builder.put(MAX_TIME, max);
|
||||
builder.put(MIN_TIME, min);
|
||||
builder.put(MAX_TIME, max);
|
||||
}
|
||||
|
||||
return Arrays.asList(
|
||||
|
|
|
@ -72,7 +72,7 @@ public class TimeBoundaryQueryQueryToolChest
|
|||
|
||||
final Predicate<T> filterPredicate;
|
||||
// optimizations to avoid hitting too many segments
|
||||
if (query.getExclude().equalsIgnoreCase(TimeBoundaryQuery.MAX_TIME)) {
|
||||
if (query.getBound().equalsIgnoreCase(TimeBoundaryQuery.MIN_TIME)) {
|
||||
filterPredicate = new Predicate<T>()
|
||||
{
|
||||
@Override
|
||||
|
@ -81,7 +81,7 @@ public class TimeBoundaryQueryQueryToolChest
|
|||
return input.getInterval().overlaps(min.getInterval());
|
||||
}
|
||||
};
|
||||
} else if (query.getExclude().equalsIgnoreCase(TimeBoundaryQuery.MIN_TIME)) {
|
||||
} else if (query.getBound().equalsIgnoreCase(TimeBoundaryQuery.MAX_TIME)) {
|
||||
filterPredicate = new Predicate<T>()
|
||||
{
|
||||
@Override
|
||||
|
|
|
@ -20,13 +20,10 @@
|
|||
package io.druid.query.timeboundary;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import io.druid.query.Druids;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryRunnerTestHelper;
|
||||
import io.druid.query.QueryWatcher;
|
||||
import io.druid.query.Result;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.Assert;
|
||||
|
@ -81,11 +78,11 @@ public class TimeBoundaryQueryRunnerTest
|
|||
|
||||
@Test
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testTimeBoundaryExcludesMin()
|
||||
public void testTimeBoundaryMax()
|
||||
{
|
||||
TimeBoundaryQuery timeBoundaryQuery = Druids.newTimeBoundaryQueryBuilder()
|
||||
.dataSource("testing")
|
||||
.exclude(TimeBoundaryQuery.MIN_TIME)
|
||||
.bound(TimeBoundaryQuery.MAX_TIME)
|
||||
.build();
|
||||
|
||||
Iterable<Result<TimeBoundaryResultValue>> results = Sequences.toList(
|
||||
|
@ -102,11 +99,11 @@ public class TimeBoundaryQueryRunnerTest
|
|||
|
||||
@Test
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testTimeBoundaryExcludesMax()
|
||||
public void testTimeBoundaryMin()
|
||||
{
|
||||
TimeBoundaryQuery timeBoundaryQuery = Druids.newTimeBoundaryQueryBuilder()
|
||||
.dataSource("testing")
|
||||
.exclude(TimeBoundaryQuery.MAX_TIME)
|
||||
.bound(TimeBoundaryQuery.MIN_TIME)
|
||||
.build();
|
||||
|
||||
Iterable<Result<TimeBoundaryResultValue>> results = Sequences.toList(
|
||||
|
|
Loading…
Reference in New Issue