Move common TypeReference into JacksonUtils (#4738)

This commit is contained in:
Charles Allen 2017-08-31 13:40:16 -07:00 committed by GitHub
parent 9078925cab
commit bdfc6fe25e
23 changed files with 168 additions and 159 deletions

View File

@ -19,7 +19,6 @@
package io.druid.timeline;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
@ -28,6 +27,7 @@ import com.google.common.collect.Range;
import com.google.common.collect.Sets;
import io.druid.TestObjectMapper;
import io.druid.data.input.InputRow;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.Intervals;
import io.druid.timeline.partition.NoneShardSpec;
@ -108,9 +108,7 @@ public class DataSegmentTest
final Map<String, Object> objectMap = mapper.readValue(
mapper.writeValueAsString(segment),
new TypeReference<Map<String, Object>>()
{
}
JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
Assert.assertEquals(10, objectMap.size());

View File

@ -26,7 +26,6 @@ import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.fasterxml.jackson.core.JsonFactory;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
@ -38,6 +37,7 @@ import io.druid.guice.annotations.Json;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.UOE;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.java.util.common.parsers.CSVParser;
import io.druid.java.util.common.parsers.DelimitedParser;
import io.druid.java.util.common.parsers.JSONParser;
@ -591,10 +591,6 @@ public class UriExtractionNamespace implements ExtractionNamespace
@JsonTypeName("simpleJson")
public static class ObjectMapperFlatDataParser implements FlatDataParser
{
private static final TypeReference<Map<String, String>> MAP_STRING_STRING = new TypeReference<Map<String, String>>()
{
};
private final Parser<String, String> parser;
@JsonCreator
@ -612,7 +608,7 @@ public class UriExtractionNamespace implements ExtractionNamespace
public Map<String, String> parse(String input)
{
try {
return jsonFactory.createParser(input).readValueAs(MAP_STRING_STRING);
return jsonFactory.createParser(input).readValueAs(JacksonUtils.TYPE_REFERENCE_MAP_STRING_STRING);
}
catch (IOException e) {
throw Throwables.propagate(e);

View File

@ -19,7 +19,6 @@
package io.druid.query.lookup;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.BeanProperty;
import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.InjectableValues;
@ -37,6 +36,7 @@ import io.druid.guice.annotations.Json;
import io.druid.guice.annotations.Self;
import io.druid.initialization.Initialization;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.java.util.common.ISE;
import io.druid.query.lookup.namespace.ExtractionNamespace;
import io.druid.query.lookup.namespace.UriExtractionNamespace;
@ -462,9 +462,7 @@ public class NamespaceLookupExtractorFactoryTest
Assert.assertFalse(namespaceLookupExtractorFactory.replaces(mapper.readValue(str, LookupExtractorFactory.class)));
final Map<String, Object> map = new HashMap<>(mapper.<Map<String, Object>>readValue(
str,
new TypeReference<Map<String, Object>>()
{
}
JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
));
map.put("firstCacheTimeout", "1");
Assert.assertTrue(namespaceLookupExtractorFactory.replaces(mapper.convertValue(map, LookupExtractorFactory.class)));

View File

@ -22,7 +22,6 @@ package io.druid.indexer;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Joiner;
@ -45,6 +44,7 @@ import io.druid.guice.annotations.Self;
import io.druid.indexer.partitions.PartitionsSpec;
import io.druid.indexer.path.PathSpec;
import io.druid.initialization.Initialization;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.JodaUtils;
import io.druid.java.util.common.StringUtils;
@ -160,10 +160,8 @@ public class HadoopDruidIndexerConfig
{
try {
return fromMap(
(Map<String, Object>) HadoopDruidIndexerConfig.JSON_MAPPER.readValue(
file, new TypeReference<Map<String, Object>>()
{
}
HadoopDruidIndexerConfig.JSON_MAPPER.readValue(
file, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
)
);
}
@ -178,10 +176,8 @@ public class HadoopDruidIndexerConfig
// This is a map to try and prevent dependency screwbally-ness
try {
return fromMap(
(Map<String, Object>) HadoopDruidIndexerConfig.JSON_MAPPER.readValue(
str, new TypeReference<Map<String, Object>>()
{
}
HadoopDruidIndexerConfig.JSON_MAPPER.readValue(
str, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
)
);
}
@ -199,10 +195,8 @@ public class HadoopDruidIndexerConfig
Reader reader = new InputStreamReader(fs.open(pt), StandardCharsets.UTF_8);
return fromMap(
(Map<String, Object>) HadoopDruidIndexerConfig.JSON_MAPPER.readValue(
reader, new TypeReference<Map<String, Object>>()
{
}
HadoopDruidIndexerConfig.JSON_MAPPER.readValue(
reader, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
)
);
}

View File

@ -19,9 +19,9 @@
package io.druid.indexer;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.java.util.common.ISE;
import org.apache.hadoop.fs.FileSystem;
@ -110,9 +110,7 @@ public class Utils
return jsonMapper.readValue(
fs.open(statsPath),
new TypeReference<Map<String, Object>>()
{
}
JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
}

View File

@ -21,7 +21,6 @@ package io.druid.indexer.updater;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
@ -34,6 +33,7 @@ import io.druid.guice.GuiceInjectors;
import io.druid.guice.JsonConfigProvider;
import io.druid.guice.annotations.Self;
import io.druid.initialization.Initialization;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMerger;
import io.druid.segment.IndexSpec;
@ -81,18 +81,14 @@ public class HadoopDruidConverterConfig
DATA_SEGMENT_PUSHER = injector.getInstance(DataSegmentPusher.class);
}
private static final TypeReference<Map<String, Object>> mapTypeReference = new TypeReference<Map<String, Object>>()
{
};
public static HadoopDruidConverterConfig fromString(final String string) throws IOException
{
return fromMap(jsonMapper.<Map<String, Object>>readValue(string, mapTypeReference));
return fromMap(jsonMapper.readValue(string, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT));
}
public static HadoopDruidConverterConfig fromFile(final File file) throws IOException
{
return fromMap(jsonMapper.<Map<String, Object>>readValue(file, mapTypeReference));
return fromMap(jsonMapper.readValue(file, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT));
}
public static HadoopDruidConverterConfig fromMap(final Map<String, Object> map)

View File

@ -19,7 +19,6 @@
package io.druid.indexing.common.actions;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Charsets;
import com.google.common.base.Throwables;
@ -33,6 +32,7 @@ import io.druid.indexing.common.RetryPolicy;
import io.druid.indexing.common.RetryPolicyFactory;
import io.druid.indexing.common.task.Task;
import io.druid.java.util.common.IOE;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.logger.Logger;
import org.jboss.netty.channel.ChannelException;
@ -114,9 +114,7 @@ public class RemoteTaskActionClient implements TaskActionClient
if (response.getStatus().getCode() / 100 == 2) {
final Map<String, Object> responseDict = jsonMapper.readValue(
response.getContent(),
new TypeReference<Map<String, Object>>()
{
}
JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
return jsonMapper.convertValue(responseDict.get("result"), taskAction.getReturnTypeReference());
} else {

View File

@ -20,7 +20,6 @@
package io.druid.server.initialization;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import com.google.inject.Injector;
@ -32,6 +31,7 @@ import io.druid.guice.JsonConfigurator;
import io.druid.initialization.Initialization;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.jackson.JacksonUtils;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
@ -267,9 +267,7 @@ public class IndexerZkConfigTest
);
Map<String, String> value = mapper.readValue(
mapper.writeValueAsString(indexerZkConfig), new TypeReference<Map<String, String>>()
{
}
mapper.writeValueAsString(indexerZkConfig), JacksonUtils.TYPE_REFERENCE_MAP_STRING_STRING
);
IndexerZkConfig newConfig = new IndexerZkConfig(
zkPathsConfig,

View File

@ -21,9 +21,9 @@ package io.druid.testing;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.java.util.common.logger.Logger;
import java.io.File;
@ -56,9 +56,7 @@ public class ConfigFileConfigProvider implements IntegrationTestingConfigProvide
ObjectMapper jsonMapper = new ObjectMapper();
try {
props = jsonMapper.readValue(
new File(configFile), new TypeReference<Map<String, String>>()
{
}
new File(configFile), JacksonUtils.TYPE_REFERENCE_MAP_STRING_STRING
);
}
catch (IOException ex) {

View File

@ -28,6 +28,7 @@ import com.metamx.http.client.HttpClient;
import com.metamx.http.client.Request;
import com.metamx.http.client.response.StatusResponseHandler;
import com.metamx.http.client.response.StatusResponseHolder;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils;
import io.druid.testing.guice.TestClient;
@ -144,10 +145,8 @@ public class EventReceiverFirehoseTestClient
int expectedEventsPosted = 0;
while ((s = reader.readLine()) != null) {
events.add(
(Map<String, Object>) this.jsonMapper.readValue(
s, new TypeReference<Map<String, Object>>()
{
}
this.jsonMapper.readValue(
s, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
)
);
ObjectMapper mapper = (totalEventsPosted % 2 == 0) ? jsonMapper : smileMapper;

View File

@ -33,6 +33,7 @@ import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.task.Task;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.RetryUtils;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import io.druid.testing.IntegrationTestingConfig;
@ -111,9 +112,7 @@ public class OverlordResourceTestClient
);
}
Map<String, String> responseData = jsonMapper.readValue(
response.getContent(), new TypeReference<Map<String, String>>()
{
}
response.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_STRING
);
String taskID = responseData.get("task");
LOG.info("Submitted task with TaskID[%s]", taskID);
@ -143,9 +142,7 @@ public class OverlordResourceTestClient
LOG.info("Index status response" + response.getContent());
Map<String, Object> responseData = jsonMapper.readValue(
response.getContent(), new TypeReference<Map<String, Object>>()
{
}
response.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
//TODO: figure out a better way to parse the response...
String status = (String) ((Map) responseData.get("status")).get("status");
@ -202,9 +199,7 @@ public class OverlordResourceTestClient
);
LOG.info("Shutdown Task %s response %s", taskID, response.getContent());
return jsonMapper.readValue(
response.getContent(), new TypeReference<Map<String, String>>()
{
}
response.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_STRING
);
}
catch (Exception e) {
@ -258,9 +253,7 @@ public class OverlordResourceTestClient
);
}
Map<String, String> responseData = jsonMapper.readValue(
response.getContent(), new TypeReference<Map<String, String>>()
{
}
response.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_STRING
);
String id = responseData.get("id");
LOG.info("Submitted supervisor with id[%s]", id);

View File

@ -19,7 +19,6 @@
package io.druid.tests.indexer;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.base.Throwables;
import com.google.inject.Inject;
import com.metamx.http.client.HttpClient;
@ -28,6 +27,7 @@ import io.druid.curator.discovery.ServerDiscoverySelector;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.java.util.common.logger.Logger;
import io.druid.testing.IntegrationTestingConfig;
import io.druid.testing.clients.EventReceiverFirehoseTestClient;
@ -227,10 +227,8 @@ public class ITRealtimeIndexTaskTest extends AbstractIndexerTest
LOG.info("sending event: [%s]\n", event);
Collection<Map<String, Object>> events = new ArrayList<Map<String, Object>>();
events.add(
(Map<String, Object>) this.jsonMapper.readValue(
event, new TypeReference<Map<String, Object>>()
{
}
this.jsonMapper.readValue(
event, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
)
);
int eventsPosted = client.postEvents(events, this.jsonMapper, MediaType.APPLICATION_JSON);

View File

@ -0,0 +1,34 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you 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.java.util.common.jackson;
import com.fasterxml.jackson.core.type.TypeReference;
import java.util.Map;
public class JacksonUtils
{
public static final TypeReference<Map<String, Object>> TYPE_REFERENCE_MAP_STRING_OBJECT = new TypeReference<Map<String, Object>>()
{
};
public static final TypeReference<Map<String, String>> TYPE_REFERENCE_MAP_STRING_STRING = new TypeReference<Map<String, String>>()
{
};
}

View File

@ -19,7 +19,6 @@
package io.druid.query.datasourcemetadata;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
@ -30,6 +29,7 @@ import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.guava.Sequences;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.query.DefaultGenericQueryMetricsFactory;
import io.druid.query.Druids;
import io.druid.query.GenericQueryMetricsFactory;
@ -156,50 +156,50 @@ public class DataSourceMetadataQueryTest
DataSourceQueryQueryToolChest toolChest = new DataSourceQueryQueryToolChest(queryMetricsFactory);
List<LogicalSegment> segments = toolChest
.filterSegments(
null,
Arrays.asList(
new LogicalSegment()
{
@Override
public Interval getInterval()
{
return Intervals.of("2012-01-01/P1D");
}
},
new LogicalSegment()
{
@Override
public Interval getInterval()
{
return Intervals.of("2012-01-01T01/PT1H");
}
},
new LogicalSegment()
{
@Override
public Interval getInterval()
{
return Intervals.of("2013-01-01/P1D");
}
},
new LogicalSegment()
{
@Override
public Interval getInterval()
{
return Intervals.of("2013-01-01T01/PT1H");
}
},
new LogicalSegment()
{
@Override
public Interval getInterval()
{
return Intervals.of("2013-01-01T02/PT1H");
}
}
)
);
null,
Arrays.asList(
new LogicalSegment()
{
@Override
public Interval getInterval()
{
return Intervals.of("2012-01-01/P1D");
}
},
new LogicalSegment()
{
@Override
public Interval getInterval()
{
return Intervals.of("2012-01-01T01/PT1H");
}
},
new LogicalSegment()
{
@Override
public Interval getInterval()
{
return Intervals.of("2013-01-01/P1D");
}
},
new LogicalSegment()
{
@Override
public Interval getInterval()
{
return Intervals.of("2013-01-01T01/PT1H");
}
},
new LogicalSegment()
{
@Override
public Interval getInterval()
{
return Intervals.of("2013-01-01T02/PT1H");
}
}
)
);
Assert.assertEquals(segments.size(), 2);
// should only have the latest segments.
@ -233,9 +233,7 @@ public class DataSourceMetadataQueryTest
final DataSourceMetadataResultValue resultValue = new DataSourceMetadataResultValue(DateTimes.of("2000-01-01T00Z"));
final Map<String, Object> resultValueMap = new DefaultObjectMapper().convertValue(
resultValue,
new TypeReference<Map<String, Object>>()
{
}
JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
Assert.assertEquals(
ImmutableMap.<String, Object>of("maxIngestedEventTime", "2000-01-01T00:00:00.000Z"),

View File

@ -19,10 +19,10 @@
package io.druid.query.lookup;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.query.extraction.MapLookupExtractor;
import org.easymock.EasyMock;
import org.junit.Assert;
@ -123,11 +123,8 @@ public class RegisteredLookupExtractionFnTest
);
EasyMock.verify(manager);
final TypeReference<Map<String, Object>> typeReference = new TypeReference<Map<String, Object>>()
{
};
final Map<String, Object> result = mapper.readValue(mapper.writeValueAsString(fn), typeReference);
Assert.assertEquals(mapper.convertValue(fn, typeReference), result);
final Map<String, Object> result = mapper.readValue(mapper.writeValueAsString(fn), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT);
Assert.assertEquals(mapper.convertValue(fn, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT), result);
Assert.assertEquals(LOOKUP_NAME, result.get("lookup"));
Assert.assertEquals(true, result.get("retainMissingValue"));
Assert.assertEquals(true, result.get("injective"));

View File

@ -22,7 +22,6 @@ package io.druid.client;
import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.core.JsonToken;
import com.fasterxml.jackson.core.ObjectCodec;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.JavaType;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.type.TypeFactory;
@ -51,6 +50,7 @@ import io.druid.java.util.common.guava.BaseSequence;
import io.druid.java.util.common.guava.CloseQuietly;
import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.java.util.common.logger.Logger;
import io.druid.query.BySegmentResultValueClass;
import io.druid.query.Query;
@ -117,7 +117,10 @@ public class DirectDruidClient<T> implements QueryRunner<T>
private final AtomicInteger openConnections;
private final boolean isSmile;
public static <T, QueryType extends Query<T>> QueryType withDefaultTimeoutAndMaxScatterGatherBytes(final QueryType query, ServerConfig serverConfig)
public static <T, QueryType extends Query<T>> QueryType withDefaultTimeoutAndMaxScatterGatherBytes(
final QueryType query,
ServerConfig serverConfig
)
{
return (QueryType) QueryContexts.withMaxScatterGatherBytes(
QueryContexts.withDefaultTimeout(
@ -251,9 +254,7 @@ public class DirectDruidClient<T> implements QueryRunner<T>
if (responseContext != null) {
context.putAll(
objectMapper.<Map<String, Object>>readValue(
responseContext, new TypeReference<Map<String, Object>>()
{
}
responseContext, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
)
);
}

View File

@ -27,6 +27,7 @@ import com.google.common.base.Function;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import io.druid.common.utils.ServletResourceUtils;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.java.util.common.logger.Logger;
import javax.annotation.Nullable;
@ -82,9 +83,7 @@ public abstract class AbstractListenerHandler<ObjType> implements ListenerHandle
try {
// This actually fails to properly convert due to type erasure. We'll try again in a second
// This effectively just parses
final Map<String, Object> tempMap = mapper.readValue(inputStream, new TypeReference<Map<String, Object>>()
{
});
final Map<String, Object> tempMap = mapper.readValue(inputStream, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT);
// Now do the ACTUAL conversion
inObjMap = ImmutableMap.copyOf(Maps.transformValues(
tempMap,

View File

@ -19,7 +19,6 @@
package io.druid.client;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
@ -28,6 +27,7 @@ import com.google.common.collect.Sets;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.segment.IndexIO;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
@ -67,7 +67,10 @@ public class DataSegmentTest
1
);
final Map<String, Object> objectMap = mapper.readValue(mapper.writeValueAsString(segment), new TypeReference<Map<String, Object>>(){});
final Map<String, Object> objectMap = mapper.readValue(
mapper.writeValueAsString(segment),
JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
Assert.assertEquals(10, objectMap.size());
Assert.assertEquals("something", objectMap.get("dataSource"));

View File

@ -28,6 +28,7 @@ import com.google.common.collect.ImmutableSet;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.Pair;
import io.druid.java.util.common.jackson.JacksonUtils;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
@ -88,9 +89,7 @@ public class SQLMetadataStorageActionHandlerTest
@Override
public TypeReference<Map<String, String>> getLogType()
{
return new TypeReference<Map<String, String>>()
{
};
return JacksonUtils.TYPE_REFERENCE_MAP_STRING_STRING;
}
@Override

View File

@ -19,7 +19,6 @@
package io.druid.segment.indexing;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
@ -31,6 +30,7 @@ import io.druid.java.util.common.IAE;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.granularity.DurationGranularity;
import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
import io.druid.segment.TestHelper;
@ -57,7 +57,7 @@ public class DataSchemaTest
null
),
null
), new TypeReference<Map<String, Object>>() {}
), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
DataSchema schema = new DataSchema(
@ -66,7 +66,7 @@ public class DataSchemaTest
new AggregatorFactory[]{
new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2"),
},
},
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))),
jsonMapper
);
@ -84,12 +84,16 @@ public class DataSchemaTest
new StringInputRowParser(
new JSONParseSpec(
new TimestampSpec("time", "auto", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("time", "dimA", "dimB", "col2")), ImmutableList.of("dimC"), null),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(ImmutableList.of("time", "dimA", "dimB", "col2")),
ImmutableList.of("dimC"),
null
),
null,
null
),
null
), new TypeReference<Map<String, Object>>() {}
), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
DataSchema schema = new DataSchema(
@ -98,7 +102,7 @@ public class DataSchemaTest
new AggregatorFactory[]{
new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2"),
},
},
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))),
jsonMapper
);
@ -116,12 +120,17 @@ public class DataSchemaTest
new StringInputRowParser(
new JSONParseSpec(
new TimestampSpec("time", "auto", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("time", "dimA", "dimB", "metric1")), ImmutableList.of("dimC"), null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of(
"time",
"dimA",
"dimB",
"metric1"
)), ImmutableList.of("dimC"), null),
null,
null
),
null
), new TypeReference<Map<String, Object>>() {}
), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
DataSchema schema = new DataSchema(
@ -130,7 +139,7 @@ public class DataSchemaTest
new AggregatorFactory[]{
new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2"),
},
},
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))),
jsonMapper
);
@ -144,12 +153,16 @@ public class DataSchemaTest
new StringInputRowParser(
new JSONParseSpec(
new TimestampSpec("time", "auto", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("time")), ImmutableList.of("dimC"), null),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(ImmutableList.of("time")),
ImmutableList.of("dimC"),
null
),
null,
null
),
null
), new TypeReference<Map<String, Object>>() {}
), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
DataSchema schema = new DataSchema(
@ -159,7 +172,7 @@ public class DataSchemaTest
new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2"),
new DoubleSumAggregatorFactory("metric1", "col3"),
},
},
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))),
jsonMapper
);
@ -242,7 +255,10 @@ public class DataSchemaTest
);
Assert.assertEquals(
actual.getGranularitySpec(),
new ArbitraryGranularitySpec(new DurationGranularity(86400000, null), ImmutableList.of(Intervals.of("2014/2015")))
new ArbitraryGranularitySpec(
new DurationGranularity(86400000, null),
ImmutableList.of(Intervals.of("2014/2015"))
)
);
}
}

