mirror of https://github.com/apache/druid.git
Make 'search' filter have a case sensitive option(#1878)
This commit is contained in:
parent
25a0eb7ed5
commit
032c3e986d
|
@ -36,6 +36,8 @@ import io.druid.query.filter.SelectorDimFilter;
|
|||
import io.druid.query.metadata.metadata.ColumnIncluderator;
|
||||
import io.druid.query.metadata.metadata.SegmentMetadataQuery;
|
||||
import io.druid.query.search.SearchResultValue;
|
||||
import io.druid.query.search.search.ContainsSearchQuerySpec;
|
||||
import io.druid.query.search.search.FragmentSearchQuerySpec;
|
||||
import io.druid.query.search.search.InsensitiveContainsSearchQuerySpec;
|
||||
import io.druid.query.search.search.SearchQuery;
|
||||
import io.druid.query.search.search.SearchQuerySpec;
|
||||
|
@ -691,6 +693,29 @@ public class Druids
|
|||
return this;
|
||||
}
|
||||
|
||||
public SearchQueryBuilder query(String q, boolean caseSensitive)
|
||||
{
|
||||
querySpec = new ContainsSearchQuerySpec(q, caseSensitive);
|
||||
return this;
|
||||
}
|
||||
|
||||
public SearchQueryBuilder query(Map<String, Object> q, boolean caseSensitive)
|
||||
{
|
||||
querySpec = new ContainsSearchQuerySpec((String) q.get("value"), caseSensitive);
|
||||
return this;
|
||||
}
|
||||
|
||||
public SearchQueryBuilder fragments(List<String> q)
|
||||
{
|
||||
return fragments(q, false);
|
||||
}
|
||||
|
||||
public SearchQueryBuilder fragments(List<String> q, boolean caseSensitive)
|
||||
{
|
||||
querySpec = new FragmentSearchQuerySpec(q, caseSensitive);
|
||||
return this;
|
||||
}
|
||||
|
||||
public SearchQueryBuilder context(Map<String, Object> c)
|
||||
{
|
||||
context = c;
|
||||
|
|
|
@ -0,0 +1,102 @@
|
|||
package io.druid.query.search.search;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.metamx.common.StringUtils;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ContainsSearchQuerySpec implements SearchQuerySpec
|
||||
{
|
||||
|
||||
private static final byte CACHE_TYPE_ID = 0x1;
|
||||
|
||||
private final String value;
|
||||
private final boolean caseSensitive;
|
||||
|
||||
private final String target;
|
||||
|
||||
@JsonCreator
|
||||
public ContainsSearchQuerySpec(
|
||||
@JsonProperty("value") String value,
|
||||
@JsonProperty("caseSensitive") boolean caseSensitive
|
||||
)
|
||||
{
|
||||
this.value = value;
|
||||
this.caseSensitive = caseSensitive;
|
||||
this.target = value == null || caseSensitive ? value : value.toLowerCase();
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getValue()
|
||||
{
|
||||
return value;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public boolean isCaseSensitive()
|
||||
{
|
||||
return caseSensitive;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean accept(String dimVal)
|
||||
{
|
||||
if (dimVal == null) {
|
||||
return false;
|
||||
}
|
||||
final String input = caseSensitive ? dimVal : dimVal.toLowerCase();
|
||||
return input.contains(target);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
byte[] valueBytes = StringUtils.toUtf8(value);
|
||||
|
||||
return ByteBuffer.allocate(2 + valueBytes.length)
|
||||
.put(caseSensitive ? (byte)1 : 0)
|
||||
.put(CACHE_TYPE_ID)
|
||||
.put(valueBytes)
|
||||
.array();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "ContainsSearchQuerySpec{" +
|
||||
"value=" + value + ", caseSensitive=" + caseSensitive +
|
||||
"}";
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
ContainsSearchQuerySpec that = (ContainsSearchQuerySpec) o;
|
||||
|
||||
if (caseSensitive ^ that.caseSensitive) {
|
||||
return false;
|
||||
}
|
||||
|
||||
if (value != null ? !value.equals(that.value) : that.value != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return value != null ? value.hashCode() : 0;
|
||||
}
|
||||
}
|
|
@ -22,7 +22,9 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.metamx.common.StringUtils;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -31,13 +33,32 @@ public class FragmentSearchQuerySpec implements SearchQuerySpec
|
|||
private static final byte CACHE_TYPE_ID = 0x2;
|
||||
|
||||
private final List<String> values;
|
||||
private final boolean caseSensitive;
|
||||
|
||||
private final String[] target;
|
||||
|
||||
@JsonCreator
|
||||
public FragmentSearchQuerySpec(
|
||||
@JsonProperty("values") List<String> values
|
||||
) {
|
||||
this(values, false);
|
||||
}
|
||||
|
||||
@JsonCreator
|
||||
public FragmentSearchQuerySpec(
|
||||
@JsonProperty("values") List<String> values,
|
||||
@JsonProperty("caseSensitive") boolean caseSensitive
|
||||
)
|
||||
{
|
||||
this.values = values;
|
||||
this.caseSensitive = caseSensitive;
|
||||
Set<String> set = new HashSet();
|
||||
if (values != null) {
|
||||
for (String value : values) {
|
||||
set.add(caseSensitive ? value : value.toLowerCase());
|
||||
}
|
||||
}
|
||||
target = set.toArray(new String[set.size()]);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -46,11 +67,21 @@ public class FragmentSearchQuerySpec implements SearchQuerySpec
|
|||
return values;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public boolean isCaseSensitive()
|
||||
{
|
||||
return caseSensitive;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean accept(String dimVal)
|
||||
{
|
||||
for (String value : values) {
|
||||
if (dimVal == null || !dimVal.toLowerCase().contains(value.toLowerCase())) {
|
||||
if (dimVal == null) {
|
||||
return false;
|
||||
}
|
||||
final String input = caseSensitive ? dimVal : dimVal.toLowerCase();
|
||||
for (String value : target) {
|
||||
if (!input.contains(value)) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
@ -69,8 +100,9 @@ public class FragmentSearchQuerySpec implements SearchQuerySpec
|
|||
++index;
|
||||
}
|
||||
|
||||
final ByteBuffer queryCacheKey = ByteBuffer.allocate(1 + valuesBytesSize)
|
||||
.put(CACHE_TYPE_ID);
|
||||
final ByteBuffer queryCacheKey = ByteBuffer.allocate(2 + valuesBytesSize)
|
||||
.put(caseSensitive ? (byte) 1 : 0)
|
||||
.put(CACHE_TYPE_ID);
|
||||
|
||||
for (byte[] bytes : valuesBytes) {
|
||||
queryCacheKey.put(bytes);
|
||||
|
@ -83,7 +115,7 @@ public class FragmentSearchQuerySpec implements SearchQuerySpec
|
|||
public String toString()
|
||||
{
|
||||
return "FragmentSearchQuerySpec{" +
|
||||
"values=" + values +
|
||||
"values=" + values + ", caseSensitive=" + caseSensitive +
|
||||
"}";
|
||||
}
|
||||
|
||||
|
@ -99,6 +131,10 @@ public class FragmentSearchQuerySpec implements SearchQuerySpec
|
|||
|
||||
FragmentSearchQuerySpec that = (FragmentSearchQuerySpec) o;
|
||||
|
||||
if (caseSensitive ^ that.caseSensitive) {
|
||||
return false;
|
||||
}
|
||||
|
||||
if (values != null ? !values.equals(that.values) : that.values != null) {
|
||||
return false;
|
||||
}
|
||||
|
|
|
@ -19,57 +19,24 @@ package io.druid.query.search.search;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.metamx.common.StringUtils;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class InsensitiveContainsSearchQuerySpec implements SearchQuerySpec
|
||||
public class InsensitiveContainsSearchQuerySpec extends ContainsSearchQuerySpec
|
||||
{
|
||||
private static final byte CACHE_TYPE_ID = 0x1;
|
||||
|
||||
private final String value;
|
||||
|
||||
@JsonCreator
|
||||
public InsensitiveContainsSearchQuerySpec(
|
||||
@JsonProperty("value") String value
|
||||
)
|
||||
{
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getValue()
|
||||
{
|
||||
return value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean accept(String dimVal)
|
||||
{
|
||||
if (dimVal == null) {
|
||||
return false;
|
||||
}
|
||||
return dimVal.toLowerCase().contains(value.toLowerCase());
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
byte[] valueBytes = StringUtils.toUtf8(value);
|
||||
|
||||
return ByteBuffer.allocate(1 + valueBytes.length)
|
||||
.put(CACHE_TYPE_ID)
|
||||
.put(valueBytes)
|
||||
.array();
|
||||
super(value, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "InsensitiveContainsSearchQuerySpec{" +
|
||||
"value=" + value +
|
||||
"value=" + getValue() +
|
||||
"}";
|
||||
}
|
||||
|
||||
|
@ -82,19 +49,6 @@ public class InsensitiveContainsSearchQuerySpec implements SearchQuerySpec
|
|||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
InsensitiveContainsSearchQuerySpec that = (InsensitiveContainsSearchQuerySpec) o;
|
||||
|
||||
if (value != null ? !value.equals(that.value) : that.value != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return value != null ? value.hashCode() : 0;
|
||||
return super.equals(o);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
|||
*/
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "contains", value = ContainsSearchQuerySpec.class),
|
||||
@JsonSubTypes.Type(name = "insensitive_contains", value = InsensitiveContainsSearchQuerySpec.class),
|
||||
@JsonSubTypes.Type(name = "fragment", value = FragmentSearchQuerySpec.class)
|
||||
})
|
||||
|
|
|
@ -332,7 +332,7 @@ public class QueryRunnerTestHelper
|
|||
{
|
||||
return new FinalizeResultsQueryRunner<T>(
|
||||
new BySegmentQueryRunner<T>(
|
||||
segmentId, adapter.getDataInterval().getStart(),
|
||||
adapter.getIdentifier(), adapter.getDataInterval().getStart(),
|
||||
factory.createRunner(adapter)
|
||||
),
|
||||
(QueryToolChest<T, Query<T>>)factory.getToolchest()
|
||||
|
|
|
@ -0,0 +1,196 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright 2012 - 2015 Metamarkets Group Inc.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package io.druid.query.search;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.io.CharSource;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import io.druid.query.Druids;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.Result;
|
||||
import io.druid.query.search.search.SearchHit;
|
||||
import io.druid.query.search.search.SearchQuery;
|
||||
import io.druid.query.search.search.SearchQueryConfig;
|
||||
import io.druid.segment.IncrementalIndexSegment;
|
||||
import io.druid.segment.QueryableIndex;
|
||||
import io.druid.segment.QueryableIndexSegment;
|
||||
import io.druid.segment.TestIndex;
|
||||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import static io.druid.query.QueryRunnerTestHelper.*;
|
||||
|
||||
/**
|
||||
*/
|
||||
@RunWith(Parameterized.class)
|
||||
public class SearchQueryRunnerWithCaseTest
|
||||
{
|
||||
@Parameterized.Parameters
|
||||
public static Iterable<Object[]> constructorFeeder() throws IOException
|
||||
{
|
||||
SearchQueryRunnerFactory factory = new SearchQueryRunnerFactory(
|
||||
new SearchQueryQueryToolChest(
|
||||
new SearchQueryConfig(),
|
||||
NoopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
NOOP_QUERYWATCHER
|
||||
);
|
||||
|
||||
CharSource input = CharSource.wrap(
|
||||
"2011-01-12T00:00:00.000Z\tspot\tAutoMotive\tPREFERRED\ta\u0001preferred\t100.000000\n" +
|
||||
"2011-01-12T00:00:00.000Z\tSPot\tbusiness\tpreferred\tb\u0001Preferred\t100.000000\n" +
|
||||
"2011-01-12T00:00:00.000Z\tspot\tentertainment\tPREFERRed\te\u0001preferred\t100.000000\n" +
|
||||
"2011-01-13T00:00:00.000Z\tspot\tautomotive\tpreferred\ta\u0001preferred\t94.874713");
|
||||
|
||||
IncrementalIndex index1 = TestIndex.makeRealtimeIndex(input, true);
|
||||
IncrementalIndex index2 = TestIndex.makeRealtimeIndex(input, false);
|
||||
|
||||
QueryableIndex index3 = TestIndex.persistRealtimeAndLoadMMapped(index1);
|
||||
QueryableIndex index4 = TestIndex.persistRealtimeAndLoadMMapped(index2);
|
||||
|
||||
return transformToConstructionFeeder(Arrays.asList(
|
||||
makeQueryRunner(factory, new IncrementalIndexSegment(index1, "index1")),
|
||||
makeQueryRunner(factory, new IncrementalIndexSegment(index2, "index2")),
|
||||
makeQueryRunner(factory, new QueryableIndexSegment("index3", index3)),
|
||||
makeQueryRunner(factory, new QueryableIndexSegment("index4", index4))
|
||||
));
|
||||
}
|
||||
|
||||
private final QueryRunner runner;
|
||||
|
||||
public SearchQueryRunnerWithCaseTest(
|
||||
QueryRunner runner
|
||||
)
|
||||
{
|
||||
this.runner = runner;
|
||||
}
|
||||
|
||||
private Druids.SearchQueryBuilder testBuilder() {
|
||||
return Druids.newSearchQueryBuilder()
|
||||
.dataSource(dataSource)
|
||||
.granularity(allGran)
|
||||
.intervals(fullOnInterval);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSearch()
|
||||
{
|
||||
Druids.SearchQueryBuilder builder = testBuilder();
|
||||
Map<String, Set<String>> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
|
||||
SearchQuery searchQuery;
|
||||
|
||||
searchQuery = builder.query("SPOT").build();
|
||||
expectedResults.put(marketDimension, Sets.newHashSet("spot", "SPot"));
|
||||
checkSearchQuery(searchQuery, expectedResults);
|
||||
|
||||
searchQuery = builder.query("spot", true).build();
|
||||
expectedResults.put(marketDimension, Sets.newHashSet("spot"));
|
||||
checkSearchQuery(searchQuery, expectedResults);
|
||||
|
||||
searchQuery = builder.query("SPot", true).build();
|
||||
expectedResults.put(marketDimension, Sets.newHashSet("SPot"));
|
||||
checkSearchQuery(searchQuery, expectedResults);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSearchSameValueInMultiDims()
|
||||
{
|
||||
SearchQuery searchQuery;
|
||||
Druids.SearchQueryBuilder builder = testBuilder()
|
||||
.dimensions(Arrays.asList(placementDimension, placementishDimension));
|
||||
Map<String, Set<String>> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
|
||||
|
||||
searchQuery = builder.query("PREFERRED").build();
|
||||
expectedResults.put(placementDimension, Sets.newHashSet("PREFERRED", "preferred", "PREFERRed"));
|
||||
expectedResults.put(placementishDimension, Sets.newHashSet("preferred", "Preferred"));
|
||||
checkSearchQuery(searchQuery, expectedResults);
|
||||
|
||||
searchQuery = builder.query("preferred", true).build();
|
||||
expectedResults.put(placementDimension, Sets.newHashSet("preferred"));
|
||||
expectedResults.put(placementishDimension, Sets.newHashSet("preferred"));
|
||||
checkSearchQuery(searchQuery, expectedResults);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFragmentSearch()
|
||||
{
|
||||
Druids.SearchQueryBuilder builder = testBuilder();
|
||||
Map<String, Set<String>> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
|
||||
SearchQuery searchQuery;
|
||||
|
||||
searchQuery = builder.fragments(Arrays.asList("auto", "ve")).build();
|
||||
expectedResults.put(qualityDimension, Sets.newHashSet("automotive", "AutoMotive"));
|
||||
checkSearchQuery(searchQuery, expectedResults);
|
||||
|
||||
searchQuery = builder.fragments(Arrays.asList("auto", "ve"), true).build();
|
||||
expectedResults.put(qualityDimension, Sets.newHashSet("automotive"));
|
||||
checkSearchQuery(searchQuery, expectedResults);
|
||||
}
|
||||
|
||||
private void checkSearchQuery(SearchQuery searchQuery, Map<String, Set<String>> expectedResults)
|
||||
{
|
||||
HashMap<String,List> context = new HashMap<>();
|
||||
Iterable<Result<SearchResultValue>> results = Sequences.toList(
|
||||
runner.run(searchQuery, context),
|
||||
Lists.<Result<SearchResultValue>>newArrayList()
|
||||
);
|
||||
|
||||
for (Result<SearchResultValue> result : results) {
|
||||
Assert.assertEquals(new DateTime("2011-01-12T00:00:00.000Z"), result.getTimestamp());
|
||||
Assert.assertNotNull(result.getValue());
|
||||
|
||||
Iterable<SearchHit> resultValues = result.getValue();
|
||||
for (SearchHit resultValue : resultValues) {
|
||||
String dimension = resultValue.getDimension();
|
||||
String theValue = resultValue.getValue();
|
||||
Assert.assertTrue(
|
||||
String.format("Result had unknown dimension[%s]", dimension),
|
||||
expectedResults.containsKey(dimension)
|
||||
);
|
||||
|
||||
Set<String> expectedSet = expectedResults.get(dimension);
|
||||
Assert.assertTrue(
|
||||
String.format("Couldn't remove dim[%s], value[%s]", dimension, theValue), expectedSet.remove(theValue)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
for (Map.Entry<String, Set<String>> entry : expectedResults.entrySet()) {
|
||||
Assert.assertTrue(
|
||||
String.format(
|
||||
"Dimension[%s] should have had everything removed, still has[%s]", entry.getKey(), entry.getValue()
|
||||
),
|
||||
entry.getValue().isEmpty()
|
||||
);
|
||||
}
|
||||
expectedResults.clear();
|
||||
}
|
||||
}
|
|
@ -22,9 +22,9 @@ package io.druid.segment;
|
|||
import com.google.common.base.Charsets;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.hash.Hashing;
|
||||
import com.google.common.io.CharStreams;
|
||||
import com.google.common.io.InputSupplier;
|
||||
import com.google.common.io.CharSource;
|
||||
import com.google.common.io.LineProcessor;
|
||||
import com.google.common.io.Resources;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.data.input.impl.DelimitedParseSpec;
|
||||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
|
@ -46,7 +46,6 @@ import org.joda.time.Interval;
|
|||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.URL;
|
||||
import java.util.Arrays;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
@ -163,10 +162,18 @@ public class TestIndex
|
|||
}
|
||||
}
|
||||
|
||||
private static IncrementalIndex makeRealtimeIndex(final String resourceFilename, final boolean useOffheap)
|
||||
{
|
||||
private static IncrementalIndex makeRealtimeIndex(final String resourceFilename, final boolean useOffheap) {
|
||||
final URL resource = TestIndex.class.getClassLoader().getResource(resourceFilename);
|
||||
if (resource == null) {
|
||||
throw new IllegalArgumentException("cannot find resource " + resourceFilename);
|
||||
}
|
||||
log.info("Realtime loading index file[%s]", resource);
|
||||
CharSource stream = Resources.asByteSource(resource).asCharSource(Charsets.UTF_8);
|
||||
return makeRealtimeIndex(stream, useOffheap);
|
||||
}
|
||||
|
||||
public static IncrementalIndex makeRealtimeIndex(final CharSource source, final boolean useOffheap)
|
||||
{
|
||||
final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder()
|
||||
.withMinTimestamp(new DateTime("2011-01-12T00:00:00.000Z").getMillis())
|
||||
.withQueryGranularity(QueryGranularity.NONE)
|
||||
|
@ -190,20 +197,8 @@ public class TestIndex
|
|||
final AtomicLong startTime = new AtomicLong();
|
||||
int lineCount;
|
||||
try {
|
||||
lineCount = CharStreams.readLines(
|
||||
CharStreams.newReaderSupplier(
|
||||
new InputSupplier<InputStream>()
|
||||
{
|
||||
@Override
|
||||
public InputStream getInput() throws IOException
|
||||
{
|
||||
return resource.openStream();
|
||||
}
|
||||
},
|
||||
Charsets.UTF_8
|
||||
),
|
||||
new LineProcessor<Integer>()
|
||||
{
|
||||
lineCount = source.readLines(
|
||||
new LineProcessor<Integer>() {
|
||||
StringInputRowParser parser = new StringInputRowParser(
|
||||
new DelimitedParseSpec(
|
||||
new TimestampSpec("ts", "iso", null),
|
||||
|
|
Loading…
Reference in New Issue