mirror of https://github.com/apache/druid.git
review comments
This commit is contained in:
parent
97e5d68635
commit
82b748ad43
|
@ -62,31 +62,22 @@ import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Determines appropriate ShardSpecs for a job by determining approximate cardinality of data set
|
* Determines appropriate ShardSpecs for a job by determining approximate cardinality of data set using HyperLogLog
|
||||||
*/
|
*/
|
||||||
public class DeterminePartitionsUsingCardinalityJob implements Jobby
|
public class DetermineHashedPartitionsJob implements Jobby
|
||||||
{
|
{
|
||||||
private static final int MAX_SHARDS = 128;
|
private static final int MAX_SHARDS = 128;
|
||||||
private static final Logger log = new Logger(DeterminePartitionsUsingCardinalityJob.class);
|
private static final Logger log = new Logger(DetermineHashedPartitionsJob.class);
|
||||||
private final HadoopDruidIndexerConfig config;
|
private final HadoopDruidIndexerConfig config;
|
||||||
|
private static final int HYPER_LOG_LOG_BIT_SIZE = 20;
|
||||||
|
|
||||||
public DeterminePartitionsUsingCardinalityJob(
|
public DetermineHashedPartitionsJob(
|
||||||
HadoopDruidIndexerConfig config
|
HadoopDruidIndexerConfig config
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.config = config;
|
this.config = config;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void injectSystemProperties(Job job)
|
|
||||||
{
|
|
||||||
final Configuration conf = job.getConfiguration();
|
|
||||||
for (String propName : System.getProperties().stringPropertyNames()) {
|
|
||||||
if (propName.startsWith("hadoop.")) {
|
|
||||||
conf.set(propName.substring("hadoop.".length()), System.getProperty(propName));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public boolean run()
|
public boolean run()
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
|
@ -97,10 +88,10 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby
|
||||||
long startTime = System.currentTimeMillis();
|
long startTime = System.currentTimeMillis();
|
||||||
final Job groupByJob = new Job(
|
final Job groupByJob = new Job(
|
||||||
new Configuration(),
|
new Configuration(),
|
||||||
String.format("%s-determine_cardinality_grouped-%s", config.getDataSource(), config.getIntervals())
|
String.format("%s-determine_partitions_hashed-%s", config.getDataSource(), config.getIntervals())
|
||||||
);
|
);
|
||||||
|
|
||||||
injectSystemProperties(groupByJob);
|
JobHelper.injectSystemProperties(groupByJob);
|
||||||
groupByJob.setInputFormatClass(TextInputFormat.class);
|
groupByJob.setInputFormatClass(TextInputFormat.class);
|
||||||
groupByJob.setMapperClass(DetermineCardinalityMapper.class);
|
groupByJob.setMapperClass(DetermineCardinalityMapper.class);
|
||||||
groupByJob.setMapOutputKeyClass(LongWritable.class);
|
groupByJob.setMapOutputKeyClass(LongWritable.class);
|
||||||
|
@ -145,6 +136,7 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby
|
||||||
log.info("Determined Intervals for Job [%s]" + config.getSegmentGranularIntervals());
|
log.info("Determined Intervals for Job [%s]" + config.getSegmentGranularIntervals());
|
||||||
}
|
}
|
||||||
Map<DateTime, List<HadoopyShardSpec>> shardSpecs = Maps.newTreeMap(DateTimeComparator.getInstance());
|
Map<DateTime, List<HadoopyShardSpec>> shardSpecs = Maps.newTreeMap(DateTimeComparator.getInstance());
|
||||||
|
int shardCount = 0;
|
||||||
for (Interval segmentGranularity : config.getSegmentGranularIntervals()) {
|
for (Interval segmentGranularity : config.getSegmentGranularIntervals()) {
|
||||||
DateTime bucket = segmentGranularity.getStart();
|
DateTime bucket = segmentGranularity.getStart();
|
||||||
|
|
||||||
|
@ -161,14 +153,17 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby
|
||||||
int numberOfShards = (int) Math.ceil((double) cardinality / config.getTargetPartitionSize());
|
int numberOfShards = (int) Math.ceil((double) cardinality / config.getTargetPartitionSize());
|
||||||
|
|
||||||
if (numberOfShards > MAX_SHARDS) {
|
if (numberOfShards > MAX_SHARDS) {
|
||||||
numberOfShards = MAX_SHARDS;
|
throw new ISE(
|
||||||
|
"Number of shards [%d] exceed the maximum limit of [%d], either targetPartitionSize is too low or data volume is too high",
|
||||||
|
numberOfShards,
|
||||||
|
MAX_SHARDS
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
List<HadoopyShardSpec> actualSpecs = Lists.newArrayListWithExpectedSize(numberOfShards);
|
List<HadoopyShardSpec> actualSpecs = Lists.newArrayListWithExpectedSize(numberOfShards);
|
||||||
if (numberOfShards == 1) {
|
if (numberOfShards == 1) {
|
||||||
actualSpecs.add(new HadoopyShardSpec(new NoneShardSpec(), 0));
|
actualSpecs.add(new HadoopyShardSpec(new NoneShardSpec(), shardCount++));
|
||||||
} else {
|
} else {
|
||||||
int shardCount = 0;
|
|
||||||
for (int i = 0; i < numberOfShards; ++i) {
|
for (int i = 0; i < numberOfShards; ++i) {
|
||||||
actualSpecs.add(new HadoopyShardSpec(new HashBasedNumberedShardSpec(i, numberOfShards), shardCount++));
|
actualSpecs.add(new HadoopyShardSpec(new HashBasedNumberedShardSpec(i, numberOfShards), shardCount++));
|
||||||
log.info("DateTime[%s], partition[%d], spec[%s]", bucket, i, actualSpecs.get(i));
|
log.info("DateTime[%s], partition[%d], spec[%s]", bucket, i, actualSpecs.get(i));
|
||||||
|
@ -183,9 +178,8 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby
|
||||||
}
|
}
|
||||||
config.setShardSpecs(shardSpecs);
|
config.setShardSpecs(shardSpecs);
|
||||||
log.info(
|
log.info(
|
||||||
"Determine partitions Using cardinality took %d millis shardSpecs %s",
|
"DetermineHashedPartitionsJob took %d millis",
|
||||||
(System.currentTimeMillis() - startTime),
|
(System.currentTimeMillis() - startTime)
|
||||||
shardSpecs
|
|
||||||
);
|
);
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
|
@ -218,7 +212,7 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby
|
||||||
determineIntervals = false;
|
determineIntervals = false;
|
||||||
final ImmutableMap.Builder<Interval, HyperLogLog> builder = ImmutableMap.builder();
|
final ImmutableMap.Builder<Interval, HyperLogLog> builder = ImmutableMap.builder();
|
||||||
for (final Interval bucketInterval : config.getGranularitySpec().bucketIntervals()) {
|
for (final Interval bucketInterval : config.getGranularitySpec().bucketIntervals()) {
|
||||||
builder.put(bucketInterval, new HyperLogLog(20));
|
builder.put(bucketInterval, new HyperLogLog(HYPER_LOG_LOG_BIT_SIZE));
|
||||||
}
|
}
|
||||||
hyperLogLogs = builder.build();
|
hyperLogLogs = builder.build();
|
||||||
}
|
}
|
||||||
|
@ -249,7 +243,7 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby
|
||||||
interval = config.getGranularitySpec().getGranularity().bucket(new DateTime(inputRow.getTimestampFromEpoch()));
|
interval = config.getGranularitySpec().getGranularity().bucket(new DateTime(inputRow.getTimestampFromEpoch()));
|
||||||
|
|
||||||
if (!hyperLogLogs.containsKey(interval)) {
|
if (!hyperLogLogs.containsKey(interval)) {
|
||||||
hyperLogLogs.put(interval, new HyperLogLog(20));
|
hyperLogLogs.put(interval, new HyperLogLog(HYPER_LOG_LOG_BIT_SIZE));
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
final Optional<Interval> maybeInterval = config.getGranularitySpec()
|
final Optional<Interval> maybeInterval = config.getGranularitySpec()
|
||||||
|
@ -307,7 +301,7 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby
|
||||||
Context context
|
Context context
|
||||||
) throws IOException, InterruptedException
|
) throws IOException, InterruptedException
|
||||||
{
|
{
|
||||||
HyperLogLog aggregate = new HyperLogLog(20);
|
HyperLogLog aggregate = new HyperLogLog(HYPER_LOG_LOG_BIT_SIZE);
|
||||||
for (BytesWritable value : values) {
|
for (BytesWritable value : values) {
|
||||||
HyperLogLog logValue = HyperLogLog.Builder.build(value.getBytes());
|
HyperLogLog logValue = HyperLogLog.Builder.build(value.getBytes());
|
||||||
try {
|
try {
|
|
@ -107,16 +107,6 @@ public class DeterminePartitionsJob implements Jobby
|
||||||
this.config = config;
|
this.config = config;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void injectSystemProperties(Job job)
|
|
||||||
{
|
|
||||||
final Configuration conf = job.getConfiguration();
|
|
||||||
for (String propName : System.getProperties().stringPropertyNames()) {
|
|
||||||
if (propName.startsWith("hadoop.")) {
|
|
||||||
conf.set(propName.substring("hadoop.".length()), System.getProperty(propName));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public boolean run()
|
public boolean run()
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
|
@ -131,7 +121,7 @@ public class DeterminePartitionsJob implements Jobby
|
||||||
String.format("%s-determine_partitions_groupby-%s", config.getDataSource(), config.getIntervals())
|
String.format("%s-determine_partitions_groupby-%s", config.getDataSource(), config.getIntervals())
|
||||||
);
|
);
|
||||||
|
|
||||||
injectSystemProperties(groupByJob);
|
JobHelper.injectSystemProperties(groupByJob);
|
||||||
groupByJob.setInputFormatClass(TextInputFormat.class);
|
groupByJob.setInputFormatClass(TextInputFormat.class);
|
||||||
groupByJob.setMapperClass(DeterminePartitionsGroupByMapper.class);
|
groupByJob.setMapperClass(DeterminePartitionsGroupByMapper.class);
|
||||||
groupByJob.setMapOutputKeyClass(BytesWritable.class);
|
groupByJob.setMapOutputKeyClass(BytesWritable.class);
|
||||||
|
@ -168,7 +158,7 @@ public class DeterminePartitionsJob implements Jobby
|
||||||
|
|
||||||
dimSelectionJob.getConfiguration().set("io.sort.record.percent", "0.19");
|
dimSelectionJob.getConfiguration().set("io.sort.record.percent", "0.19");
|
||||||
|
|
||||||
injectSystemProperties(dimSelectionJob);
|
JobHelper.injectSystemProperties(dimSelectionJob);
|
||||||
|
|
||||||
if (!config.getPartitionsSpec().isAssumeGrouped()) {
|
if (!config.getPartitionsSpec().isAssumeGrouped()) {
|
||||||
// Read grouped data from the groupByJob.
|
// Read grouped data from the groupByJob.
|
||||||
|
|
|
@ -74,7 +74,7 @@ public class HadoopDruidIndexerJob implements Jobby
|
||||||
|
|
||||||
if (config.isDeterminingPartitions()) {
|
if (config.isDeterminingPartitions()) {
|
||||||
if(config.getPartitionDimension() == null){
|
if(config.getPartitionDimension() == null){
|
||||||
jobs.add(new DeterminePartitionsUsingCardinalityJob(config));
|
jobs.add(new DetermineHashedPartitionsJob(config));
|
||||||
} else {
|
} else {
|
||||||
jobs.add(new DeterminePartitionsJob(config));
|
jobs.add(new DeterminePartitionsJob(config));
|
||||||
}
|
}
|
||||||
|
@ -143,12 +143,7 @@ public class HadoopDruidIndexerJob implements Jobby
|
||||||
);
|
);
|
||||||
|
|
||||||
job.getConfiguration().set("io.sort.record.percent", "0.19");
|
job.getConfiguration().set("io.sort.record.percent", "0.19");
|
||||||
for (String propName : System.getProperties().stringPropertyNames()) {
|
JobHelper.injectSystemProperties(job);
|
||||||
Configuration conf = job.getConfiguration();
|
|
||||||
if (propName.startsWith("hadoop.")) {
|
|
||||||
conf.set(propName.substring("hadoop.".length()), System.getProperty(propName));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
config.addInputPaths(job);
|
config.addInputPaths(job);
|
||||||
}
|
}
|
||||||
|
|
|
@ -84,9 +84,7 @@ import java.util.zip.ZipOutputStream;
|
||||||
public class IndexGeneratorJob implements Jobby
|
public class IndexGeneratorJob implements Jobby
|
||||||
{
|
{
|
||||||
private static final Logger log = new Logger(IndexGeneratorJob.class);
|
private static final Logger log = new Logger(IndexGeneratorJob.class);
|
||||||
|
|
||||||
private final HadoopDruidIndexerConfig config;
|
private final HadoopDruidIndexerConfig config;
|
||||||
|
|
||||||
private IndexGeneratorStats jobStats;
|
private IndexGeneratorStats jobStats;
|
||||||
|
|
||||||
public IndexGeneratorJob(
|
public IndexGeneratorJob(
|
||||||
|
@ -97,6 +95,39 @@ public class IndexGeneratorJob implements Jobby
|
||||||
this.jobStats = new IndexGeneratorStats();
|
this.jobStats = new IndexGeneratorStats();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static List<DataSegment> getPublishedSegments(HadoopDruidIndexerConfig config)
|
||||||
|
{
|
||||||
|
|
||||||
|
final Configuration conf = new Configuration();
|
||||||
|
final ObjectMapper jsonMapper = HadoopDruidIndexerConfig.jsonMapper;
|
||||||
|
|
||||||
|
ImmutableList.Builder<DataSegment> publishedSegmentsBuilder = ImmutableList.builder();
|
||||||
|
|
||||||
|
for (String propName : System.getProperties().stringPropertyNames()) {
|
||||||
|
if (propName.startsWith("hadoop.")) {
|
||||||
|
conf.set(propName.substring("hadoop.".length()), System.getProperty(propName));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
final Path descriptorInfoDir = config.makeDescriptorInfoDir();
|
||||||
|
|
||||||
|
try {
|
||||||
|
FileSystem fs = descriptorInfoDir.getFileSystem(conf);
|
||||||
|
|
||||||
|
for (FileStatus status : fs.listStatus(descriptorInfoDir)) {
|
||||||
|
final DataSegment segment = jsonMapper.readValue(fs.open(status.getPath()), DataSegment.class);
|
||||||
|
publishedSegmentsBuilder.add(segment);
|
||||||
|
log.info("Adding segment %s to the list of published segments", segment.getIdentifier());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (IOException e) {
|
||||||
|
throw Throwables.propagate(e);
|
||||||
|
}
|
||||||
|
List<DataSegment> publishedSegments = publishedSegmentsBuilder.build();
|
||||||
|
|
||||||
|
return publishedSegments;
|
||||||
|
}
|
||||||
|
|
||||||
public IndexGeneratorStats getJobStats()
|
public IndexGeneratorStats getJobStats()
|
||||||
{
|
{
|
||||||
return jobStats;
|
return jobStats;
|
||||||
|
@ -112,12 +143,7 @@ public class IndexGeneratorJob implements Jobby
|
||||||
|
|
||||||
job.getConfiguration().set("io.sort.record.percent", "0.23");
|
job.getConfiguration().set("io.sort.record.percent", "0.23");
|
||||||
|
|
||||||
for (String propName : System.getProperties().stringPropertyNames()) {
|
JobHelper.injectSystemProperties(job);
|
||||||
Configuration conf = job.getConfiguration();
|
|
||||||
if (propName.startsWith("hadoop.")) {
|
|
||||||
conf.set(propName.substring("hadoop.".length()), System.getProperty(propName));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
job.setInputFormatClass(TextInputFormat.class);
|
job.setInputFormatClass(TextInputFormat.class);
|
||||||
|
|
||||||
|
@ -156,39 +182,6 @@ public class IndexGeneratorJob implements Jobby
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static List<DataSegment> getPublishedSegments(HadoopDruidIndexerConfig config)
|
|
||||||
{
|
|
||||||
|
|
||||||
final Configuration conf = new Configuration();
|
|
||||||
final ObjectMapper jsonMapper = HadoopDruidIndexerConfig.jsonMapper;
|
|
||||||
|
|
||||||
ImmutableList.Builder<DataSegment> publishedSegmentsBuilder = ImmutableList.builder();
|
|
||||||
|
|
||||||
for (String propName : System.getProperties().stringPropertyNames()) {
|
|
||||||
if (propName.startsWith("hadoop.")) {
|
|
||||||
conf.set(propName.substring("hadoop.".length()), System.getProperty(propName));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
final Path descriptorInfoDir = config.makeDescriptorInfoDir();
|
|
||||||
|
|
||||||
try {
|
|
||||||
FileSystem fs = descriptorInfoDir.getFileSystem(conf);
|
|
||||||
|
|
||||||
for (FileStatus status : fs.listStatus(descriptorInfoDir)) {
|
|
||||||
final DataSegment segment = jsonMapper.readValue(fs.open(status.getPath()), DataSegment.class);
|
|
||||||
publishedSegmentsBuilder.add(segment);
|
|
||||||
log.info("Adding segment %s to the list of published segments", segment.getIdentifier());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
catch (IOException e) {
|
|
||||||
throw Throwables.propagate(e);
|
|
||||||
}
|
|
||||||
List<DataSegment> publishedSegments = publishedSegmentsBuilder.build();
|
|
||||||
|
|
||||||
return publishedSegments;
|
|
||||||
}
|
|
||||||
|
|
||||||
public static class IndexGeneratorMapper extends HadoopDruidIndexerMapper<BytesWritable, Text>
|
public static class IndexGeneratorMapper extends HadoopDruidIndexerMapper<BytesWritable, Text>
|
||||||
|
|
||||||
{
|
{
|
||||||
|
@ -225,7 +218,15 @@ public class IndexGeneratorJob implements Jobby
|
||||||
final ByteBuffer bytes = ByteBuffer.wrap(bytesWritable.getBytes());
|
final ByteBuffer bytes = ByteBuffer.wrap(bytesWritable.getBytes());
|
||||||
bytes.position(4); // Skip length added by SortableBytes
|
bytes.position(4); // Skip length added by SortableBytes
|
||||||
int shardNum = bytes.getInt();
|
int shardNum = bytes.getInt();
|
||||||
|
if (System.getProperty("mapred.job.tracker").equals("local")) {
|
||||||
return shardNum % numPartitions;
|
return shardNum % numPartitions;
|
||||||
|
} else {
|
||||||
|
if (shardNum >= numPartitions) {
|
||||||
|
throw new ISE("Not enough partitions, shard[%,d] >= numPartitions[%,d]", shardNum, numPartitions);
|
||||||
|
}
|
||||||
|
return shardNum;
|
||||||
|
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -94,4 +94,14 @@ public class JobHelper
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static void injectSystemProperties(Job job)
|
||||||
|
{
|
||||||
|
final Configuration conf = job.getConfiguration();
|
||||||
|
for (String propName : System.getProperties().stringPropertyNames()) {
|
||||||
|
if (propName.startsWith("hadoop.")) {
|
||||||
|
conf.set(propName.substring("hadoop.".length()), System.getProperty(propName));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -39,9 +39,9 @@ import java.util.Set;
|
||||||
public class HashBasedNumberedShardSpec extends NumberedShardSpec
|
public class HashBasedNumberedShardSpec extends NumberedShardSpec
|
||||||
{
|
{
|
||||||
|
|
||||||
private static HashFunction hashFunction = null;
|
private static final HashFunction hashFunction = Hashing.murmur3_32();
|
||||||
@JacksonInject
|
@JacksonInject
|
||||||
public ObjectMapper jsonMapper;
|
private ObjectMapper jsonMapper;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public HashBasedNumberedShardSpec(
|
public HashBasedNumberedShardSpec(
|
||||||
|
@ -50,7 +50,6 @@ public class HashBasedNumberedShardSpec extends NumberedShardSpec
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(partitionNum, partitions);
|
super(partitionNum, partitions);
|
||||||
hashFunction = Hashing.murmur3_32();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -19,8 +19,12 @@
|
||||||
|
|
||||||
package io.druid;
|
package io.druid;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.BeanProperty;
|
||||||
|
import com.fasterxml.jackson.databind.DeserializationContext;
|
||||||
|
import com.fasterxml.jackson.databind.InjectableValues;
|
||||||
import com.fasterxml.jackson.databind.Module;
|
import com.fasterxml.jackson.databind.Module;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
import com.metamx.common.ISE;
|
||||||
import io.druid.guice.ServerModule;
|
import io.druid.guice.ServerModule;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
import io.druid.jackson.DefaultObjectMapper;
|
||||||
|
|
||||||
|
@ -37,5 +41,20 @@ public class TestUtil
|
||||||
for (Module module : list) {
|
for (Module module : list) {
|
||||||
MAPPER.registerModule(module);
|
MAPPER.registerModule(module);
|
||||||
}
|
}
|
||||||
|
MAPPER.setInjectableValues(
|
||||||
|
new InjectableValues()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Object findInjectableValue(
|
||||||
|
Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance
|
||||||
|
)
|
||||||
|
{
|
||||||
|
if (valueId.equals("com.fasterxml.jackson.databind.ObjectMapper")) {
|
||||||
|
return TestUtil.MAPPER;
|
||||||
|
}
|
||||||
|
throw new ISE("No Injectable value found");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -38,25 +38,6 @@ import java.util.List;
|
||||||
|
|
||||||
public class HashBasedNumberedShardSpecTest
|
public class HashBasedNumberedShardSpecTest
|
||||||
{
|
{
|
||||||
@Before
|
|
||||||
public void setup()
|
|
||||||
{
|
|
||||||
TestUtil.MAPPER.setInjectableValues(
|
|
||||||
new InjectableValues()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Object findInjectableValue(
|
|
||||||
Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance
|
|
||||||
)
|
|
||||||
{
|
|
||||||
if (valueId.equals("com.fasterxml.jackson.databind.ObjectMapper")) {
|
|
||||||
return TestUtil.MAPPER;
|
|
||||||
}
|
|
||||||
throw new ISE("No Injectable value found");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSerdeRoundTrip() throws Exception
|
public void testSerdeRoundTrip() throws Exception
|
||||||
|
|
Loading…
Reference in New Issue