mirror of https://github.com/apache/druid.git
fix: avro types exception in sketch (#3167)
This commit is contained in:
parent
f46ad9a4cb
commit
ebd654228b
|
@ -23,7 +23,6 @@ import org.apache.avro.generic.GenericRecord;
|
|||
import org.apache.avro.util.Utf8;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapreduce.InputFormat;
|
||||
import org.apache.hadoop.mapreduce.Job;
|
||||
import org.apache.hadoop.mapreduce.RecordReader;
|
||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||
|
@ -38,7 +37,7 @@ import java.io.IOException;
|
|||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
public class DruidParquetInputFormatTest
|
||||
public class DruidParquetInputTest
|
||||
{
|
||||
@Test
|
||||
public void test() throws IOException, InterruptedException
|
||||
|
@ -55,7 +54,10 @@ public class DruidParquetInputFormatTest
|
|||
Path path = new Path(testFile.getAbsoluteFile().toURI());
|
||||
FileSplit split = new FileSplit(path, 0, testFile.length(), null);
|
||||
|
||||
InputFormat inputFormat = ReflectionUtils.newInstance(DruidParquetInputFormat.class, job.getConfiguration());
|
||||
DruidParquetInputFormat inputFormat = ReflectionUtils.newInstance(
|
||||
DruidParquetInputFormat.class,
|
||||
job.getConfiguration()
|
||||
);
|
||||
|
||||
TaskAttemptContext context = new TaskAttemptContextImpl(job.getConfiguration(), new TaskAttemptID());
|
||||
RecordReader reader = inputFormat.createRecordReader(split, context);
|
||||
|
@ -68,8 +70,8 @@ public class DruidParquetInputFormatTest
|
|||
|
||||
// field not read, should return null
|
||||
assertEquals(data.get("added"), null);
|
||||
|
||||
assertEquals(data.get("page"), new Utf8("Gypsy Danger"));
|
||||
assertEquals(config.getParser().parse(data).getDimension("page").get(0), "Gypsy Danger");
|
||||
|
||||
reader.close();
|
||||
}
|
|
@ -22,6 +22,7 @@ import com.google.common.base.Function;
|
|||
import com.google.common.collect.Lists;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.util.Utf8;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
|
@ -104,6 +105,9 @@ public class GenericRecordAsMap implements Map<String, Object>
|
|||
if (field instanceof ByteBuffer) {
|
||||
return Arrays.toString(((ByteBuffer) field).array());
|
||||
}
|
||||
if (field instanceof Utf8) {
|
||||
return field.toString();
|
||||
}
|
||||
return field;
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue