Skip to content

Commit

Permalink
Report field name when decoding Elasticsearch value fails
Browse files Browse the repository at this point in the history
Cherry-pick of trinodb/trino@13a3c52

Co-authored-by: Martin Traverso mtraverso@gmail.com
  • Loading branch information
zhenxiao committed Jun 10, 2020
1 parent 74f944d commit 90f72c3
Show file tree
Hide file tree
Showing 13 changed files with 120 additions and 29 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -293,34 +293,34 @@ private List<Decoder> createDecoders(ConnectorSession session, List<Elasticsearc
private Decoder createDecoder(ConnectorSession session, String path, Type type)
{
if (type.equals(VARCHAR)) {
return new VarcharDecoder();
return new VarcharDecoder(path);
}
else if (type.equals(VARBINARY)) {
return new VarbinaryDecoder();
return new VarbinaryDecoder(path);
}
else if (type.equals(TIMESTAMP)) {
return new TimestampDecoder(session, path);
}
else if (type.equals(BOOLEAN)) {
return new BooleanDecoder();
return new BooleanDecoder(path);
}
else if (type.equals(DOUBLE)) {
return new DoubleDecoder();
return new DoubleDecoder(path);
}
else if (type.equals(REAL)) {
return new RealDecoder();
return new RealDecoder(path);
}
else if (type.equals(TINYINT)) {
return new TinyintDecoder();
return new TinyintDecoder(path);
}
else if (type.equals(SMALLINT)) {
return new SmallintDecoder();
return new SmallintDecoder(path);
}
else if (type.equals(INTEGER)) {
return new IntegerDecoder();
return new IntegerDecoder(path);
}
else if (type.equals(BIGINT)) {
return new BigintDecoder();
return new BigintDecoder(path);
}
else if (type instanceof RowType) {
RowType rowType = (RowType) type;
Expand All @@ -334,12 +334,12 @@ else if (type instanceof RowType) {
.map(Optional::get)
.collect(toImmutableList());

return new RowDecoder(fieldNames, decoders);
return new RowDecoder(path, fieldNames, decoders);
}
else if (type instanceof ArrayType) {
Type elementType = ((ArrayType) type).getElementType();

return new ArrayDecoder(createDecoder(session, path, elementType));
return new ArrayDecoder(path, createDecoder(session, path, elementType));
}

throw new UnsupportedOperationException("Type not supported: " + type);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,14 +21,18 @@
import java.util.function.Supplier;

import static com.facebook.presto.elasticsearch.ElasticsearchErrorCode.ELASTICSEARCH_TYPE_MISMATCH;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;

public class ArrayDecoder
implements Decoder
{
private final String path;
private final Decoder elementDecoder;

public ArrayDecoder(Decoder elementDecoder)
public ArrayDecoder(String path, Decoder elementDecoder)
{
this.path = requireNonNull(path, "path is null");
this.elementDecoder = elementDecoder;
}

Expand All @@ -46,7 +50,7 @@ else if (data instanceof List) {
output.closeEntry();
}
else {
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, "Expected list of elements for ARRAY field");
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, format("Expected list of elements for field '%s' of type ARRAY: %s [%s]", path, data, data.getClass().getSimpleName()));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,19 @@

import static com.facebook.presto.common.type.BigintType.BIGINT;
import static com.facebook.presto.elasticsearch.ElasticsearchErrorCode.ELASTICSEARCH_TYPE_MISMATCH;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;

public class BigintDecoder
implements Decoder
{
private final String path;

public BigintDecoder(String path)
{
this.path = requireNonNull(path, "path is null");
}

@Override
public void decode(SearchHit hit, Supplier<Object> getter, BlockBuilder output)
{
Expand All @@ -36,7 +45,7 @@ else if (value instanceof Number) {
BIGINT.writeLong(output, ((Number) value).longValue());
}
else {
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, "Expected a numeric value for BIGINT field");
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, format("Expected a numeric value for field %s of type BIGINT: %s [%s]", path, value, value.getClass().getSimpleName()));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,19 @@

import static com.facebook.presto.common.type.BooleanType.BOOLEAN;
import static com.facebook.presto.elasticsearch.ElasticsearchErrorCode.ELASTICSEARCH_TYPE_MISMATCH;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;

public class BooleanDecoder
implements Decoder
{
private final String path;

public BooleanDecoder(String path)
{
this.path = requireNonNull(path, "path is null");
}

@Override
public void decode(SearchHit hit, Supplier<Object> getter, BlockBuilder output)
{
Expand All @@ -36,7 +45,7 @@ else if (value instanceof Boolean) {
BOOLEAN.writeBoolean(output, (Boolean) value);
}
else {
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, "Expected a boolean value for BOOLEAN field");
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, format("Expected a boolean value for field %s of type BOOLEAN: %s [%s]", path, value, value.getClass().getSimpleName()));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,19 @@

import static com.facebook.presto.common.type.DoubleType.DOUBLE;
import static com.facebook.presto.elasticsearch.ElasticsearchErrorCode.ELASTICSEARCH_TYPE_MISMATCH;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;

public class DoubleDecoder
implements Decoder
{
private final String path;

public DoubleDecoder(String path)
{
this.path = requireNonNull(path, "path is null");
}

@Override
public void decode(SearchHit hit, Supplier<Object> getter, BlockBuilder output)
{
Expand All @@ -36,7 +45,7 @@ else if (value instanceof Number) {
DOUBLE.writeDouble(output, ((Number) value).doubleValue());
}
else {
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, "Expected a numeric value for DOUBLE field");
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, format("Expected a numeric value for field %s of type DOUBLE: %s [%s]", path, value, value.getClass().getSimpleName()));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,10 +22,19 @@
import static com.facebook.presto.common.type.IntegerType.INTEGER;
import static com.facebook.presto.elasticsearch.ElasticsearchErrorCode.ELASTICSEARCH_TYPE_MISMATCH;
import static java.lang.Math.toIntExact;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;

public class IntegerDecoder
implements Decoder
{
private final String path;

public IntegerDecoder(String path)
{
this.path = requireNonNull(path, "path is null");
}

@Override
public void decode(SearchHit hit, Supplier<Object> getter, BlockBuilder output)
{
Expand All @@ -37,7 +46,7 @@ else if (value instanceof Number) {
INTEGER.writeLong(output, toIntExact(((Number) value).longValue()));
}
else {
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, "Expected a numeric value for INTEGER field");
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, format("Expected a numeric value for field '%s' of type INTEGER: %s [%s]", path, value, value.getClass().getSimpleName()));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,19 @@

import static com.facebook.presto.common.type.RealType.REAL;
import static com.facebook.presto.elasticsearch.ElasticsearchErrorCode.ELASTICSEARCH_TYPE_MISMATCH;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;

public class RealDecoder
implements Decoder
{
private final String path;

public RealDecoder(String path)
{
this.path = requireNonNull(path, "path is null");
}

@Override
public void decode(SearchHit hit, Supplier<Object> getter, BlockBuilder output)
{
Expand All @@ -36,7 +45,7 @@ else if (value instanceof Number) {
REAL.writeLong(output, Float.floatToRawIntBits(((Number) value).floatValue()));
}
else {
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, "Expected a numeric value for REAL field");
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, format("Expected a numeric value for field %s of type REAL: %s [%s]", path, value, value.getClass().getSimpleName()));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,15 +23,19 @@

import static com.facebook.presto.elasticsearch.ElasticsearchErrorCode.ELASTICSEARCH_TYPE_MISMATCH;
import static com.facebook.presto.elasticsearch.ElasticsearchPageSource.getField;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;

public class RowDecoder
implements Decoder
{
private final String path;
private final List<String> fieldNames;
private final List<Decoder> decoders;

public RowDecoder(List<String> fieldNames, List<Decoder> decoders)
public RowDecoder(String path, List<String> fieldNames, List<Decoder> decoders)
{
this.path = requireNonNull(path, "path is null");
this.fieldNames = fieldNames;
this.decoders = decoders;
}
Expand All @@ -53,7 +57,7 @@ else if (data instanceof Map) {
output.closeEntry();
}
else {
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, "Expected object for ROW field");
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, format("Expected object for field '%s' of type ROW: %s [%s]", path, data, data.getClass().getSimpleName()));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,19 @@

import static com.facebook.presto.common.type.SmallintType.SMALLINT;
import static com.facebook.presto.elasticsearch.ElasticsearchErrorCode.ELASTICSEARCH_TYPE_MISMATCH;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;

public class SmallintDecoder
implements Decoder
{
private final String path;

public SmallintDecoder(String path)
{
this.path = requireNonNull(path, "path is null");
}

@Override
public void decode(SearchHit hit, Supplier<Object> getter, BlockBuilder output)
{
Expand All @@ -36,13 +45,13 @@ else if (value instanceof Number) {
long decoded = ((Number) value).longValue();

if (decoded < Short.MIN_VALUE || decoded > Short.MAX_VALUE) {
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, "Value out of range for SMALLINT field");
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, format("Value out of range for field '%s' of type SMALLINT: %s", path, decoded));
}

SMALLINT.writeLong(output, decoded);
}
else {
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, "Expected a numeric value for SMALLINT field");
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, format("Expected a numeric value for field '%s' of type SMALLINT: %s [%s]", path, value, value.getClass().getSimpleName()));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED;
import static java.lang.String.format;
import static java.time.format.DateTimeFormatter.ISO_DATE_TIME;
import static java.util.Objects.requireNonNull;

public class TimestampDecoder
implements Decoder
Expand All @@ -40,7 +41,7 @@ public class TimestampDecoder

public TimestampDecoder(ConnectorSession session, String path)
{
this.path = path;
this.path = requireNonNull(path, "path is null");
this.zoneId = ZoneId.of(session.getSqlFunctionProperties().getTimeZoneKey().getId());
}

Expand All @@ -52,7 +53,7 @@ public void decode(SearchHit hit, Supplier<Object> getter, BlockBuilder output)

if (documentField != null) {
if (documentField.getValues().size() > 1) {
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, "Expected single value for column: " + path);
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, format("Expected single value for column '%s', found: %s", path, documentField.getValues().size()));
}
value = documentField.getValue();
}
Expand All @@ -73,7 +74,8 @@ else if (value instanceof Number) {
}
else {
throw new PrestoException(NOT_SUPPORTED, format(
"Unsupported representation for timestamp type: %s [%s]",
"Unsupported representation for field '%s' of type TIMESTAMP: %s [%s]",
path,
value.getClass().getSimpleName(),
value));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,19 @@

import static com.facebook.presto.common.type.TinyintType.TINYINT;
import static com.facebook.presto.elasticsearch.ElasticsearchErrorCode.ELASTICSEARCH_TYPE_MISMATCH;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;

public class TinyintDecoder
implements Decoder
{
private final String path;

public TinyintDecoder(String path)
{
this.path = requireNonNull(path, "path is null");
}

@Override
public void decode(SearchHit hit, Supplier<Object> getter, BlockBuilder output)
{
Expand All @@ -36,13 +45,13 @@ else if (value instanceof Number) {
long decoded = ((Number) value).longValue();

if (decoded < Byte.MIN_VALUE || decoded > Byte.MAX_VALUE) {
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, "Value out of range for TINYINT field");
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, format("Value out of range for field '%s' of type TINYINT: %s", path, decoded));
}

TINYINT.writeLong(output, decoded);
}
else {
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, "Expected a numeric value for TINYINT field");
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, format("Expected a numeric value for field '%s' of type TINYINT: %s [%s]", path, value, value.getClass().getSimpleName()));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,10 +23,19 @@

import static com.facebook.presto.common.type.VarbinaryType.VARBINARY;
import static com.facebook.presto.elasticsearch.ElasticsearchErrorCode.ELASTICSEARCH_TYPE_MISMATCH;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;

public class VarbinaryDecoder
implements Decoder
{
private final String path;

public VarbinaryDecoder(String path)
{
this.path = requireNonNull(path, "path is null");
}

@Override
public void decode(SearchHit hit, Supplier<Object> getter, BlockBuilder output)
{
Expand All @@ -38,7 +47,7 @@ else if (value instanceof String) {
VARBINARY.writeSlice(output, Slices.wrappedBuffer(Base64.getDecoder().decode(value.toString())));
}
else {
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, "Expected a string value for VARBINARY field");
throw new PrestoException(ELASTICSEARCH_TYPE_MISMATCH, format("Expected a string value for field '%s' of type VARBINARY: %s [%s]", path, value, value.getClass().getSimpleName()));
}
}
}
Loading

0 comments on commit 90f72c3

Please sign in to comment.