View File

@ -19,11 +19,11 @@
package io.druid.server.coordination;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.segment.IndexIO;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
@ -61,7 +61,7 @@ public class SegmentChangeRequestDropTest
final SegmentChangeRequestDrop segmentDrop = new SegmentChangeRequestDrop(segment);
Map<String, Object> objectMap = mapper.readValue(
mapper.writeValueAsString(segmentDrop), new TypeReference<Map<String, Object>>(){}
mapper.writeValueAsString(segmentDrop), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
Assert.assertEquals(11, objectMap.size());

View File

@ -19,10 +19,10 @@
package io.druid.server.coordination;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.java.util.common.Intervals;
import io.druid.segment.IndexIO;
import io.druid.timeline.DataSegment;
@ -61,7 +61,7 @@ public class SegmentChangeRequestLoadTest
final SegmentChangeRequestLoad segmentDrop = new SegmentChangeRequestLoad(segment);
Map<String, Object> objectMap = mapper.readValue(
mapper.writeValueAsString(segmentDrop), new TypeReference<Map<String, Object>>(){}
mapper.writeValueAsString(segmentDrop), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
Assert.assertEquals(11, objectMap.size());

View File

@ -20,12 +20,12 @@
package io.druid.server.log;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Charsets;
import com.google.common.collect.ImmutableMap;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.query.BaseQuery;
import io.druid.query.DataSource;
import io.druid.query.LegacyDataSource;
@ -160,9 +160,7 @@ public class LoggingRequestLoggerTest
private static Map<String, Object> readContextMap(byte[] bytes) throws Exception
{
final Map<String, Object> rawMap = mapper.readValue(bytes, new TypeReference<Map<String, Object>>()
{
});
final Map<String, Object> rawMap = mapper.readValue(bytes, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT);
final Object contextMap = rawMap.get("contextMap");
if (contextMap == null) {
return null;