diff --git a/lib/trino-hive-formats/pom.xml b/lib/trino-hive-formats/pom.xml index 9c9360118a2c..de4913f35295 100644 --- a/lib/trino-hive-formats/pom.xml +++ b/lib/trino-hive-formats/pom.xml @@ -125,12 +125,43 @@ test + + io.starburst.openjson + openjson + 1.8-e.10 + test + + + + io.starburst.openx.data + json-serde + 1.3.9-e.10 + test + + + org.apache.hive + hive-serde + + + org.apache.hive + hive-exec + + + + it.unimi.dsi fastutil test + + org.apache.commons + commons-lang3 + 3.12.0 + test + + org.assertj assertj-core diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/HiveFormatUtils.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/HiveFormatUtils.java index 57804932572b..395ecb19b76a 100644 --- a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/HiveFormatUtils.java +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/HiveFormatUtils.java @@ -89,6 +89,7 @@ public final class HiveFormatUtils .toFormatter(); private static final DateTimeFormatter DEFAULT_TIMESTAMP_PARSER = new DateTimeFormatterBuilder() + .parseCaseInsensitive() .parseLenient() // Date part .appendValue(YEAR, 1, 10, SignStyle.NORMAL) @@ -97,7 +98,8 @@ public final class HiveFormatUtils .appendLiteral('-') .appendValue(DAY_OF_MONTH, 1, 2, SignStyle.NORMAL) // Time part - .optionalStart().appendLiteral(" ") + .optionalStart() + .appendLiteral(" ") .appendValue(HOUR_OF_DAY, 1, 2, SignStyle.NORMAL) .appendLiteral(':') .appendValue(MINUTE_OF_HOUR, 1, 2, SignStyle.NORMAL) @@ -108,6 +110,13 @@ public final class HiveFormatUtils .toFormatter() .withResolverStyle(LENIENT); + private static final DateTimeFormatter ISO_TIMESTAMP_PARSER = new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .parseLenient() + .append(DateTimeFormatter.ISO_LOCAL_DATE_TIME) + .toFormatter() + .withResolverStyle(LENIENT); + private static final DateTimeFormatter TIMESTAMP_FORMATTER = new DateTimeFormatterBuilder() // Date part .appendValue(YEAR, 1, 10, SignStyle.NORMAL) @@ -116,7 +125,8 @@ public final class HiveFormatUtils .appendLiteral('-') .appendValue(DAY_OF_MONTH, 2, 2, SignStyle.NORMAL) // Time part - .optionalStart().appendLiteral(" ") + .optionalStart() + .appendLiteral(" ") .appendValue(HOUR_OF_DAY, 2, 2, SignStyle.NORMAL) .appendLiteral(':') .appendValue(MINUTE_OF_HOUR, 2, 2, SignStyle.NORMAL) @@ -159,13 +169,18 @@ public static void writeDecimal(String value, DecimalType decimalType, BlockBuil public static BigDecimal parseDecimal(String value, DecimalType decimalType) { try { - return new BigDecimal(value).setScale(DecimalConversions.intScale(decimalType.getScale()), HALF_UP); + return scaleDecimal(new BigDecimal(value), decimalType); } catch (NumberFormatException e) { throw new NumberFormatException(format("Cannot convert '%s' to %s. Value is not a number.", value, decimalType)); } } + public static BigDecimal scaleDecimal(BigDecimal bigDecimal, DecimalType decimalType) + { + return bigDecimal.setScale(DecimalConversions.intScale(decimalType.getScale()), HALF_UP); + } + public static Function createTimestampParser(List timestampFormats) { requireNonNull(timestampFormats, "timestampFormats is null"); @@ -229,7 +244,7 @@ public static DecodedTimestamp parseHiveTimestamp(String value) } catch (DateTimeParseException e) { // Try ISO-8601 format - localDateTime = LocalDateTime.parse(value); + localDateTime = LocalDateTime.parse(value, ISO_TIMESTAMP_PARSER); } return new DecodedTimestamp(localDateTime.toEpochSecond(ZoneOffset.UTC), localDateTime.getNano()); } diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/InvalidJsonException.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/InvalidJsonException.java new file mode 100644 index 000000000000..49662071e67e --- /dev/null +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/InvalidJsonException.java @@ -0,0 +1,25 @@ +/* + * 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.trino.hive.formats.line.openxjson; + +import java.io.IOException; + +class InvalidJsonException + extends IOException +{ + public InvalidJsonException(String message, int position, String json) + { + super("%s at character %d of JSON: %s".formatted(message, position, json)); + } +} diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/JsonReader.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/JsonReader.java new file mode 100644 index 000000000000..f8b71f4908d8 --- /dev/null +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/JsonReader.java @@ -0,0 +1,368 @@ +/* + * 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.trino.hive.formats.line.openxjson; + +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.StringJoiner; +import java.util.function.Function; + +import static java.util.Collections.emptyMap; +import static java.util.Collections.unmodifiableList; +import static java.util.Collections.unmodifiableMap; +import static java.util.Objects.requireNonNull; + +/** + * Reads a JSON encoded string into the corresponding object. + *

+ * For legacy reasons this parser is lenient, so a successful parse does + * not indicate that the input string was valid JSON. All the following syntax + * errors will be ignored: + *

    + *
  • End of line comments starting with {@code //} or {@code #} + *
  • C-style comments starting with {@code /*} and ending with + * {@code *}{@code /} + *
  • Strings that are unquoted or {@code 'single quoted'}. + *
  • Hexadecimal integers prefixed with {@code 0x} or {@code 0X}. + *
  • Octal integers prefixed with {@code 0}. + *
  • Array elements separated by {@code ;}. + *
  • Unnecessary array separators. These are interpreted as if null was the + * omitted value. + *
  • Key-value pairs separated by {@code =} or {@code =>}. + *
  • Key-value pairs separated by {@code ;}. + *
+ */ +final class JsonReader +{ + /** + * Decode the JSON value. + *

+ * This will only combinations of {@code null}, {@code JsonString}, {@code Map}, or + * {@code List}. A list may contain nulls, but the map will not contain nulls. + * + * @param json valid JSON + * @return JsonString, Map, List, or null + * @throws InvalidJsonException if the value is not valid json + */ + public static Object readJson(String json, Function keyMap) + throws InvalidJsonException + { + return new JsonReader<>(json, keyMap).nextValue(); + } + + private final String json; + private final Function keyMap; + private int position; + + private JsonReader(String json, Function keyMap) + { + requireNonNull(json, "json is null"); + this.keyMap = keyMap; + this.json = json; + } + + private Object nextValue() + throws InvalidJsonException + { + int c = seekNextToken(); + if (c == -1) { + throw new InvalidJsonException("End of input", position, json); + } + if (c == '{') { + return readObject(); + } + if (c == '[') { + return readArray(); + } + if (c == '\'' || c == '"') { + return readQuotedString((char) c); + } + + position--; + return readLiteral(); + } + + private Map readObject() + throws InvalidJsonException + { + // Peek to see if this is the empty object. + int first = seekNextToken(); + if (first == '}') { + return emptyMap(); + } + else if (first != -1) { + position--; + } + + Map jsonObject = new LinkedHashMap<>(); + while (true) { + Object name = nextValue(); + if (name == null) { + throw new InvalidJsonException("Field name is null literal", position, json); + } + + if (name instanceof JsonString jsonString) { + name = jsonString.value(); + } + else { + String message = "Names must be strings, but %s is of type %s".formatted(name, name.getClass().getSimpleName()); + throw new InvalidJsonException(message, position, json); + } + + // name/value separator can be ':', '=', or '=>' + int separator = seekNextToken(); + if (separator != ':' && separator != '=') { + throw new InvalidJsonException("Expected ':' after field " + name, position, json); + } + if (separator == '=' && position < json.length() && json.charAt(position) == '>') { + position++; + } + + T key = keyMap.apply((String) name); + // Linked hash map does not replace existing keys that match, which is important for case-insensitive mappings + jsonObject.remove(key); + jsonObject.put(key, nextValue()); + + int c = seekNextToken(); + if (c == '}') { + return unmodifiableMap(jsonObject); + } + // entry separator can be `,` or `;` + if (c != ',' && c != ';') { + throw new InvalidJsonException("Unterminated object", position, json); + } + } + } + + private List readArray() + throws InvalidJsonException + { + List jsonArray = new ArrayList<>(); + + boolean hasTrailingSeparator = false; + while (true) { + int c = seekNextToken(); + if (c == -1) { + throw new InvalidJsonException("Unterminated array", position, json); + } + if (c == ']') { + if (hasTrailingSeparator) { + jsonArray.add(null); + } + return unmodifiableList(jsonArray); + } + + // A separator without a value first means is an implicit null + if (c == ',' || c == ';') { + jsonArray.add(null); + hasTrailingSeparator = true; + continue; + } + + // add the next value + position--; + jsonArray.add(nextValue()); + + // next token must end the array or be a separator + c = seekNextToken(); + if (c == ']') { + return unmodifiableList(jsonArray); + } + + // value separator can be ',' or ';' + if (c != ',' && c != ';') { + throw new InvalidJsonException("Unterminated array", position, json); + } + hasTrailingSeparator = true; + } + } + + private JsonString readQuotedString(char quote) + throws InvalidJsonException + { + // only used when there are escapes + StringBuilder builder = null; + + int start = position; + while (position < json.length()) { + int c = json.charAt(position++); + if (c == quote) { + if (builder == null) { + // copy directly from source + return new JsonString(json.substring(start, position - 1), true); + } + builder.append(json, start, position - 1); + return new JsonString(builder.toString(), true); + } + + if (c == '\\') { + if (position == json.length()) { + throw new InvalidJsonException("Unterminated escape sequence", position, json); + } + if (builder == null) { + builder = new StringBuilder(); + } + builder.append(json, start, position - 1); + builder.append(decodeEscapeCharacter()); + start = position; + } + } + + throw new InvalidJsonException("Unterminated string", position, json); + } + + private char decodeEscapeCharacter() + throws InvalidJsonException + { + char escaped = json.charAt(position++); + switch (escaped) { + case 'u' -> { + if (position + 4 > json.length()) { + throw new InvalidJsonException("Unterminated escape sequence", position, json); + } + String hex = json.substring(position, position + 4); + position += 4; + try { + return (char) Integer.parseInt(hex, 16); + } + catch (NumberFormatException nfe) { + throw new InvalidJsonException("Invalid escape sequence: " + hex, position, json); + } + } + case 't' -> { + return '\t'; + } + case 'b' -> { + return '\b'; + } + case 'n' -> { + return '\n'; + } + case 'r' -> { + return '\r'; + } + case 'f' -> { + return '\f'; + } + case 'a' -> { + return '\007'; + } + case 'v' -> { + return '\011'; + } + } + return escaped; + } + + /** + * Reads a null or unquoted string literal value. + */ + private JsonString readLiteral() + throws InvalidJsonException + { + String literal = literalToken(); + if (literal.isEmpty()) { + throw new InvalidJsonException("Expected literal value", position, json); + } + + if ("null".equalsIgnoreCase(literal)) { + return null; + } + + return new JsonString(literal, false); + } + + private String literalToken() + { + int start = position; + while (position < json.length()) { + char c = json.charAt(position); + if (c == ' ' || c == '\r' || c == '\n' || c == '\t' || c == '\f' || + c == '{' || c == '}' || c == '[' || c == ']' || + c == '/' || c == '\\' || + c == ':' || c == ',' || c == '=' || c == ';' || c == '#') { + return json.substring(start, position); + } + position++; + } + return json.substring(start); + } + + /** + * Seek to the start of the next token, skipping any whitespace and comments. + */ + private int seekNextToken() + throws InvalidJsonException + { + while (position < json.length()) { + int c = json.charAt(position++); + switch (c) { + case '\t', ' ', '\n', '\r' -> { + // ignore whitespace + } + case '#' -> skipEndOfLineComment(); + case '/' -> { + // possible comment + if (position == json.length()) { + return c; + } + char peek = json.charAt(position); + if (peek == '*') { + // c-style comment + position++; + int commentEnd = json.indexOf("*/", position); + if (commentEnd == -1) { + throw new InvalidJsonException("Unterminated c-style comment", position, json); + } + position = commentEnd + 2; + continue; + } + if (peek == '/') { + skipEndOfLineComment(); + continue; + } + return c; + } + default -> { + return c; + } + } + } + + return -1; + } + + // This will never actually find an end of line character as these are handled by the Hive line format + private void skipEndOfLineComment() + { + while (position < json.length()) { + char c = json.charAt(position); + position++; + if (c == '\r' || c == '\n') { + return; + } + } + } + + @Override + public String toString() + { + return new StringJoiner(", ", JsonReader.class.getSimpleName() + "[", "]") + .add("in='" + json + "'") + .add("position=" + position) + .toString(); + } +} diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/JsonString.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/JsonString.java new file mode 100644 index 000000000000..b01daaf22d6e --- /dev/null +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/JsonString.java @@ -0,0 +1,23 @@ +/* + * 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.trino.hive.formats.line.openxjson; + +record JsonString(String value, boolean quoted) +{ + @Override + public String toString() + { + return value; + } +} diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/JsonWriter.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/JsonWriter.java new file mode 100644 index 000000000000..b12c80e578f5 --- /dev/null +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/JsonWriter.java @@ -0,0 +1,343 @@ +/* + * 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.trino.hive.formats.line.openxjson; + +import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import static io.trino.hive.formats.line.openxjson.OpenXJsonDeserializer.parseDecimalHexOctalLong; +import static java.util.Objects.requireNonNull; + +/** + * Writer only accepts a {@link Map}, {@link List}, JsonString, String, Boolean, or Number. + */ +final class JsonWriter +{ + public static String writeJsonObject(Map jsonObject) + { + requireNonNull(jsonObject, "jsonObject is null"); + + JsonWriter writer = new JsonWriter(); + writer.value(jsonObject); + return writer.toString(); + } + + public static String writeJsonArray(List jsonArray) + { + requireNonNull(jsonArray, "jsonArray is null"); + + JsonWriter writer = new JsonWriter(); + writer.value(jsonArray); + return writer.toString(); + } + + public static String canonicalizeJsonString(JsonString jsonString) + { + if (!jsonString.quoted()) { + String canonicalUnquotedString = canonicalizeUnquotedString(jsonString.value()); + if (canonicalUnquotedString != null) { + return canonicalUnquotedString; + } + } + return jsonString.value(); + } + + private final StringBuilder out = new StringBuilder(); + private final List scopes = new ArrayList<>(); + + private JsonWriter() {} + + private void beginArray() + { + openNewScope(Scope.EMPTY_ARRAY, "["); + } + + private void beginObject() + { + openNewScope(Scope.EMPTY_OBJECT, "{"); + } + + private void endArray() + { + closeCurrentScope(Scope.EMPTY_ARRAY, Scope.NONEMPTY_ARRAY, "]"); + } + + private void endObject() + { + closeCurrentScope(Scope.EMPTY_OBJECT, Scope.NONEMPTY_OBJECT, "}"); + } + + private void fieldName(Object name) + { + if (name == null) { + throw new IllegalStateException("Field name must not be null"); + } + + Scope context = getCurrentScope(); + if (context == Scope.NONEMPTY_OBJECT) { + // another object entry + out.append(','); + } + else if (context != Scope.EMPTY_OBJECT) { + throw new IllegalStateException("Nesting problem"); + } + replaceCurrentScope(Scope.DANGLING_FIELD_NAME); + + writeJsonString(name.toString()); + } + + private void value(Object value) + { + if (value instanceof List list) { + beginArray(); + for (Object entry : list) { + value(entry); + } + endArray(); + return; + } + if (value instanceof Map map) { + beginObject(); + for (Entry entry : map.entrySet()) { + fieldName(entry.getKey()); + value(entry.getValue()); + } + endObject(); + return; + } + + // primitive value without a wrapping array or object is not supported + if (scopes.isEmpty()) { + throw new IllegalStateException("Nesting problem"); + } + + beforeValue(); + + if (value == null || value instanceof Boolean) { + out.append(value); + } + else if (value instanceof Number number) { + writeJsonNumber(number); + } + else if (value instanceof String string) { + writeJsonString(string); + } + else if (value instanceof JsonString string) { + if (!string.quoted()) { + String canonicalUnquotedString = canonicalizeUnquotedString(string.value()); + if (canonicalUnquotedString != null) { + out.append(canonicalUnquotedString); + return; + } + } + writeJsonString(string.value()); + } + else { + throw new IllegalArgumentException("Unsupported value type: " + value.getClass().getSimpleName()); + } + } + + private static String canonicalizeUnquotedString(String value) + { + // canonicalize unquoted values + // boolean to lower case + if (value.equalsIgnoreCase("true")) { + return "true"; + } + if (value.equalsIgnoreCase("false")) { + return "false"; + } + + // decimal, hex, and octal numbers rendered as plain decimal + try { + return String.valueOf(parseDecimalHexOctalLong(value)); + } + catch (NumberFormatException ignored) { + } + // Use BigDecimal for all other values + try { + return new BigDecimal(value).toString(); + } + catch (NumberFormatException ignored) { + } + return null; + } + + private void writeJsonString(String value) + { + out.append("\""); + char currentChar = 0; + + for (int i = 0, length = value.length(); i < length; i++) { + char previousChar = currentChar; + currentChar = value.charAt(i); + + /* + * From RFC 4627, "All Unicode characters may be placed within the + * quotation marks except for the characters that must be escaped: + * quotation mark, reverse solidus, and the control characters + * (U+0000 through U+001F)." + */ + switch (currentChar) { + case '"', '\\' -> out.append('\\').append(currentChar); + case '/' -> { + // it makes life easier for HTML embedding of javascript if we escape out.append("\\t"); + case '\b' -> out.append("\\b"); + case '\n' -> out.append("\\n"); + case '\r' -> out.append("\\r"); + case '\f' -> out.append("\\f"); + default -> { + if (currentChar <= 0x1F) { + out.append(String.format("\\u%04x", (int) currentChar)); + } + else { + out.append(currentChar); + } + } + } + } + out.append("\""); + } + + private void writeJsonNumber(Number number) + { + requireNonNull(number, "number is null"); + + // for double and float, render as a long if possible without loss of permission + if (number instanceof Double || number instanceof Float) { + // the original returns "-0" instead of "-0.0" for negative zero + if (number.equals(-0.0d)) { + out.append("-0"); + return; + } + + // render doubles as a fixed integer if possible + //noinspection FloatingPointEquality + if (number.doubleValue() == number.longValue()) { + out.append(number.longValue()); + return; + } + } + out.append(number); + } + + private void beforeValue() + { + // value without a wrapping array or object is not supported + if (scopes.isEmpty()) { + return; + } + + Scope context = getCurrentScope(); + // first in array + if (context == Scope.EMPTY_ARRAY) { + replaceCurrentScope(Scope.NONEMPTY_ARRAY); + return; + } + // another array element + if (context == Scope.NONEMPTY_ARRAY) { + out.append(','); + return; + } + // value for a key + if (context == Scope.DANGLING_FIELD_NAME) { + out.append(":"); + replaceCurrentScope(Scope.NONEMPTY_OBJECT); + return; + } + throw new IllegalStateException("Nesting problem"); + } + + private void openNewScope(Scope empty, String openBracket) + { + if (scopes.isEmpty() && !out.isEmpty()) { + throw new IllegalStateException("Nesting problem: multiple top-level roots"); + } + beforeValue(); + scopes.add(empty); + out.append(openBracket); + } + + private void closeCurrentScope(Scope empty, Scope nonempty, String closeBracket) + { + Scope context = getCurrentScope(); + if (context != nonempty && context != empty) { + throw new IllegalStateException("Nesting problem"); + } + + scopes.remove(scopes.size() - 1); + out.append(closeBracket); + } + + private Scope getCurrentScope() + { + if (scopes.isEmpty()) { + throw new IllegalStateException("Nesting problem"); + } + return scopes.get(scopes.size() - 1); + } + + private void replaceCurrentScope(Scope topOfStack) + { + scopes.set(scopes.size() - 1, topOfStack); + } + + @Override + public String toString() + { + return out.isEmpty() ? null : out.toString(); + } + + /** + * Lexical scoping elements within this stringer, necessary to insert the + * appropriate separator characters (i.e., commas and colons) and to detect + * nesting errors. + */ + private enum Scope + { + /** + * An array with no elements + */ + EMPTY_ARRAY, + + /** + * An array with at least one value + */ + NONEMPTY_ARRAY, + + /** + * An object with no keys or values + */ + EMPTY_OBJECT, + + /** + * An object whose most recent element is a field name + */ + DANGLING_FIELD_NAME, + + /** + * An object with at least one entry + */ + NONEMPTY_OBJECT, + } +} diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/OpenXJsonDeserializer.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/OpenXJsonDeserializer.java new file mode 100644 index 000000000000..80711b97955a --- /dev/null +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/OpenXJsonDeserializer.java @@ -0,0 +1,982 @@ +/* + * 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.trino.hive.formats.line.openxjson; + +import com.google.common.base.CharMatcher; +import com.google.common.collect.ImmutableMap; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import io.trino.hive.formats.DistinctMapKeys; +import io.trino.hive.formats.line.Column; +import io.trino.hive.formats.line.LineBuffer; +import io.trino.hive.formats.line.LineDeserializer; +import io.trino.plugin.base.type.DecodedTimestamp; +import io.trino.plugin.base.type.TrinoTimestampEncoder; +import io.trino.spi.PageBuilder; +import io.trino.spi.block.Block; +import io.trino.spi.block.BlockBuilder; +import io.trino.spi.block.SingleRowBlockWriter; +import io.trino.spi.type.ArrayType; +import io.trino.spi.type.CharType; +import io.trino.spi.type.DecimalType; +import io.trino.spi.type.Int128; +import io.trino.spi.type.MapType; +import io.trino.spi.type.RowType; +import io.trino.spi.type.RowType.Field; +import io.trino.spi.type.TimestampType; +import io.trino.spi.type.Type; +import io.trino.spi.type.VarcharType; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.RoundingMode; +import java.time.ZoneOffset; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; +import java.time.format.DateTimeParseException; +import java.util.Base64; +import java.util.Collection; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.function.IntFunction; +import java.util.regex.Pattern; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.hive.formats.HiveFormatUtils.parseHiveDate; +import static io.trino.hive.formats.HiveFormatUtils.parseHiveTimestamp; +import static io.trino.hive.formats.HiveFormatUtils.scaleDecimal; +import static io.trino.hive.formats.line.openxjson.JsonWriter.canonicalizeJsonString; +import static io.trino.hive.formats.line.openxjson.JsonWriter.writeJsonArray; +import static io.trino.hive.formats.line.openxjson.JsonWriter.writeJsonObject; +import static io.trino.plugin.base.type.TrinoTimestampEncoderFactory.createTimestampEncoder; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.Chars.truncateToLengthAndTrimSpaces; +import static io.trino.spi.type.DateType.DATE; +import static io.trino.spi.type.Decimals.overflows; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.spi.type.IntegerType.INTEGER; +import static io.trino.spi.type.RealType.REAL; +import static io.trino.spi.type.RowType.field; +import static io.trino.spi.type.SmallintType.SMALLINT; +import static io.trino.spi.type.Timestamps.MILLISECONDS_PER_SECOND; +import static io.trino.spi.type.Timestamps.NANOSECONDS_PER_MILLISECOND; +import static io.trino.spi.type.TinyintType.TINYINT; +import static io.trino.spi.type.VarbinaryType.VARBINARY; +import static io.trino.spi.type.Varchars.truncateToLength; +import static java.lang.Character.toLowerCase; +import static java.lang.Float.floatToRawIntBits; +import static java.lang.StrictMath.floorDiv; +import static java.lang.StrictMath.floorMod; +import static java.lang.StrictMath.toIntExact; +import static java.nio.charset.StandardCharsets.UTF_8; +import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE_TIME; +import static java.time.format.ResolverStyle.LENIENT; +import static java.util.HexFormat.isHexDigit; +import static java.util.Objects.requireNonNull; +import static org.joda.time.DateTimeZone.UTC; + +public final class OpenXJsonDeserializer + implements LineDeserializer +{ + private final List types; + private final OpenXJsonOptions options; + private final RowDecoder rowDecoder; + + public OpenXJsonDeserializer(List columns, OpenXJsonOptions options) + { + this.options = requireNonNull(options, "options is null"); + + this.types = columns.stream() + .map(Column::type) + .collect(toImmutableList()); + + List timestampFormatters = options.getTimestampFormats().stream() + .map(DateTimeFormatter::ofPattern) + .map(formatter -> formatter.withZone(ZoneOffset.UTC)) + .collect(toImmutableList()); + + rowDecoder = new RowDecoder( + RowType.from(columns.stream() + .map(column -> field(column.name().toLowerCase(Locale.ROOT), column.type())) + .collect(toImmutableList())), + options, + columns.stream() + .map(Column::type) + .map(fieldType -> createDecoder(fieldType, options, timestampFormatters)) + .collect(toImmutableList())); + } + + @Override + public List getTypes() + { + return types; + } + + @Override + public void deserialize(LineBuffer lineBuffer, PageBuilder builder) + throws IOException + { + String line = new String(lineBuffer.getBuffer(), 0, lineBuffer.getLength(), UTF_8).trim(); + + // only object and array lines are supported; everything else results in a null row + Object jsonNode = null; + if (line.startsWith("[") || line.startsWith("{")) { + try { + // fields names from JSON are not mapped + jsonNode = JsonReader.readJson(line, originalValue -> new FieldName(originalValue, options.isCaseInsensitive(), options.isDotsInFieldNames(), ImmutableMap.of())); + } + catch (InvalidJsonException e) { + if (!options.isIgnoreMalformedJson()) { + throw e; + } + } + } + + if (jsonNode == null) { + // if json did not parse all columns are null + builder.declarePosition(); + for (int columnIndex = 0; columnIndex < types.size(); columnIndex++) { + builder.getBlockBuilder(columnIndex).appendNull(); + } + } + else { + rowDecoder.decode(jsonNode, builder); + } + } + + private static Decoder createDecoder(Type type, OpenXJsonOptions options, List timestampFormatters) + { + if (BOOLEAN.equals(type)) { + return new BooleanDecoder(); + } + if (BIGINT.equals(type)) { + return new BigintDecoder(); + } + if (INTEGER.equals(type)) { + return new IntegerDecoder(); + } + if (SMALLINT.equals(type)) { + return new SmallintDecoder(); + } + if (TINYINT.equals(type)) { + return new TinyintDecoder(); + } + if (type instanceof DecimalType decimalType) { + return new DecimalDecoder(decimalType); + } + if (REAL.equals(type)) { + return new RealDecoder(); + } + if (DOUBLE.equals(type)) { + return new DoubleDecoder(); + } + if (DATE.equals(type)) { + return new DateDecoder(); + } + if (type instanceof TimestampType timestampType) { + return new TimestampDecoder(timestampType, timestampFormatters); + } + if (VARBINARY.equals(type)) { + return new VarbinaryDecoder(); + } + if (type instanceof VarcharType varcharType) { + return new VarcharDecoder(varcharType); + } + if (type instanceof CharType charType) { + return new CharDecoder(charType); + } + if (type instanceof ArrayType arrayType) { + return new ArrayDecoder(createDecoder(arrayType.getElementType(), options, timestampFormatters)); + } + if (type instanceof MapType mapType) { + return new MapDecoder( + mapType, + createDecoder(mapType.getKeyType(), options, timestampFormatters), + createDecoder(mapType.getValueType(), options, timestampFormatters)); + } + if (type instanceof RowType rowType) { + return new RowDecoder( + rowType, + options, + rowType.getFields().stream() + .map(Field::getType) + .map(fieldType -> createDecoder(fieldType, options, timestampFormatters)) + .collect(toImmutableList())); + } + throw new UnsupportedOperationException("Unsupported column type: " + type); + } + + private abstract static class Decoder + { + public final void decode(Object jsonValue, BlockBuilder builder) + { + if (jsonValue == null) { + builder.appendNull(); + return; + } + decodeValue(jsonValue, builder); + } + + /** + * Implementations only receive a Boolean, String, Integer, Long, Double, BigDecimal, Map, or List, and never null. + */ + abstract void decodeValue(Object jsonValue, BlockBuilder builder); + } + + private static class BooleanDecoder + extends Decoder + { + @Override + void decodeValue(Object jsonValue, BlockBuilder builder) + { + BOOLEAN.writeBoolean(builder, parseBoolean(jsonValue)); + } + + private static boolean parseBoolean(Object jsonValue) + { + return Boolean.parseBoolean(jsonValue.toString()); + } + } + + private static class BigintDecoder + extends Decoder + { + @Override + void decodeValue(Object jsonValue, BlockBuilder builder) + { + if (!(jsonValue instanceof JsonString jsonString)) { + throw invalidJson("%s can not be coerced to a BIGINT".formatted(jsonValue.getClass().getSimpleName())); + } + + try { + BIGINT.writeLong(builder, parseLong(jsonString.value())); + return; + } + catch (NumberFormatException | ArithmeticException ignored) { + } + builder.appendNull(); + } + } + + private static class IntegerDecoder + extends Decoder + { + @Override + void decodeValue(Object jsonValue, BlockBuilder builder) + { + if (!(jsonValue instanceof JsonString jsonString)) { + throw invalidJson("%s can not be coerced to an INTEGER".formatted(jsonValue.getClass().getSimpleName())); + } + + try { + long longValue = parseLong(jsonString.value()); + if ((int) longValue == longValue) { + INTEGER.writeLong(builder, longValue); + return; + } + } + catch (NumberFormatException | ArithmeticException ignored) { + } + builder.appendNull(); + } + } + + private static class SmallintDecoder + extends Decoder + { + @Override + void decodeValue(Object jsonValue, BlockBuilder builder) + { + if (!(jsonValue instanceof JsonString jsonString)) { + throw invalidJson("%s can not be coerced to a SMALLINT".formatted(jsonValue.getClass().getSimpleName())); + } + + try { + long longValue = parseLong(jsonString.value()); + if ((short) longValue == longValue) { + SMALLINT.writeLong(builder, longValue); + return; + } + } + catch (NumberFormatException | ArithmeticException ignored) { + } + builder.appendNull(); + } + } + + private static class TinyintDecoder + extends Decoder + { + @Override + void decodeValue(Object jsonValue, BlockBuilder builder) + { + if (!(jsonValue instanceof JsonString jsonString)) { + throw invalidJson("%s can not be coerced to a TINYINT".formatted(jsonValue.getClass().getSimpleName())); + } + + try { + long longValue = parseLong(jsonString.value()); + if ((byte) longValue == longValue) { + TINYINT.writeLong(builder, longValue); + return; + } + } + catch (NumberFormatException | ArithmeticException ignored) { + } + builder.appendNull(); + } + } + + private static class DecimalDecoder + extends Decoder + { + private final DecimalType decimalType; + + public DecimalDecoder(DecimalType decimalType) + { + this.decimalType = decimalType; + } + + @Override + void decodeValue(Object jsonValue, BlockBuilder builder) + { + if (!(jsonValue instanceof JsonString jsonString)) { + throw invalidJson("%s can not be coerced to a %s".formatted(jsonValue.getClass().getSimpleName(), decimalType)); + } + + try { + BigDecimal bigDecimal = scaleDecimal(new BigDecimal(jsonString.value()), decimalType); + if (!overflows(bigDecimal, decimalType.getPrecision())) { + if (decimalType.isShort()) { + decimalType.writeLong(builder, bigDecimal.unscaledValue().longValueExact()); + } + else { + decimalType.writeObject(builder, Int128.valueOf(bigDecimal.unscaledValue())); + } + return; + } + } + catch (NumberFormatException ignored) { + } + builder.appendNull(); + } + } + + private static class RealDecoder + extends Decoder + { + @Override + void decodeValue(Object jsonValue, BlockBuilder builder) + { + if (!(jsonValue instanceof JsonString jsonString)) { + throw invalidJson("%s can not be coerced to a REAL".formatted(jsonValue.getClass().getSimpleName())); + } + + try { + REAL.writeLong(builder, floatToRawIntBits(Float.parseFloat(jsonString.value()))); + } + catch (NumberFormatException ignored) { + builder.appendNull(); + } + } + } + + private static class DoubleDecoder + extends Decoder + { + @Override + void decodeValue(Object jsonValue, BlockBuilder builder) + { + if (!(jsonValue instanceof JsonString jsonString)) { + throw invalidJson("%s can not be coerced to a DOUBLE".formatted(jsonValue.getClass().getSimpleName())); + } + + try { + DOUBLE.writeDouble(builder, Double.parseDouble(jsonString.value())); + } + catch (NumberFormatException e) { + builder.appendNull(); + } + } + } + + private static class DateDecoder + extends Decoder + { + @Override + void decodeValue(Object jsonValue, BlockBuilder builder) + { + if (!(jsonValue instanceof JsonString jsonString)) { + throw invalidJson("%s can not be coerced to a DATE".formatted(jsonValue.getClass().getSimpleName())); + } + + String dateString = jsonString.value(); + try { + DATE.writeLong(builder, toIntExact(parseHiveDate(dateString).toEpochDay())); + return; + } + catch (DateTimeParseException | ArithmeticException ignored) { + } + try { + DATE.writeLong(builder, toIntExact(parseDecimalHexOctalLong(dateString))); + return; + } + catch (NumberFormatException | ArithmeticException ignored) { + } + builder.appendNull(); + } + } + + private static class TimestampDecoder + extends Decoder + { + private final TimestampType timestampType; + private final List timestampFormatters; + private final TrinoTimestampEncoder> timestampEncoder; + + public TimestampDecoder(TimestampType timestampType, List timestampFormatters) + { + this.timestampType = timestampType; + this.timestampFormatters = timestampFormatters; + this.timestampEncoder = createTimestampEncoder(timestampType, UTC); + } + + @Override + void decodeValue(Object jsonValue, BlockBuilder builder) + { + if (!(jsonValue instanceof JsonString jsonString)) { + throw invalidJson("%s can not be coerced to a %s".formatted(jsonValue.getClass().getSimpleName(), timestampType)); + } + try { + DecodedTimestamp timestamp = parseTimestamp(jsonString.value(), timestampFormatters); + timestampEncoder.write(timestamp, builder); + } + catch (DateTimeParseException | NumberFormatException | ArithmeticException e) { + builder.appendNull(); + } + } + + private static final int MIN_NUMERIC_TIMESTAMP_MILLIS_LENGTH = 13; + + public static DecodedTimestamp parseTimestamp(String value, List timestampFormatters) + { + // first try specified formatters + for (DateTimeFormatter formatter : timestampFormatters) { + try { + ZonedDateTime zonedDateTime = formatter.parse(value, ZonedDateTime::from); + long epochSeconds = zonedDateTime.toEpochSecond(); + return new DecodedTimestamp(epochSeconds, zonedDateTime.getNano()); + } + catch (DateTimeParseException ignored) { + } + } + + // always try the build in timestamp formats + + // timestamp with time + if (value.indexOf(':') > 0) { + if (value.endsWith("z") || value.endsWith("Z") || HAS_TZ_OFFSET.matcher(value).matches()) { + try { + ZonedDateTime zonedDateTime = ZonedDateTime.parse(value, ZONED_DATE_TIME_PARSER_NO_COLON); + zonedDateTime = zonedDateTime.withZoneSameInstant(ZoneOffset.UTC); + return new DecodedTimestamp(zonedDateTime.toEpochSecond(), zonedDateTime.getNano()); + } + catch (DateTimeParseException ignored) { + } + try { + ZonedDateTime zonedDateTime = ZonedDateTime.parse(value, ZONED_DATE_TIME_PARSER_WITH_COLON); + zonedDateTime = zonedDateTime.withZoneSameInstant(ZoneOffset.UTC); + return new DecodedTimestamp(zonedDateTime.toEpochSecond(), zonedDateTime.getNano()); + } + catch (DateTimeParseException ignored) { + } + } + return parseHiveTimestamp(value); + } + + if (!CharMatcher.anyOf("-+.0123456789").matchesAllOf(value)) { + throw new DateTimeParseException("Invalid timestamp", value, 0); + } + + // decimal seconds + if (value.indexOf('.') >= 0) { + long epochMillis = new BigDecimal(value) + .scaleByPowerOfTen(3) + .setScale(0, RoundingMode.DOWN) + .longValueExact(); + return ofEpochMilli(epochMillis); + } + + // integer millis or seconds based on text length + long timestampNumber = Long.parseLong(value); + if (value.length() >= MIN_NUMERIC_TIMESTAMP_MILLIS_LENGTH) { + return ofEpochMilli(timestampNumber); + } + return new DecodedTimestamp(timestampNumber, 0); + } + + // There is no way to have a parser that supports a zone wih an optional colon, so we must have two copies of the parser + @SuppressWarnings("SpellCheckingInspection") + private static final DateTimeFormatter ZONED_DATE_TIME_PARSER_NO_COLON = new DateTimeFormatterBuilder() + .parseLenient() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_TIME) + .optionalStart() + .appendOffset("+HHMM", "Z") + .optionalEnd() + .toFormatter() + .withResolverStyle(LENIENT); + + private static final DateTimeFormatter ZONED_DATE_TIME_PARSER_WITH_COLON = new DateTimeFormatterBuilder() + .parseLenient() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_TIME) + .optionalStart() + .appendOffset("+HH:MM", "Z") + .optionalEnd() + .toFormatter() + .withResolverStyle(LENIENT); + + private static final Pattern HAS_TZ_OFFSET = Pattern.compile(".+([+\\-])\\d{2}:?\\d{2}$"); + + private static DecodedTimestamp ofEpochMilli(long epochMillis) + { + long epochSeconds = floorDiv(epochMillis, (long) MILLISECONDS_PER_SECOND); + long fractionalSecond = floorMod(epochMillis, (long) MILLISECONDS_PER_SECOND); + int nanosOfSecond = toIntExact(fractionalSecond * (long) NANOSECONDS_PER_MILLISECOND); + return new DecodedTimestamp(epochSeconds, nanosOfSecond); + } + } + + private static class VarbinaryDecoder + extends Decoder + { + @Override + void decodeValue(Object jsonValue, BlockBuilder builder) + { + if (!(jsonValue instanceof JsonString jsonString)) { + throw invalidJson("%s can not be coerced to a VARBINARY".formatted(jsonValue.getClass().getSimpleName())); + } + + if (!jsonString.quoted()) { + throw invalidJson("Unquoted JSON string is not allowed for VARBINARY: " + jsonValue.getClass().getSimpleName()); + } + + Slice binaryValue = Slices.wrappedBuffer(Base64.getDecoder().decode(jsonString.value())); + VARBINARY.writeSlice(builder, binaryValue); + } + } + + private static class VarcharDecoder + extends Decoder + { + private final VarcharType varcharType; + + public VarcharDecoder(VarcharType varcharType) + { + this.varcharType = varcharType; + } + + @Override + void decodeValue(Object jsonValue, BlockBuilder builder) + { + String string; + if (jsonValue instanceof Map jsonObject) { + string = writeJsonObject(jsonObject); + } + else if (jsonValue instanceof List jsonList) { + string = writeJsonArray(jsonList); + } + else { + JsonString jsonString = (JsonString) jsonValue; + string = canonicalizeJsonString(jsonString); + } + varcharType.writeSlice(builder, truncateToLength(Slices.utf8Slice(string), varcharType)); + } + } + + private static class CharDecoder + extends Decoder + { + private final CharType charType; + + public CharDecoder(CharType charType) + { + this.charType = charType; + } + + @Override + void decodeValue(Object jsonValue, BlockBuilder builder) + { + String string; + if (jsonValue instanceof Map jsonObject) { + string = writeJsonObject(jsonObject); + } + else if (jsonValue instanceof List jsonList) { + string = writeJsonArray(jsonList); + } + else { + JsonString jsonString = (JsonString) jsonValue; + string = canonicalizeJsonString(jsonString); + } + charType.writeSlice(builder, truncateToLengthAndTrimSpaces(Slices.utf8Slice(string), charType)); + } + } + + private static class ArrayDecoder + extends Decoder + { + private final Decoder elementDecoder; + + public ArrayDecoder(Decoder elementDecoder) + { + this.elementDecoder = elementDecoder; + } + + @Override + void decodeValue(Object jsonValue, BlockBuilder builder) + { + // empty string is coerced to null; otherwise string is coerced to single element array below + if (jsonValue instanceof JsonString stingValue && stingValue.value().isEmpty()) { + builder.appendNull(); + return; + } + + BlockBuilder elementBuilder = builder.beginBlockEntry(); + + if (jsonValue instanceof List jsonArray) { + for (Object element : jsonArray) { + elementDecoder.decode(element, elementBuilder); + } + } + else { + // all other values are coerced to a single element array + elementDecoder.decode(jsonValue, elementBuilder); + } + builder.closeEntry(); + } + } + + private static class MapDecoder + extends Decoder + { + private final Decoder keyDecoder; + private final Decoder valueDecoder; + private final Type keyType; + + private final DistinctMapKeys distinctMapKeys; + private BlockBuilder keyBlockBuilder; + + public MapDecoder(MapType mapType, Decoder keyDecoder, Decoder valueDecoder) + { + this.keyType = mapType.getKeyType(); + this.keyDecoder = keyDecoder; + this.valueDecoder = valueDecoder; + + this.distinctMapKeys = new DistinctMapKeys(mapType, true); + this.keyBlockBuilder = mapType.getKeyType().createBlockBuilder(null, 128); + } + + @Override + void decodeValue(Object jsonValue, BlockBuilder builder) + { + // string containing only whitespace is coerced to null; otherwise an exception is thrown + if (jsonValue instanceof JsonString jsonString) { + // string containing only whitespace is coerced to null + if (!jsonString.value().trim().isEmpty()) { + throw invalidJson("Primitive can not be coerced to a MAP"); + } + builder.appendNull(); + return; + } + + checkArgument(jsonValue instanceof Map, "%s can not be coerced to a MAP", jsonValue.getClass().getSimpleName()); + Map jsonObject = (Map) jsonValue; + Set fieldNames = jsonObject.keySet(); + // field names strings are distinct, but after parsing the values may not be distinct (e.g., boolean) + Block keyBlock = readKeys(fieldNames); + boolean[] distinctKeys = distinctMapKeys.selectDistinctKeys(keyBlock); + + BlockBuilder entryBuilder = builder.beginBlockEntry(); + int keyIndex = 0; + for (Object fieldName : fieldNames) { + if (distinctKeys[keyIndex]) { + keyType.appendTo(keyBlock, keyIndex, entryBuilder); + valueDecoder.decode(jsonObject.get(fieldName), entryBuilder); + } + keyIndex++; + } + builder.closeEntry(); + } + + private Block readKeys(Collection fieldNames) + { + for (Object fieldName : fieldNames) { + // field names are always processed as a quoted JSON string even though they may + // have not been quoted in the original JSON text + JsonString jsonValue = new JsonString(fieldName.toString(), true); + keyDecoder.decode(jsonValue, keyBlockBuilder); + } + + Block keyBlock = keyBlockBuilder.build(); + keyBlockBuilder = keyType.createBlockBuilder(null, keyBlock.getPositionCount()); + return keyBlock; + } + } + + private static class RowDecoder + extends Decoder + { + private final List fieldNames; + private final List fieldDecoders; + private final boolean dotsInKeyNames; + + public RowDecoder(RowType rowType, OpenXJsonOptions options, List fieldDecoders) + { + this.fieldNames = rowType.getFields().stream() + .map(field -> field.getName().orElseThrow()) + .map(fieldName -> fieldName.toLowerCase(Locale.ROOT)) + .map(originalValue -> new FieldName(originalValue, options)) + .collect(toImmutableList()); + this.fieldDecoders = fieldDecoders; + this.dotsInKeyNames = options.isDotsInFieldNames(); + } + + public void decode(Object jsonValue, PageBuilder builder) + throws IOException + { + builder.declarePosition(); + decodeValue(jsonValue, builder::getBlockBuilder); + } + + @Override + void decodeValue(Object jsonValue, BlockBuilder builder) + { + SingleRowBlockWriter currentBuilder = (SingleRowBlockWriter) builder.beginBlockEntry(); + decodeValue(jsonValue, currentBuilder::getFieldBlockBuilder); + builder.closeEntry(); + } + + private void decodeValue(Object jsonValue, IntFunction fieldBuilders) + { + if (jsonValue instanceof JsonString jsonString) { + decodeValueFromString(jsonString, fieldBuilders); + } + else if (jsonValue instanceof Map jsonObject) { + decodeValueFromMap(jsonObject, fieldBuilders); + } + else if (jsonValue instanceof List jsonArray) { + decodeValueFromList(jsonArray, fieldBuilders); + } + else { + throw invalidJson("Expected JSON object: " + jsonValue.getClass().getSimpleName()); + } + } + + private void decodeValueFromString(JsonString jsonString, IntFunction fieldBuilders) + { + // string containing only whitespace is coerced to a row with all fields set to null; otherwise an exception is thrown + if (!jsonString.value().trim().isEmpty()) { + throw invalidJson("Primitive can not be coerced to a ROW"); + } + + for (int i = 0; i < fieldDecoders.size(); i++) { + BlockBuilder blockBuilder = fieldBuilders.apply(i); + blockBuilder.appendNull(); + } + } + + private void decodeValueFromMap(Map jsonObject, IntFunction fieldBuilders) + { + for (int i = 0; i < fieldDecoders.size(); i++) { + FieldName fieldName = fieldNames.get(i); + Object fieldValue = null; + if (jsonObject.containsKey(fieldName)) { + fieldValue = jsonObject.get(fieldName); + } + else if (dotsInKeyNames) { + // check if any field matches any keys after dots have been replaced with underscores + for (Object key : jsonObject.keySet()) { + if (fieldName.originalValueMatchesDottedFieldName((FieldName) key)) { + fieldValue = jsonObject.get(key); + break; + } + } + } + + BlockBuilder blockBuilder = fieldBuilders.apply(i); + if (fieldValue == null) { + blockBuilder.appendNull(); + } + else { + fieldDecoders.get(i).decode(fieldValue, blockBuilder); + } + } + } + + private void decodeValueFromList(List jsonArray, IntFunction fieldBuilders) + { + for (int i = 0; i < fieldDecoders.size(); i++) { + Object fieldValue = jsonArray.size() > i ? jsonArray.get(i) : null; + BlockBuilder blockBuilder = fieldBuilders.apply(i); + if (fieldValue == null) { + blockBuilder.appendNull(); + } + else { + fieldDecoders.get(i).decode(fieldValue, blockBuilder); + } + } + } + } + + public static long parseLong(String stringValue) + throws NumberFormatException, ArithmeticException + { + try { + return parseDecimalHexOctalLong(stringValue); + } + catch (NumberFormatException ignored) { + } + + BigDecimal bigDecimal = new BigDecimal(stringValue).setScale(0, RoundingMode.DOWN); + return bigDecimal.longValueExact(); + } + + public static long parseDecimalHexOctalLong(String stringValue) + throws NumberFormatException + { + if (isHex(stringValue)) { + // Negative values will fail + return Long.parseLong(stringValue.substring(2), 16); + } + if (isOctal(stringValue)) { + // Negative values will fail + return Long.parseLong(stringValue.substring(1), 8); + } + return Long.parseLong(stringValue); + } + + private static boolean isHex(String s) + { + // This does not allow for `0x-123` + return s.length() >= 3 && + s.charAt(0) == '0' && + toLowerCase(s.charAt(1)) == 'x' && + isHexDigit(s.charAt(2)); + } + + private static boolean isOctal(String s) + { + // This does not allow for `0-123` + return s.length() >= 2 && + s.charAt(0) == '0' && + isOctalDigit(s.charAt(1)); + } + + private static boolean isOctalDigit(char c) + { + int digit = (int) c - (int) '0'; + return digit >= 0 && digit <= 8; + } + + private static RuntimeException invalidJson(String message) + { + return new RuntimeException("Invalid JSON: " + message); + } + + private static final class FieldName + { + private final String originalValue; + private final String mappedName; + private final boolean caseInsensitive; + private final int hashCode; + private final boolean canMatchDottedFieldName; + private final boolean isDottedFieldName; + + public FieldName(String originalValue, OpenXJsonOptions options) + { + this(originalValue, options.isCaseInsensitive(), options.isDotsInFieldNames(), options.getFieldNameMappings()); + } + + public FieldName(String originalValue, boolean caseInsensitive, boolean dotsInFieldNames, Map fieldNameMappings) + { + this.originalValue = requireNonNull(originalValue, "originalValue is null"); + this.mappedName = fieldNameMappings.getOrDefault(originalValue, originalValue); + this.caseInsensitive = caseInsensitive; + + // the hashcode is always used, so just precompute it + // equality is based on the mapped value + if (caseInsensitive) { + this.hashCode = mappedName.toLowerCase(Locale.ROOT).hashCode(); + } + else { + this.hashCode = mappedName.hashCode(); + } + + // only names containing underscores can match dotted field names + canMatchDottedFieldName = dotsInFieldNames && originalValue.indexOf('_') >= 0; + isDottedFieldName = dotsInFieldNames && originalValue.indexOf('.') >= 0; + } + + /** + * If the supplied name contains dots, replace all dots with underscores, and compare to + * the original value using the set case sensitivity. This strange behavior is follows + * the logic of the original code. + */ + public boolean originalValueMatchesDottedFieldName(FieldName dottedName) + { + // skip impossible matches + if (!canMatchDottedFieldName || !dottedName.isDottedFieldName) { + return false; + } + + // substitute . with _ and check if name matches the *original* value + // Note: this is not precomputed to save memory + // Note: this could be sped up with a custom equality method + String underscoreName = dottedName.originalValue.replace('.', '_'); + return caseInsensitive ? originalValue.equalsIgnoreCase(underscoreName) : originalValue.equals(underscoreName); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + FieldName that = (FieldName) o; + // equality is based on the mapped value + return caseInsensitive ? mappedName.equalsIgnoreCase(that.mappedName) : mappedName.equals(that.mappedName); + } + + @Override + public int hashCode() + { + return hashCode; + } + + @Override + public String toString() + { + return originalValue; + } + } +} diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/OpenXJsonDeserializerFactory.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/OpenXJsonDeserializerFactory.java new file mode 100644 index 000000000000..0bc798118df1 --- /dev/null +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/OpenXJsonDeserializerFactory.java @@ -0,0 +1,39 @@ +/* + * 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.trino.hive.formats.line.openxjson; + +import io.trino.hive.formats.line.Column; +import io.trino.hive.formats.line.LineDeserializer; +import io.trino.hive.formats.line.LineDeserializerFactory; + +import java.util.List; +import java.util.Map; + +import static io.trino.hive.formats.line.openxjson.OpenXJsonOptions.HIVE_SERDE_CLASS_NAMES; + +public class OpenXJsonDeserializerFactory + implements LineDeserializerFactory +{ + @Override + public List getHiveSerDeClassNames() + { + return HIVE_SERDE_CLASS_NAMES; + } + + @Override + public LineDeserializer create(List columns, Map serdeProperties) + { + return new OpenXJsonDeserializer(columns, OpenXJsonOptions.fromSchema(serdeProperties)); + } +} diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/OpenXJsonOptions.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/OpenXJsonOptions.java new file mode 100644 index 000000000000..85e315dd0656 --- /dev/null +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/OpenXJsonOptions.java @@ -0,0 +1,243 @@ +/* + * 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.trino.hive.formats.line.openxjson; + +import com.google.common.base.Splitter; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; + +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Map.Entry; + +public class OpenXJsonOptions +{ + public static final OpenXJsonOptions DEFAULT_OPEN_X_JSON_OPTIONS = builder().build(); + + static final ImmutableList HIVE_SERDE_CLASS_NAMES = ImmutableList.of("org.openx.data.jsonserde.JsonSerDe"); + + private static final String CASE_INSENSITIVE_KEY = "case.insensitive"; + private static final String FIELD_MAPPING_KEY_PREFIX = "mapping."; + private static final String IGNORE_MALFORMED_JSON_KEY = "ignore.malformed.json"; + private static final String DOTS_IN_FIELD_NAMES_KEY = "dots.in.keys"; + private static final String EXPLICIT_NULL_KEY = "explicit.null"; + private static final String TIMESTAMP_FORMATS_KEY = "timestamp.formats"; + + private final boolean ignoreMalformedJson; + private final Map fieldNameMappings; + private final boolean caseInsensitive; + private final boolean dotsInFieldNames; + private final boolean explicitNull; + private final List timestampFormats; + + private OpenXJsonOptions( + boolean ignoreMalformedJson, + Map fieldNameMappings, + boolean caseInsensitive, + boolean dotsInFieldNames, + boolean explicitNull, + List timestampFormats) + { + this.ignoreMalformedJson = ignoreMalformedJson; + this.fieldNameMappings = ImmutableMap.copyOf(fieldNameMappings); + this.caseInsensitive = caseInsensitive; + this.dotsInFieldNames = dotsInFieldNames; + this.explicitNull = explicitNull; + this.timestampFormats = ImmutableList.copyOf(timestampFormats); + } + + public boolean isIgnoreMalformedJson() + { + return ignoreMalformedJson; + } + + public Map getFieldNameMappings() + { + return fieldNameMappings; + } + + public boolean isCaseInsensitive() + { + return caseInsensitive; + } + + public boolean isDotsInFieldNames() + { + return dotsInFieldNames; + } + + public boolean isExplicitNull() + { + return explicitNull; + } + + public List getTimestampFormats() + { + return timestampFormats; + } + + public Map toSchema() + { + ImmutableMap.Builder schema = ImmutableMap.builder(); + + if (ignoreMalformedJson) { + schema.put(IGNORE_MALFORMED_JSON_KEY, "true"); + } + + if (!caseInsensitive) { + schema.put(CASE_INSENSITIVE_KEY, "false"); + } + + for (Entry entry : fieldNameMappings.entrySet()) { + schema.put(FIELD_MAPPING_KEY_PREFIX + entry.getKey(), entry.getValue()); + } + + if (dotsInFieldNames) { + schema.put(DOTS_IN_FIELD_NAMES_KEY, "true"); + } + + if (explicitNull) { + schema.put(EXPLICIT_NULL_KEY, "true"); + } + + if (!timestampFormats.isEmpty()) { + schema.put(TIMESTAMP_FORMATS_KEY, String.join(",", timestampFormats)); + } + return schema.buildOrThrow(); + } + + public static OpenXJsonOptions fromSchema(Map serdeProperties) + { + Builder builder = builder(); + + if ("true".equalsIgnoreCase(serdeProperties.get(IGNORE_MALFORMED_JSON_KEY))) { + builder.ignoreMalformedJson(); + } + + boolean caseInsensitive = "true".equalsIgnoreCase(serdeProperties.getOrDefault(CASE_INSENSITIVE_KEY, "true")); + if (!caseInsensitive) { + builder.caseSensitive(); + } + + for (Entry entry : serdeProperties.entrySet()) { + String key = entry.getKey(); + if (key.startsWith(FIELD_MAPPING_KEY_PREFIX)) { + String hiveField = key.substring(FIELD_MAPPING_KEY_PREFIX.length()); + String jsonField = caseInsensitive ? entry.getValue().toLowerCase(Locale.ROOT) : entry.getValue(); + builder.addFieldMapping(hiveField, jsonField); + } + } + + if ("true".equalsIgnoreCase(serdeProperties.get(DOTS_IN_FIELD_NAMES_KEY))) { + builder.dotsInFieldNames(); + } + + if ("true".equalsIgnoreCase(serdeProperties.get(EXPLICIT_NULL_KEY))) { + builder.explicitNull(); + } + + String timestampFormats = serdeProperties.get(TIMESTAMP_FORMATS_KEY); + if (timestampFormats != null) { + // Note there is no escaping for commas in timestamps + builder.timestampFormats(Splitter.on(',').splitToList(timestampFormats)); + } + + return builder.build(); + } + + public static Builder builder() + { + return new Builder(); + } + + public static Builder builder(OpenXJsonOptions options) + { + return new Builder(options); + } + + public static class Builder + { + private boolean ignoreMalformedJson; + private boolean caseInsensitive = true; + private final ImmutableMap.Builder fieldNameMappings = ImmutableMap.builder(); + private boolean dotsInFieldNames; + private boolean explicitNull; + private List timestampFormats = ImmutableList.of(); + + public Builder() {} + + private Builder(OpenXJsonOptions options) + { + ignoreMalformedJson = options.isIgnoreMalformedJson(); + caseInsensitive = options.isCaseInsensitive(); + fieldNameMappings.putAll(options.getFieldNameMappings()); + dotsInFieldNames = options.isDotsInFieldNames(); + explicitNull = options.isExplicitNull(); + timestampFormats = options.getTimestampFormats(); + } + + public Builder ignoreMalformedJson() + { + this.ignoreMalformedJson = true; + return this; + } + + public Builder addFieldMapping(String hiveField, String jsonField) + { + this.fieldNameMappings.put(hiveField, jsonField); + return this; + } + + public Builder caseSensitive() + { + this.caseInsensitive = false; + return this; + } + + public Builder dotsInFieldNames() + { + this.dotsInFieldNames = true; + return this; + } + + public Builder explicitNull() + { + this.explicitNull = true; + return this; + } + + public Builder timestampFormats(String... timestampFormats) + { + return timestampFormats(ImmutableList.copyOf(timestampFormats)); + } + + public Builder timestampFormats(List timestampFormats) + { + this.timestampFormats = timestampFormats; + return this; + } + + public OpenXJsonOptions build() + { + return new OpenXJsonOptions( + ignoreMalformedJson, + fieldNameMappings.buildOrThrow(), + caseInsensitive, + dotsInFieldNames, + explicitNull, + timestampFormats); + } + } +} diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/OpenXJsonSerializer.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/OpenXJsonSerializer.java new file mode 100644 index 000000000000..b48a8d84ea2d --- /dev/null +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/OpenXJsonSerializer.java @@ -0,0 +1,283 @@ +/* + * 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.trino.hive.formats.line.openxjson; + +import com.google.common.collect.ImmutableList; +import io.airlift.slice.SliceOutput; +import io.trino.hive.formats.HiveFormatUtils; +import io.trino.hive.formats.line.Column; +import io.trino.hive.formats.line.LineSerializer; +import io.trino.spi.Page; +import io.trino.spi.block.Block; +import io.trino.spi.type.ArrayType; +import io.trino.spi.type.CharType; +import io.trino.spi.type.Chars; +import io.trino.spi.type.DecimalType; +import io.trino.spi.type.MapType; +import io.trino.spi.type.RowType; +import io.trino.spi.type.RowType.Field; +import io.trino.spi.type.SqlDecimal; +import io.trino.spi.type.SqlTimestamp; +import io.trino.spi.type.TimestampType; +import io.trino.spi.type.Type; +import io.trino.spi.type.VarcharType; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; +import java.time.format.SignStyle; +import java.util.ArrayList; +import java.util.Base64; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.hive.formats.line.openxjson.JsonWriter.writeJsonArray; +import static io.trino.hive.formats.line.openxjson.JsonWriter.writeJsonObject; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.DateType.DATE; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.spi.type.IntegerType.INTEGER; +import static io.trino.spi.type.RealType.REAL; +import static io.trino.spi.type.SmallintType.SMALLINT; +import static io.trino.spi.type.TinyintType.TINYINT; +import static io.trino.spi.type.VarbinaryType.VARBINARY; +import static java.lang.Float.intBitsToFloat; +import static java.time.temporal.ChronoField.DAY_OF_MONTH; +import static java.time.temporal.ChronoField.HOUR_OF_DAY; +import static java.time.temporal.ChronoField.MINUTE_OF_HOUR; +import static java.time.temporal.ChronoField.MONTH_OF_YEAR; +import static java.time.temporal.ChronoField.NANO_OF_SECOND; +import static java.time.temporal.ChronoField.SECOND_OF_MINUTE; +import static java.time.temporal.ChronoField.YEAR; +import static java.util.Objects.requireNonNull; + +public class OpenXJsonSerializer + implements LineSerializer +{ + private static final DateTimeFormatter UTC_PRINT_FORMATTER = new DateTimeFormatterBuilder() + .appendValue(YEAR, 1, 10, SignStyle.NORMAL) + .appendLiteral('-') + .appendValue(MONTH_OF_YEAR, 2, 2, SignStyle.NORMAL) + .appendLiteral('-') + .appendValue(DAY_OF_MONTH, 2, 2, SignStyle.NORMAL) + .appendLiteral('T') + .appendValue(HOUR_OF_DAY, 2, 2, SignStyle.NORMAL) + .appendLiteral(':') + .appendValue(MINUTE_OF_HOUR, 2, 2, SignStyle.NORMAL) + .appendLiteral(':') + .appendValue(SECOND_OF_MINUTE, 2, 2, SignStyle.NORMAL) + .optionalStart() + .appendFraction(NANO_OF_SECOND, 0, 9, true) + .optionalEnd() + .appendLiteral('Z') + .toFormatter(); + + private final List columns; + private final OpenXJsonOptions options; + + public OpenXJsonSerializer(List columns, OpenXJsonOptions options) + { + this.columns = ImmutableList.copyOf(columns); + this.options = requireNonNull(options, "options is null"); + for (Column column : columns) { + if (!isSupportedType(column.type())) { + throw new IllegalArgumentException("Unsupported column type: " + column); + } + } + } + + @Override + public List getTypes() + { + return columns.stream().map(Column::type).collect(toImmutableList()); + } + + @Override + public void write(Page page, int position, SliceOutput sliceOutput) + throws IOException + { + Map jsonObject = new LinkedHashMap<>(); + + for (int columnIndex = 0; columnIndex < columns.size(); columnIndex++) { + Column column = columns.get(columnIndex); + String fieldName = column.name(); + + Block block = page.getBlock(columnIndex); + Object fieldValue = writeValue(column.type(), block, position); + if (options.isExplicitNull() || fieldValue != null) { + jsonObject.put(fieldName, fieldValue); + } + } + sliceOutput.write(writeJsonObject(jsonObject).getBytes(StandardCharsets.UTF_8)); + } + + private Object writeValue(Type type, Block block, int position) + throws InvalidJsonException + { + if (block.isNull(position)) { + return null; + } + + if (BOOLEAN.equals(type)) { + return BOOLEAN.getBoolean(block, position); + } + else if (BIGINT.equals(type)) { + return BIGINT.getLong(block, position); + } + else if (INTEGER.equals(type)) { + return INTEGER.getLong(block, position); + } + else if (SMALLINT.equals(type)) { + return SMALLINT.getLong(block, position); + } + else if (TINYINT.equals(type)) { + return TINYINT.getLong(block, position); + } + else if (type instanceof DecimalType) { + // decimal type is read-only in Hive, but we support it + SqlDecimal value = (SqlDecimal) type.getObjectValue(null, block, position); + return value.toBigDecimal().toString(); + } + else if (REAL.equals(type)) { + return intBitsToFloat((int) REAL.getLong(block, position)); + } + else if (DOUBLE.equals(type)) { + return DOUBLE.getDouble(block, position); + } + else if (DATE.equals(type)) { + // date type is read-only in Hive, but we support it + return HiveFormatUtils.formatHiveDate(block, position); + } + else if (type instanceof TimestampType) { + SqlTimestamp objectValue = (SqlTimestamp) type.getObjectValue(null, block, position); + LocalDateTime localDateTime = objectValue.toLocalDateTime(); + return UTC_PRINT_FORMATTER.format(localDateTime); + } + else if (VARBINARY.equals(type)) { + // varbinary type is read-only in Hive, but we support it + return Base64.getEncoder().encodeToString(VARBINARY.getSlice(block, position).getBytes()); + } + else if (type instanceof VarcharType) { + return type.getSlice(block, position).toStringUtf8(); + } + else if (type instanceof CharType charType) { + // char type is read-only in Hive, but we support it + return Chars.padSpaces(charType.getSlice(block, position), charType).toStringUtf8(); + } + else if (type instanceof ArrayType arrayType) { + Type elementType = arrayType.getElementType(); + Block arrayBlock = arrayType.getObject(block, position); + + List jsonArray = new ArrayList<>(); + for (int arrayIndex = 0; arrayIndex < arrayBlock.getPositionCount(); arrayIndex++) { + Object elementValue = writeValue(elementType, arrayBlock, arrayIndex); + jsonArray.add(elementValue); + } + return jsonArray; + } + else if (type instanceof MapType mapType) { + Type keyType = mapType.getKeyType(); + if (isStructuralType(keyType)) { + throw new RuntimeException("Unsupported map key type: " + keyType); + } + Type valueType = mapType.getValueType(); + Block mapBlock = mapType.getObject(block, position); + + Map jsonMap = new LinkedHashMap<>(); + for (int mapIndex = 0; mapIndex < mapBlock.getPositionCount(); mapIndex += 2) { + try { + Object key = writeValue(keyType, mapBlock, mapIndex); + if (key == null) { + throw new RuntimeException("OpenX JsonSerDe can not write a null map key"); + } + String fieldName; + if (key instanceof Map jsonObject) { + fieldName = writeJsonObject(jsonObject); + } + else if (key instanceof List list) { + fieldName = writeJsonArray(list); + } + else { + fieldName = key.toString(); + } + + Object value = writeValue(valueType, mapBlock, mapIndex + 1); + jsonMap.put(fieldName, value); + } + catch (InvalidJsonException ignored) { + } + } + return jsonMap; + } + else if (type instanceof RowType rowType) { + List fields = rowType.getFields(); + Block rowBlock = rowType.getObject(block, position); + + Map jsonObject = new LinkedHashMap<>(); + for (int fieldIndex = 0; fieldIndex < fields.size(); fieldIndex++) { + Field field = fields.get(fieldIndex); + String fieldName = field.getName().orElseThrow(); + Object fieldValue = writeValue(field.getType(), rowBlock, fieldIndex); + if (options.isExplicitNull() || fieldValue != null) { + jsonObject.put(fieldName, fieldValue); + } + } + return jsonObject; + } + else { + throw new UnsupportedOperationException("Unsupported column type: " + type); + } + } + + public static boolean isSupportedType(Type type) + { + if (type instanceof ArrayType arrayType) { + return isSupportedType(arrayType.getElementType()); + } + if (type instanceof MapType mapType) { + return !isStructuralType(mapType.getKeyType()) && + isSupportedType(mapType.getKeyType()) && + isSupportedType(mapType.getValueType()); + } + if (type instanceof RowType rowType) { + return rowType.getFields().stream() + .map(Field::getType) + .allMatch(OpenXJsonSerializer::isSupportedType); + } + + return BOOLEAN.equals(type) || + BIGINT.equals(type) || + INTEGER.equals(type) || + SMALLINT.equals(type) || + TINYINT.equals(type) || + type instanceof DecimalType || + REAL.equals(type) || + DOUBLE.equals(type) || + DATE.equals(type) || + type instanceof TimestampType || + VARBINARY.equals(type) || + type instanceof VarcharType || + type instanceof CharType; + } + + private static boolean isStructuralType(Type type) + { + return type instanceof MapType || type instanceof ArrayType || type instanceof RowType; + } +} diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/OpenXJsonSerializerFactory.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/OpenXJsonSerializerFactory.java new file mode 100644 index 000000000000..eedd3c001ab6 --- /dev/null +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/OpenXJsonSerializerFactory.java @@ -0,0 +1,39 @@ +/* + * 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.trino.hive.formats.line.openxjson; + +import io.trino.hive.formats.line.Column; +import io.trino.hive.formats.line.LineSerializer; +import io.trino.hive.formats.line.LineSerializerFactory; + +import java.util.List; +import java.util.Map; + +import static io.trino.hive.formats.line.openxjson.OpenXJsonOptions.HIVE_SERDE_CLASS_NAMES; + +public class OpenXJsonSerializerFactory + implements LineSerializerFactory +{ + @Override + public List getHiveSerDeClassNames() + { + return HIVE_SERDE_CLASS_NAMES; + } + + @Override + public LineSerializer create(List columns, Map serdeProperties) + { + return new OpenXJsonSerializer(columns, OpenXJsonOptions.fromSchema(serdeProperties)); + } +} diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/README.md b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/README.md new file mode 100644 index 000000000000..1573b97a35bb --- /dev/null +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/README.md @@ -0,0 +1,253 @@ +# OpenX JSON format + +This code in this package implements the OpenX JSON SerDe (`org.openx.data.jsonserde.JsonSerDe`) +with extensions. The behavior is based on the implementations maintained by +[Rcongiu](https://github.com/rcongiu/Hive-JSON-Serde) and [Starburst](https://github.com/starburstdata/hive-json-serde). +The Rcongiu implementation is based on very old Hive systems, has many quirks and bugs, and is not +actively maintained (as of this writing). The Starburst version is much more modern, and actively +maintained, but has it own quirks. This implementation generally a super set of both of these +implementations, including support for reading and writing more types, and generally attempts to +normalize the behavior of type mappers, and line processing. Except for some noted cases below, +this implementation will read JSON that the other implementations can read, and can read the JSON +written by the other implementations. Additionally, the JSON written by this implementation can +be read by the other implementations, assuming they support the data type at all. + +# Line Format + +Hive SerDe operate on a per-line basis, so multi line JSON can not be supported. Each line is +processed as follows: + +1. If the first non-whitespace character of the line is not `{` or `]`, a row with all fields + set to null is produced. +2. The line is parsed using the very lenient [open-json](https://github.com/tdunning/open-json) + parser. A parser failure can be ignored by setting the `ignore.malformed.json` table property, + and in this case a row with all fields set to null is produced. +3. Any text after the close of the JSON object or array is ignored. +4. The JSON is parsed using the Type Mapping rules below. + +# Type Mapping + +The JSON parser will only produce three Java types: `Map`, `List`, and `JsonString`. A `JsonString` +is normal quoted JSON string, or an unquoted JSON value. In valid JSON, unquoted strings are +limited to numbers, and boolean `true` or `false`, but the lenient parser used in OpenX JSON +SerDe allow for field names and value to be any unquoted string. + +Specific type mapping rules are described below, but, general guidelines are: + +* Parse failures result in `null`. +* Values that are outside the allowed type range result in `null`. +* JSON object or array values for numeric or temporal types, throw an exception. +* Text types support all JSON values by rendering the value as canonical JSON. + +## BOOLEAN + +`true` only if quoted or unquoted string is equals ignoring case to `true`. All other values, +including array and object, result in `false`. + +### Known Differences + +* Rcongiu SerDe fails on array or object value. + +## BIGINT, INTEGER, SMALLINT, and TINYINT + +Values that fail to parse properly, or that are outside the allowed numeric range, result in `null`. + +### Formats + +* Any value that can be parsed by `new BigDecimal(String)` +* Hex `0x1234`. Negative numbers are not supported. +* Octal `01234`. Negative numbers are not supported. + +### Known Differences + +* Rcongiu fails on values with decimals or exponents. +* Starburst performs a narrowing cast to expected size. +* Starburst and Rcongiu do not support octal in map keys or quoted octal values. +* Starburst and Rcongiu both have a bug that allows negative hex and octal values formatted like `0x-123` or `0-123`. + +## DOUBLE and REAL + +Values that fail to parse properly result in `null`. + +### Formats + +* Any value that can be parsed by `Double.parseDouble` or `Float.parseFloat` +* `NaN`, `Infinity`, `+Infinity`, and `-Infinity` are supported + +### Known Differences + +Rcongiu maps keys to lower case when in case-insensitive mode (default), and that causes a failure when +processing the `NaN` and `Infinity` literals. + +## DECIMAL + +Values are parsed with `BigDecimal`, rounded (half up) to declared scale. Values that fail to parse properly, +or that exceeds the maximum precision, result in `null`. + +### Formats + +* Any value that can be parsed by `new BigDecimal(String)` + +### Known Differences + +* Rcongiu SerDe does not support `DECIMAL` type. +* Starburst fails if the value fails to parse. +* Starburst writes all `DECIMAL` values as `null` . + +## VARCHAR and CHAR + +All JSON values are supported by rendering canonical JSON to text. JSON will be rendered without any +additional whitespace. Unquoted values that can be parsed to a boolean or a number are canonicalized. +Unquoted field names are not canonicalized. + +### Coercions + +* **Boolean**: either `true` or `false` +* **Number**: the number parsed and formatted with BigDecimal +* **Array**: value is rendered back to JSON +* **Object**: value is rendered back to JSON + +### Known Differences + +* Rcongiu fails on unquoted boolean and numbers in field names, but all other unquoted values are allowed. +* Starburst and fails for unquoted field names. +* Rcongiu and Starburst will use `Double` in some cases for canonicalization of numbers which results in + a slightly different format from `BigDecimal`. +* Rcongiu and Starburst do not support conversions for `CHAR` or bounded `VARCHAR`. +* Rcongiu and Starburst write `CHAR` or bounded `VARCHAR` values as `null` regardless of the actual value. + +## VARBINARY + +Values are decoded with RFC 4648 Base64. Unquoted strings are not supported and result in `null`. + +### Formats + +* Base64 decode; `null` if the value is not a quoted string, or cannot be decoded. + +### Known Differences + +* Rcongiu SerDe does not support `VARBINARY` type. + +## DATE + +Values that fail to parse properly, or that are outside the allowed range, result in `null`. + +### Formats + +* `YYYY-MM-DD` with any trailing text after a space +* Decimal number of days since `1970-01-01` +* Hex `0x1234` days. Negative numbers are not supported. +* Octal `01234` days. Negative numbers are not supported. + +### Known Differences + +* Rcongiu SerDe parses dates using the broken `java.text.DateFormat` code which produces + incorrect results for dates before 1582-10-15. +* Rcongiu does not support integer days since epoch format. +* Starburst fails for boolean +* Starburst and Rcongiu writes all `DATE` values as `null` + +## TIMESTAMP + +Values that fail to parse properly, or that are outside the allowed range, result in `null`. + +### Text Formats + +Then time zone is present the value is converted to the same instant in UTC, and then that local +time is used. The parser uses lenient mode, so values too large for a field are wrapped into the +next field + +* `yyyy-MM-dd'T'HH:mm:ss.ffff'Z'` (default) +* `yyyy-MM-dd'T'HH:mm:ss.ffff+HHMM` +* `yyyy-MM-dd hh:mm:ss.ffff` + +### Numeric Formats + +Numeric formats are the number of seconds or milliseconds since 1970-01-01. Numbers are always +in base 10. Sign and leading zeros are supported. When counting digits sign and leading zeros +are included. + +* decimal number: `seconds.millis` +* 13 or more digits: `millis` +* 12 or fewer digits: `seconds` + +### Table Properties + +* `timestamp.formats`: A comma separated list of `java.time.format.DateTimeFormatter` patterns. + If the value does not match one of the patterns, the default patterns are used. + +### Known Differences + +* Rcongiu SerDe parses dates using the broken `java.text.DateFormat` code which produces + incorrect results for dates before 1582-10-15. +* Rcongiu does not support the `timestamp.formats` property. +* Starburst does not use built in formats when timestamp formats are provided. +* Starburst allows unquoted octal values in numeric formats. +* Rcongiu and Starburst both fail when patterns do not match. +* Rcongiu and Starburst both allow values with exponents in numeric formats. +* Rcongiu and Starburst uses a more strict timestamp parser mode. + +## ARRAY + +### Coercions + +* Empty string is coerced to `null`. +* Primitive or object value (without array wrapper) is coerced to a single element array. + +### Known Differences + +None + +## MAP + +Entries where the key evaluates to `null` are skipped. For maps with duplicate keys values +(after parsing), only last value parsed and used. For serialization, map keys must be a +primitive type, but for legacy reasons, there is very limited support for reading structural +map keys due to the implicit string coercions in ARRAY, MAP, and ROW types. It is not +recommended that structural keys are used as they can generally only result in empty maps, +or for ARRAY, a single element array key. + +### Coercions + +* String containing only whitespace is `null` +* All other types fail + +### Known Differences + +* Rcongiu SerDe map all keys to lower case during parsing, which affected all parsers. +* Rcongiu fails for duplicate map keys during parse, but duplicates are supported for keys + that parse to the same value (e.g., doubles with trailing zeros). + +## ROW + +Keys that do not map to a row field are ignored. For objects with duplicate keys values, +only last value parsed and used. + +### Coercions + +* String containing only whitespace is coerced to a row with all fields set to `null` +* All other types fail + +### Table Properties + +* `case.insensitive` (default true): Keys are mapped to fields using a case-insensitive name match + unless disabled. When case-insensitivity is disabled, fields containing uppercase names will + never match as Hive field names are always lowercase. +* `dots.in.keys` (default false): If key does not have an exact match, find the first key that + matches after replacing `.` with `_` in the key. This behavior is affected by the + `case.insensitivity` property, but not the `mapping` properties. +* `mapping.{field-name}=key`: Matches the field to the specified key. This behavior is affected + by the `case.insensitivity` property, but not the `dots.in.keys` property. +* `explicit.null` (default false): When enabled, a `null` field is rendered with an explicit JSON + `null`, otherwise the field is not rendered at all. + +### Coercions + +* String containing only whitespace is `null` + +### Known Differences + +* Rcongiu SerDe map all keys to lower case during parsing, which affected all parsers. +* Rcongiu fails for duplicate map keys during parse, but duplicates are supported for keys that + parse to the same value (e.g., doubles with trailing zeros). +* Rcongiu does not support `explicit.null` option. diff --git a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/FormatTestUtils.java b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/FormatTestUtils.java index 5eb725450fe1..4d7063b1a023 100644 --- a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/FormatTestUtils.java +++ b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/FormatTestUtils.java @@ -121,7 +121,6 @@ import static org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory.javaTimestampObjectInspector; import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertNull; public final class FormatTestUtils { @@ -293,6 +292,8 @@ private static Object decodeRecordReaderMap(Type type, Map map, Optional newMap.put( decodeRecordReaderValue(keyType, entryKey, hiveStorageTimeZone), decodeRecordReaderValue(valueType, entryValue, hiveStorageTimeZone))); + // Trino does not support null keys for hive, so remove them here + newMap.remove(null); return newMap; } @@ -381,10 +382,20 @@ private static Object unwrapWritable(Writable writable) throw new IllegalArgumentException("Unsupported writable type: " + writable.getClass().getSimpleName()); } + public static void assertColumnValuesEquals(List columns, List actualValues, List expectedValues) + { + for (int i = 0; i < columns.size(); i++) { + Type type = columns.get(i).type(); + Object actualValue = actualValues.get(i); + Object expectedValue = expectedValues.get(i); + assertColumnValueEquals(type, actualValue, expectedValue); + } + } + public static void assertColumnValueEquals(Type type, Object actual, Object expected) { - if (actual == null) { - assertNull(expected); + if (actual == null || expected == null) { + assertEquals(actual, expected); return; } if (type instanceof ArrayType) { diff --git a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/line/openxjson/JsonReaderTest.java b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/line/openxjson/JsonReaderTest.java new file mode 100644 index 000000000000..548994e1695c --- /dev/null +++ b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/line/openxjson/JsonReaderTest.java @@ -0,0 +1,307 @@ +/* + * 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.trino.hive.formats.line.openxjson; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.starburst.openjson.JSONArray; +import io.starburst.openjson.JSONException; +import io.starburst.openjson.JSONObject; +import io.starburst.openjson.JSONTokener; +import org.testng.annotations.Test; + +import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static java.util.Collections.singletonList; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +public class JsonReaderTest +{ + @Test + public void testJsonNull() + throws InvalidJsonException + { + assertJsonValue("null", null); + } + + @Test + public void testJsonPrimitive() + throws InvalidJsonException + { + // unquoted values + assertJsonValue("true", new JsonString("true", false)); + assertJsonValue("false", new JsonString("false", false)); + assertJsonValue("TRUE", new JsonString("TRUE", false)); + assertJsonValue("FALSE", new JsonString("FALSE", false)); + assertJsonValue("42", new JsonString("42", false)); + assertJsonValue("1.23", new JsonString("1.23", false)); + assertJsonValue("1.23e10", new JsonString("1.23e10", false)); + assertJsonValue("1.23E10", new JsonString("1.23E10", false)); + assertJsonValue("Infinity", new JsonString("Infinity", false)); + assertJsonValue("NaN", new JsonString("NaN", false)); + assertJsonValue("abc", new JsonString("abc", false)); + + // anything is allowed after the value ends, which requires a separator + assertJsonValue("true;anything", new JsonString("true", false)); + assertJsonValue("false anything", new JsonString("false", false)); + + // Quoted string values + assertJsonValue("\"\"", new JsonString("", true)); + assertJsonValue("\"abc\"", new JsonString("abc", true)); + + // escapes + assertJsonValue("\" \\\\ \\t \\b \\n \\r \\f \\a \\v \\u1234 \\uFFFD \\ufffd \"", + new JsonString(" \\ \t \b \n \r \f \007 \011 \u1234 \uFFFD \ufffd ", true)); + + // any other character is just passed through + assertJsonValue("\"\\X\"", new JsonString("X", true)); + assertJsonValue("\"\\\"\"", new JsonString("\"", true)); + assertJsonValue("\"\\'\"", new JsonString("'", true)); + + // unterminated escapes are an error + assertJsonFails("\"\\\""); + assertJsonFails("\"\\u1\""); + assertJsonFails("\"\\u12\""); + assertJsonFails("\"\\u123\""); + + // unicode escape requires hex + assertJsonFails("\"\\u123X\""); + + // unterminated string is an error + assertJsonFails("\"abc"); + assertJsonFails("\"a\\tc"); + + // anything is allowed after the value + assertJsonValue("\"abc\"anything", new JsonString("abc", true)); + } + + @Test + public void testJsonObject() + throws InvalidJsonException + { + assertJsonValue("{}", ImmutableMap.of()); + assertJsonValue("{ }", ImmutableMap.of()); + assertJsonFails("{"); + assertJsonFails("{{"); + + // anything is allowed after the object + assertJsonValue("{}anything allowed", ImmutableMap.of()); + + assertJsonValue("{ \"a\" : 2.34 }", ImmutableMap.of("a", new JsonString("2.34", false))); + assertJsonValue("{ \"a\" = 2.34 }", ImmutableMap.of("a", new JsonString("2.34", false))); + assertJsonValue("{ \"a\" => 2.34 }", ImmutableMap.of("a", new JsonString("2.34", false))); + assertJsonValue("{ a : 2.34 }", ImmutableMap.of("a", new JsonString("2.34", false))); + assertJsonValue("{ a = 2.34 }", ImmutableMap.of("a", new JsonString("2.34", false))); + assertJsonValue("{ a => 2.34 }", ImmutableMap.of("a", new JsonString("2.34", false))); + assertJsonFails("{ \"a\""); + assertJsonFails("{ a"); + assertJsonFails("{ \"a\","); + assertJsonFails("{ \"a\";"); + assertJsonFails("{ \"a\",2.34"); + assertJsonFails("{ \"a\";2.34"); + assertJsonFails("{ a x 2.34 }"); + assertJsonFails("{ a ~ 2.34 }"); + assertJsonFails("{ a -> 2.34 }"); + // starburst allows for :> due to a bug, but the original code did not support this + assertJsonTrinoFails("{ a :> 2.34 }"); + assertJsonHive("{ a :> 2.34 }", ImmutableMap.of("a", new JsonString("2.34", false))); + + assertJsonValue("{ a : 2.34 , b : false}", ImmutableMap.of("a", new JsonString("2.34", false), "b", new JsonString("false", false))); + assertJsonValue("{ a : 2.34 ; b : false}", ImmutableMap.of("a", new JsonString("2.34", false), "b", new JsonString("false", false))); + assertJsonFails("{ a : 2.34 x b : false}"); + assertJsonFails("{ a : 2.34 ^ b : false}"); + assertJsonFails("{ a : 2.34 : b : false}"); + + assertJsonValue("{ a : NaN }", ImmutableMap.of("a", new JsonString("NaN", false))); + assertJsonValue("{ a : \"NaN\" }", ImmutableMap.of("a", new JsonString("NaN", true))); + + // Starburst hive does not allow unquoted field names, but Trino and the original code does + assertJsonTrinoOnly("{ true : NaN }", ImmutableMap.of("true", new JsonString("NaN", false))); + assertJsonTrinoOnly("{ 123 : NaN }", ImmutableMap.of("123", new JsonString("NaN", false))); + + // field name can not be null + assertJsonFails("{ null : \"NaN\" }"); + // field name can not be structural + assertJsonFails("{ [] : \"NaN\" }"); + assertJsonFails("{ {} : \"NaN\" }"); + + // map can contain c-style comments + assertJsonValue("/*foo*/{/*foo*/\"a\"/*foo*/:/*foo*/2.34/*foo*/}/*foo*/", ImmutableMap.of("a", new JsonString("2.34", false))); + // unterminated comment is an error + assertJsonFails("/*foo*/{/*foo*/\"a\"/*foo*/:/*foo"); + // end of line comments are always an error since the value is unterminated + assertJsonFails("/*foo*/{/*foo*/\"a\"/*foo*/:#end-of-line"); + assertJsonFails("/*foo*/{/*foo*/\"a\"/*foo*/://end-of-line"); + } + + @Test + public void testJsonArray() + throws InvalidJsonException + { + assertJsonValue("[]", ImmutableList.of()); + assertJsonValue("[,]", Arrays.asList(null, null)); + assertJsonFails("["); + assertJsonFails("[42"); + assertJsonFails("[42,"); + + // anything is allowed after the array + assertJsonValue("[]anything allowed", ImmutableList.of()); + + assertJsonValue("[ 2.34 ]", singletonList(new JsonString("2.34", false))); + assertJsonValue("[ NaN ]", singletonList(new JsonString("NaN", false))); + assertJsonValue("[ \"NaN\" ]", singletonList(new JsonString("NaN", true))); + + assertJsonValue("[ 2.34 , ]", Arrays.asList(new JsonString("2.34", false), null)); + assertJsonValue("[ NaN , ]", Arrays.asList(new JsonString("NaN", false), null)); + assertJsonValue("[ \"NaN\" , ]", Arrays.asList(new JsonString("NaN", true), null)); + + // map can contain c-style comments + assertJsonValue("/*foo*/[/*foo*/\"a\"/*foo*/,/*foo*/2.34/*foo*/]/*foo*/", ImmutableList.of(new JsonString("a", true), new JsonString("2.34", false))); + // unterminated comment is an error + assertJsonFails("/*foo*/[/*foo*/\"a\"/*foo*/,/*foo"); + // end of line comments are always an error since the value is unterminated + assertJsonFails("/*foo*/[/*foo*/\"a\"/*foo*/,#end-of-line"); + assertJsonFails("/*foo*/[/*foo*/\"a\"/*foo*/,//end-of-line"); + } + + private static void assertJsonValue(String json, Object expectedTrinoValue) + throws InvalidJsonException + { + assertJsonTrino(json, expectedTrinoValue); + assertJsonHive(json, expectedTrinoValue); + } + + private static void assertJsonTrinoOnly(String json, Object expected) + throws InvalidJsonException + { + assertJsonTrino(json, expected); + assertJsonHiveFails(json); + } + + private static void assertJsonTrino(String json, Object expected) + throws InvalidJsonException + { + assertThat(JsonReader.readJson(json, Function.identity())) + .isEqualTo(expected); + } + + private static void assertJsonHive(String json, Object expectedTrinoValue) + { + Object actualHiveValue = unwrapHiveValue(new JSONTokener(false, json).nextValue()); + Object expectedHiveValue = toHiveEquivalent(expectedTrinoValue); + assertThat(actualHiveValue).isEqualTo(expectedHiveValue); + } + + private static void assertJsonFails(String json) + { + assertJsonTrinoFails(json); + assertJsonHiveFails(json); + } + + private static void assertJsonTrinoFails(String json) + { + assertThatThrownBy(() -> JsonReader.readJson(json, Function.identity())) + .isInstanceOf(InvalidJsonException.class); + } + + private static void assertJsonHiveFails(String json) + { + assertThatThrownBy(() -> new JSONTokener(false, json).nextValue()) + .isInstanceOf(JSONException.class); + } + + private static Object unwrapHiveValue(Object value) + { + if (value instanceof JSONObject jsonObject) { + LinkedHashMap unwrapped = new LinkedHashMap<>(); + for (String key : jsonObject.keySet()) { + unwrapped.put(key, jsonObject.opt(key)); + } + return unwrapped; + } + if (value instanceof JSONArray jsonArray) { + List unwrapped = new ArrayList<>(); + for (int i = 0; i < jsonArray.length(); i++) { + unwrapped.add(jsonArray.opt(i)); + } + return unwrapped; + } + if (value == JSONObject.NULL) { + return null; + } + return value; + } + + private static Object toHiveEquivalent(Object value) + { + if (value == null) { + return null; + } + if (value instanceof Map map) { + return map.entrySet().stream() + .collect(Collectors.toUnmodifiableMap(Entry::getKey, entry -> toHiveEquivalent(entry.getValue()))); + } + if (value instanceof List list) { + return list.stream() + .map(JsonReaderTest::toHiveEquivalent) + .collect(Collectors.toCollection(ArrayList::new)); + } + if (value instanceof JsonString jsonString) { + if (jsonString.quoted()) { + return jsonString.value(); + } + + String string = jsonString.value(); + + if (string.equalsIgnoreCase("true")) { + return true; + } + if (string.equalsIgnoreCase("false")) { + return false; + } + + try { + long longValue = Long.parseLong(string); + if (longValue <= Integer.MAX_VALUE && longValue >= Integer.MIN_VALUE) { + return (int) longValue; + } + else { + return longValue; + } + } + catch (NumberFormatException ignored) { + } + + try { + BigDecimal asDecimal = new BigDecimal(string); + double asDouble = Double.parseDouble(string); + return asDecimal.compareTo(BigDecimal.valueOf(asDouble)) == 0 ? asDouble : asDecimal; + } + catch (NumberFormatException ignored) { + } + + return string; + } + throw new IllegalArgumentException("Unsupported type: " + value.getClass().getSimpleName()); + } +} diff --git a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/line/openxjson/TestOpenxJsonFormat.java b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/line/openxjson/TestOpenxJsonFormat.java new file mode 100644 index 000000000000..200a67883f99 --- /dev/null +++ b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/line/openxjson/TestOpenxJsonFormat.java @@ -0,0 +1,1612 @@ +/* + * 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.trino.hive.formats.line.openxjson; + +import com.google.common.base.CharMatcher; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.primitives.Ints; +import com.google.common.primitives.Shorts; +import com.google.common.primitives.SignedBytes; +import io.airlift.slice.DynamicSliceOutput; +import io.airlift.slice.SliceOutput; +import io.trino.hive.formats.FormatTestUtils; +import io.trino.hive.formats.line.Column; +import io.trino.hive.formats.line.LineDeserializer; +import io.trino.hive.formats.line.LineSerializer; +import io.trino.spi.Page; +import io.trino.spi.PageBuilder; +import io.trino.spi.type.ArrayType; +import io.trino.spi.type.CharType; +import io.trino.spi.type.DecimalType; +import io.trino.spi.type.Decimals; +import io.trino.spi.type.MapType; +import io.trino.spi.type.RowType; +import io.trino.spi.type.SqlDate; +import io.trino.spi.type.SqlDecimal; +import io.trino.spi.type.SqlTimestamp; +import io.trino.spi.type.SqlVarbinary; +import io.trino.spi.type.TimestampType; +import io.trino.spi.type.Type; +import io.trino.spi.type.TypeOperators; +import io.trino.spi.type.VarcharType; +import org.apache.hadoop.hive.serde2.Deserializer; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.SettableStructObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.JobConf; +import org.openx.data.jsonserde.JsonSerDe; +import org.testng.annotations.Test; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Base64; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.function.LongFunction; +import java.util.logging.Level; +import java.util.logging.Logger; +import java.util.stream.Collectors; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.hadoop.ConfigurationInstantiator.newEmptyConfiguration; +import static io.trino.hive.formats.FormatTestUtils.assertColumnValueEquals; +import static io.trino.hive.formats.FormatTestUtils.assertColumnValuesEquals; +import static io.trino.hive.formats.FormatTestUtils.createLineBuffer; +import static io.trino.hive.formats.FormatTestUtils.decodeRecordReaderValue; +import static io.trino.hive.formats.FormatTestUtils.isScalarType; +import static io.trino.hive.formats.FormatTestUtils.readTrinoValues; +import static io.trino.hive.formats.FormatTestUtils.toHiveWriteValue; +import static io.trino.hive.formats.FormatTestUtils.toSingleRowPage; +import static io.trino.hive.formats.FormatTestUtils.toSqlTimestamp; +import static io.trino.hive.formats.line.openxjson.OpenXJsonOptions.DEFAULT_OPEN_X_JSON_OPTIONS; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.CharType.createCharType; +import static io.trino.spi.type.Chars.padSpaces; +import static io.trino.spi.type.DateType.DATE; +import static io.trino.spi.type.DecimalType.createDecimalType; +import static io.trino.spi.type.Decimals.MAX_PRECISION; +import static io.trino.spi.type.Decimals.MAX_SHORT_PRECISION; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.spi.type.IntegerType.INTEGER; +import static io.trino.spi.type.RealType.REAL; +import static io.trino.spi.type.RowType.field; +import static io.trino.spi.type.RowType.rowType; +import static io.trino.spi.type.SmallintType.SMALLINT; +import static io.trino.spi.type.TimestampType.TIMESTAMP_MICROS; +import static io.trino.spi.type.TimestampType.TIMESTAMP_MILLIS; +import static io.trino.spi.type.TimestampType.TIMESTAMP_NANOS; +import static io.trino.spi.type.TimestampType.TIMESTAMP_SECONDS; +import static io.trino.spi.type.TinyintType.TINYINT; +import static io.trino.spi.type.VarbinaryType.VARBINARY; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static io.trino.spi.type.VarcharType.createVarcharType; +import static java.lang.Math.toIntExact; +import static java.time.ZoneOffset.UTC; +import static java.util.Collections.emptyMap; +import static java.util.Collections.singletonList; +import static java.util.Collections.singletonMap; +import static java.util.stream.Collectors.joining; +import static org.apache.hadoop.hive.serde.serdeConstants.LIST_COLUMNS; +import static org.apache.hadoop.hive.serde.serdeConstants.LIST_COLUMN_TYPES; +import static org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_LIB; +import static org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.getStandardStructObjectInspector; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.testng.Assert.assertTrue; + +public class TestOpenxJsonFormat +{ + static { + // Increase the level of the JsonSerDe logger as it is excessively logs + Logger.getLogger(JsonSerDe.class.getName()).setLevel(Level.SEVERE); + } + + private static final TypeOperators TYPE_OPERATORS = new TypeOperators(); + + private static final DecimalType SHORT_DECIMAL = createDecimalType(MAX_SHORT_PRECISION, 2); + private static final DecimalType LONG_DECIMAL = createDecimalType(MAX_PRECISION, 2); + + @Test + public void testLine() + { + List columns = ImmutableList.of( + new Column("a", DOUBLE, 0), + new Column("b", VARCHAR, 1)); + + // if line does not start with '{' or '[', after trim, all values are null + assertLine(columns, "", Arrays.asList(null, null), DEFAULT_OPEN_X_JSON_OPTIONS); + assertLine(columns, "null", Arrays.asList(null, null), DEFAULT_OPEN_X_JSON_OPTIONS); + assertLine(columns, "123", Arrays.asList(null, null), DEFAULT_OPEN_X_JSON_OPTIONS); + assertLine(columns, "#", Arrays.asList(null, null), DEFAULT_OPEN_X_JSON_OPTIONS); + + // leading and trailing whitespace is ignored + assertLine(columns, " {\"a\":1.23} ", Arrays.asList(1.23, null), DEFAULT_OPEN_X_JSON_OPTIONS); + assertLine(columns, " [1.23] ", Arrays.asList(1.23, null), DEFAULT_OPEN_X_JSON_OPTIONS); + + // trailing junk is ignored + assertLine(columns, " {\"a\":1.23}anything here", Arrays.asList(1.23, null), DEFAULT_OPEN_X_JSON_OPTIONS); + assertLine(columns, " [1.23]anything here", Arrays.asList(1.23, null), DEFAULT_OPEN_X_JSON_OPTIONS); + + // malformed lines resul in a failure, unless ignore malformed is enabled + OpenXJsonOptions ignoreMalformed = OpenXJsonOptions.builder().ignoreMalformedJson().build(); + assertLineFails(columns, "[", DEFAULT_OPEN_X_JSON_OPTIONS); + assertLine(columns, "[", Arrays.asList(null, null, null), ignoreMalformed); + assertLineFails(columns, "{", DEFAULT_OPEN_X_JSON_OPTIONS); + assertLine(columns, "{", Arrays.asList(null, null, null), ignoreMalformed); + } + + @Test + public void testExplicitNulls() + { + OpenXJsonOptions explicitNulls = OpenXJsonOptions.builder().explicitNull().build(); + List columns = ImmutableList.of( + new Column("a", DOUBLE, 0), + new Column("b", rowType(field("x", BOOLEAN), field("y", VARCHAR)), 1), + new Column("c", new MapType(BIGINT, DOUBLE, TYPE_OPERATORS), 1)); + // all columns nulls + assertExactLine(columns, Arrays.asList(null, null, null), "{}", DEFAULT_OPEN_X_JSON_OPTIONS); + assertExactLine(columns, Arrays.asList(null, null, null), "{\"a\":null,\"b\":null,\"c\":null}", explicitNulls); + // single primitive non-null column + assertExactLine(columns, Arrays.asList(4.56, null, null), "{\"a\":4.56}", DEFAULT_OPEN_X_JSON_OPTIONS); + assertExactLine(columns, Arrays.asList(4.56, null, null), "{\"a\":4.56,\"b\":null,\"c\":null}", explicitNulls); + // empty row + assertExactLine(columns, Arrays.asList(null, Arrays.asList(null, null), null), "{\"b\":{}}", DEFAULT_OPEN_X_JSON_OPTIONS); + assertExactLine(columns, Arrays.asList(null, Arrays.asList(null, null), null), "{\"a\":null,\"b\":{\"x\":null,\"y\":null},\"c\":null}", explicitNulls); + // single value in row + assertExactLine(columns, Arrays.asList(null, Arrays.asList(true, null), null), "{\"b\":{\"x\":true}}", DEFAULT_OPEN_X_JSON_OPTIONS); + assertExactLine(columns, Arrays.asList(null, Arrays.asList(true, null), null), "{\"a\":null,\"b\":{\"x\":true,\"y\":null},\"c\":null}", explicitNulls); + // empty map + assertExactLine(columns, Arrays.asList(null, null, emptyMap()), "{\"c\":{}}", DEFAULT_OPEN_X_JSON_OPTIONS); + assertExactLine(columns, Arrays.asList(null, null, emptyMap()), "{\"a\":null,\"b\":null,\"c\":{}}", explicitNulls); + // map with null value (Starburst does not write null values) + assertThat(writeTrinoLine(columns, Arrays.asList(null, null, singletonMap(42L, null)), DEFAULT_OPEN_X_JSON_OPTIONS)) + .isEqualTo("{\"c\":{\"42\":null}}"); + assertThat(writeTrinoLine(columns, Arrays.asList(null, null, singletonMap(42L, null)), explicitNulls)) + .isEqualTo("{\"a\":null,\"b\":null,\"c\":{\"42\":null}}"); + } + + @Test + public void testCaseInsensitive() + { + OpenXJsonOptions caseSensitive = OpenXJsonOptions.builder().caseSensitive().build(); + // this is only valid if we do not lower keys like the starburst version does + assertValue(VARCHAR, "{\"FOO\":42}", "{\"FOO\":42}", DEFAULT_OPEN_X_JSON_OPTIONS, false); + assertValue(VARCHAR, "{\"FOO\":42}", "{\"FOO\":42}", caseSensitive, false); + // case-insensitive eliminates duplicates in maps keys that only differ in case + assertValue(VARCHAR, "{\"FOO\":42,\"FoO\":42}", "{\"FoO\":42}", DEFAULT_OPEN_X_JSON_OPTIONS, false); + assertValue(VARCHAR, "{\"FOO\":42,\"FoO\":42}", "{\"FOO\":42,\"FoO\":42}", caseSensitive, false); + + // top level column + assertLine(ImmutableList.of(new Column("foo", BOOLEAN, 0)), "{\"FoO\":true}", singletonList(true), DEFAULT_OPEN_X_JSON_OPTIONS); + assertLine(ImmutableList.of(new Column("foo", BOOLEAN, 0)), "{\"FoO\":true}", singletonList(null), caseSensitive); + + // row value + assertValue(rowType(field("a", rowType(field("foo", BOOLEAN)))), "{\"a\":{\"FoO\":true}}", singletonList(singletonList(true)), DEFAULT_OPEN_X_JSON_OPTIONS, false); + assertValue(rowType(field("a", rowType(field("foo", BOOLEAN)))), "{\"a\":{\"FoO\":true}}", singletonList(singletonList(null)), caseSensitive, false); + + // multiple levels + assertValue(rowType(field("a", rowType(field("foo", BOOLEAN)))), "{\"A\":{\"FoO\":true}}", singletonList(singletonList(true)), DEFAULT_OPEN_X_JSON_OPTIONS, false); + assertValue(rowType(field("a", rowType(field("foo", BOOLEAN)))), "{\"A\":{\"FoO\":true}}", singletonList(null), caseSensitive, false); + } + + @Test + public void testDotsInFieldNames() + { + OpenXJsonOptions dotsInFieldNames = OpenXJsonOptions.builder().dotsInFieldNames().build(); + + // top level column + assertLine(ImmutableList.of(new Column("a_b", BIGINT, 0)), "{\"a.b\":42}", singletonList(null), DEFAULT_OPEN_X_JSON_OPTIONS); + assertLine(ImmutableList.of(new Column("a_b", BIGINT, 0)), "{\"a.b\":42}", singletonList(42L), dotsInFieldNames); + // todo Starburst is always case-sensitive for dotted names + internalAssertLineHive(ImmutableList.of(new Column("a_b", BIGINT, 0)), "{\"a.B\":42}", singletonList(null), dotsInFieldNames); + internalAssertLineTrino(ImmutableList.of(new Column("a_b", BIGINT, 0)), "{\"a.B\":42}", singletonList(42L), dotsInFieldNames); + + // row value + assertValue(rowType(field("a", rowType(field("x_y", BIGINT)))), "{\"a\":{\"x.y\":42}}", singletonList(singletonList(null)), DEFAULT_OPEN_X_JSON_OPTIONS, false); + assertValue(rowType(field("a", rowType(field("x_y", BIGINT)))), "{\"a\":{\"x.y\":42}}", singletonList(singletonList(42L)), dotsInFieldNames, false); + // todo Starburst is always case-sensitive for dotted names + internalAssertValueHive(rowType(field("a", rowType(field("x_y", BIGINT)))), "{\"a\":{\"X.y\":42}}", singletonList(singletonList(null)), dotsInFieldNames); + internalAssertValueTrino(rowType(field("a", rowType(field("x_y", BIGINT)))), "{\"a\":{\"X.y\":42}}", singletonList(singletonList(42L)), dotsInFieldNames); + + // multiple levels + assertValue(rowType(field("a_b", rowType(field("x_y", BIGINT)))), "{\"a.b\":{\"x.y\":42}}", singletonList(null), DEFAULT_OPEN_X_JSON_OPTIONS, false); + assertValue(rowType(field("a_b", rowType(field("x_y", BIGINT)))), "{\"a.b\":{\"x.y\":42}}", singletonList(singletonList(42L)), dotsInFieldNames, false); + + // field mappings are not considered for dotted names + OpenXJsonOptions mappedFieldNames = OpenXJsonOptions.builder(dotsInFieldNames) + .addFieldMapping("apple", "a_b") + .build(); + assertLine(ImmutableList.of(new Column("apple", BIGINT, 0)), "{\"a.b\":42}", singletonList(null), mappedFieldNames); + assertLine(ImmutableList.of(new Column("a_b", BIGINT, 0)), "{\"a.b\":42}", singletonList(42L), mappedFieldNames); + } + + @Test + public void testMappedFieldNames() + { + OpenXJsonOptions mappedFieldNames = OpenXJsonOptions.builder() + .addFieldMapping("apple", "a") + .addFieldMapping("banana", "b") + .addFieldMapping("cherry", "c") + .build(); + OpenXJsonOptions caseSensitiveMapping = OpenXJsonOptions.builder(mappedFieldNames) + .caseSensitive() + .build(); + + // top level column + assertLine(ImmutableList.of(new Column("apple", BIGINT, 0)), "{\"a\":42}", singletonList(null), DEFAULT_OPEN_X_JSON_OPTIONS); + assertLine(ImmutableList.of(new Column("apple", BIGINT, 0)), "{\"a\":42}", singletonList(42L), mappedFieldNames); + // mappings follow case sensitivity + internalAssertLineTrino(ImmutableList.of(new Column("apple", BIGINT, 0)), "{\"A\":42}", singletonList(42L), mappedFieldNames); + internalAssertLineTrino(ImmutableList.of(new Column("apple", BIGINT, 0)), "{\"A\":42}", singletonList(null), caseSensitiveMapping); + // declared mappings are case-sensitive + assertLine(ImmutableList.of(new Column("Apple", BIGINT, 0)), "{\"a\":42}", singletonList(42L), mappedFieldNames); + + // row value + assertValue(rowType(field("x", rowType(field("banana", BIGINT)))), "{\"x\":{\"b\":42}}", singletonList(singletonList(null)), DEFAULT_OPEN_X_JSON_OPTIONS, false); + assertValue(rowType(field("x", rowType(field("banana", BIGINT)))), "{\"x\":{\"b\":42}}", singletonList(singletonList(42L)), mappedFieldNames, false); + internalAssertValueTrino(rowType(field("x", rowType(field("banana", BIGINT)))), "{\"x\":{\"B\":42}}", singletonList(singletonList(42L)), mappedFieldNames); + internalAssertValueTrino(rowType(field("x", rowType(field("Banana", BIGINT)))), "{\"x\":{\"B\":42}}", singletonList(singletonList(null)), caseSensitiveMapping); + + // multiple levels + assertValue(rowType(field("apple", rowType(field("banana", BIGINT)))), "{\"a\":{\"b\":42}}", singletonList(null), DEFAULT_OPEN_X_JSON_OPTIONS, false); + assertValue(rowType(field("apple", rowType(field("banana", BIGINT)))), "{\"a\":{\"b\":42}}", singletonList(singletonList(42L)), mappedFieldNames, false); + assertValue(rowType(field("apple", rowType(field("banana", BIGINT)))), "{\"a\":{\"B\":42}}", singletonList(singletonList(42L)), mappedFieldNames, false); + assertValue(rowType(field("apple", rowType(field("banana", BIGINT)))), "{\"a\":{\"B\":42}}", singletonList(singletonList(null)), caseSensitiveMapping, false); + } + + @Test + public void testRow() + { + RowType rowType = rowType(field("a", BIGINT), field("b", BIGINT), field("c", BIGINT)); + + assertValue(rowType, "null", null); + + // string containing only whitespace is null + assertValue(rowType, "\"\"", Arrays.asList(null, null, null)); + assertValue(rowType, "\" \"", Arrays.asList(null, null, null)); + assertValue(rowType, "\" \t\t \"", Arrays.asList(null, null, null)); + + assertValue(rowType, "{ \"a\" : 1, \"b\" : 2, \"c\" : 3 }", ImmutableList.of(1L, 2L, 3L)); + assertValue(rowType, "{ \"c\" : 3, \"a\" : 1, \"b\" : 2 }", ImmutableList.of(1L, 2L, 3L)); + assertValue(rowType, "{ \"x\" : 3, \"c\" : 3, \"a\" : 1, \"b\" : 2 , \"y\" : 2 }", ImmutableList.of(1L, 2L, 3L)); + assertValue(rowType, "{}", Arrays.asList(null, null, null)); + assertValue(rowType, "{ \"b\" : 2 }", Arrays.asList(null, 2L, null)); + + // Duplicate fields are supported, and the last value is used + assertValue(rowType, "{ \"a\" : 1, \"a\" : 2 }", Arrays.asList(2L, null, null)); + // and we only parse the last field + assertValue(rowType, "{ \"a\" : true, \"a\" : 42 }", Arrays.asList(42L, null, null)); + + // OpenX JsonSerDe supports arrays using column ordinal position + assertValue(rowType, "[ 1, 2, 3 ]", Arrays.asList(1L, 2L, 3L)); + assertValue(rowType, "[ 1, , 3 ]", Arrays.asList(1L, null, 3L)); + assertValue(rowType, "[ 1, , 3, ]", Arrays.asList(1L, null, 3L)); + + assertValueFails(rowType, "true"); + assertValueFails(rowType, "12"); + assertValueFails(rowType, "12.34"); + assertValueFails(rowType, "\"string\""); + } + + @Test + public void testMap() + { + MapType mapType = new MapType(VARCHAR, BIGINT, TYPE_OPERATORS); + assertValue(mapType, "null", null); + + // string containing only whitespace is null + assertValue(mapType, "\"\"", null); + assertValue(mapType, "\" \"", null); + assertValue(mapType, "\" \t\t \"", null); + + assertValue(mapType, "{}", ImmutableMap.of()); + assertValue( + mapType, + "{ \"a\" : 1, \"b\" : 2, \"c\" : 3 }", + ImmutableMap.builder() + .put("a", 1L) + .put("b", 2L) + .put("c", 3L) + .buildOrThrow()); + assertValue( + mapType, + "{ \"c\" : 3, \"a\" : 1, \"b\" : 2 }", + ImmutableMap.builder() + .put("a", 1L) + .put("b", 2L) + .put("c", 3L) + .buildOrThrow()); + + // Duplicate fields are supported, and the last value is used + assertValue( + mapType, + "{ \"a\" : 1, \"b\" : 2 , \"a\" : 3 , \"b\" : 4 }", + ImmutableMap.builder() + .put("a", 3L) + .put("b", 4L) + .buildOrThrow()); + + assertValueFails(mapType, "true"); + assertValueFails(mapType, "12"); + assertValueFails(mapType, "12.34"); + assertValueFails(mapType, "\"string\""); + assertValueFails(mapType, "[ 42 ]"); + } + + @Test + public void testMapWithContainerKey() + { + // Generally containers can not be used for key, because there are limited + // coercions from primitive values to container types. + + MapType arrayKey = new MapType(new ArrayType(VARCHAR), BIGINT, TYPE_OPERATORS); + assertValue(arrayKey, "null", null); + assertValue(arrayKey, "{}", ImmutableMap.of()); + // empty string is coerced to null + assertValue(arrayKey, "{\"\":42}", ImmutableMap.of(), false); + // single values are automatically wrapped in a single element array + assertValue(arrayKey, "{\"a\":42}", ImmutableMap.of(List.of("a"), 42L), false); + assertValue(new MapType(new ArrayType(BOOLEAN), BIGINT, TYPE_OPERATORS), "{\"true\":42}", ImmutableMap.of(List.of(true), 42L), false); + + MapType mapKey = new MapType(new MapType(VARCHAR, VARCHAR, TYPE_OPERATORS), BIGINT, TYPE_OPERATORS); + assertValue(mapKey, "null", null); + assertValue(mapKey, "{}", ImmutableMap.of()); + // strings containing only white space are coerced to null + assertValue(mapKey, "{\"\":42}", ImmutableMap.of(), false); + assertValue(mapKey, "{\" \t\t \":42}", ImmutableMap.of(), false); + + MapType rowKey = new MapType(rowType(field("a", BIGINT), field("b", BIGINT)), BIGINT, TYPE_OPERATORS); + assertValue(rowKey, "null", null); + assertValue(rowKey, "{}", ImmutableMap.of()); + // strings containing only white space are coerced to null + assertValue(rowKey, "{\"\":42}", ImmutableMap.of(Arrays.asList(null, null), 42L), false); + assertValue(rowKey, "{\" \t\t \":42}", ImmutableMap.of(Arrays.asList(null, null), 42L), false); + } + + @Test + public void testVarchar() + { + testString(VARCHAR, createVarcharType(3)); + testString(createVarcharType(100), createVarcharType(3)); + } + + @Test + public void testChar() + { + testString(createCharType(100), createCharType(3)); + } + + private static void testString(Type unboundedType, Type boundedType) + { + assertValue(unboundedType, "null", null); + + assertString(unboundedType, ""); + assertString(unboundedType, " "); + + assertString(unboundedType, "value"); + assertString(unboundedType, " value"); + assertString(unboundedType, "value "); + assertString(unboundedType, " value "); + + // Truncation + assertString(boundedType, "v"); + assertString(boundedType, "val"); + assertString(boundedType, "value", "val"); + + // Escape + assertString(unboundedType, "tab \\t tab", "tab \t tab"); + assertString(unboundedType, "new \\n line", "new \n line"); + assertString(unboundedType, "carriage \\r return", "carriage \r return"); + + assertVarcharCanonicalization(unboundedType, "true", "true", "true"); + assertVarcharCanonicalization(unboundedType, "false", "false", "false"); + assertVarcharCanonicalization(unboundedType, "tRUe", "true", "true"); + assertVarcharCanonicalization(unboundedType, "fAlSe", "false", "false"); + + assertVarcharCanonicalization(unboundedType, "-1", "-1", "-1"); + assertVarcharCanonicalization(unboundedType, "1.23", "1.23", "1.23"); + assertVarcharCanonicalization(unboundedType, "1.23e45", "1.23E+45", "1.23E45"); + assertVarcharCanonicalization(unboundedType, "33.23e45", "3.323E+46", "3.323E46"); + assertVarcharCanonicalization(unboundedType, "1.23E45", "1.23E+45", "1.23E45"); + assertVarcharCanonicalization(unboundedType, "33.23E45", "3.323E+46", "3.323E46"); + assertString(unboundedType, "NaN", "NaN"); + assertString(unboundedType, "Infinity", "Infinity"); + assertString(unboundedType, "+Infinity", "+Infinity"); + assertString(unboundedType, "-Infinity", "-Infinity"); + + assertStringObjectOrArrayCoercion(unboundedType, "[ \"value\" ]", "[\"value\"]"); + assertStringObjectOrArrayCoercion(unboundedType, "[ \"foo\"; TrUe]", "[\"foo\",true]"); + + assertStringObjectOrArrayCoercion(unboundedType, "{ \"x\" = \"value\" }", "{\"x\":\"value\"}"); + assertStringObjectOrArrayCoercion(unboundedType, "{ \"foo\" => TrUe }", "{\"foo\":true}"); + } + + private static void assertString(Type type, String jsonValue) + { + assertString(type, jsonValue, jsonValue); + } + + private static void assertString(Type type, String jsonValue, String expectedValue) + { + if (type instanceof CharType charType) { + expectedValue = padSpaces(expectedValue, charType); + } + + if (!jsonValue.startsWith("[") && !jsonValue.startsWith("{")) { + assertValue(type, "\"" + jsonValue + "\"", expectedValue); + if (!jsonValue.isEmpty() && CharMatcher.whitespace().matchesNoneOf(jsonValue)) { + assertValue(type, jsonValue, expectedValue); + } + } + else { + assertValue(type, jsonValue, expectedValue, false); + } + } + + private static void assertStringObjectOrArrayCoercion(Type type, String jsonValue, String expectedValue) + { + if (type instanceof CharType charType) { + expectedValue = padSpaces(expectedValue, charType); + } + + if (type == VARCHAR) { + assertValue(type, jsonValue, expectedValue, false); + } + else { + // Starburst code can not handle JSON array and map coercions for bounded VARCHAR or CHAR types + assertValueTrino(type, jsonValue, expectedValue, DEFAULT_OPEN_X_JSON_OPTIONS, false); + assertValueFailsHive(type, jsonValue, DEFAULT_OPEN_X_JSON_OPTIONS, false); + } + } + + private static void assertVarcharCanonicalization(Type type, String jsonValue, String trinoCanonicalValue, String hiveCanonicalValue) + { + String nonCanonicalValue = jsonValue; + if (type instanceof CharType charType) { + nonCanonicalValue = padSpaces(nonCanonicalValue, charType); + trinoCanonicalValue = padSpaces(trinoCanonicalValue, charType); + hiveCanonicalValue = padSpaces(hiveCanonicalValue, charType); + } + + assertTrue(CharMatcher.whitespace().matchesNoneOf(jsonValue)); + + // quoted values are not canonicalized + assertValue(type, "\"" + jsonValue + "\"", nonCanonicalValue); + + // field names are not canonicalized + internalAssertValueTrino(new MapType(type, BIGINT, TYPE_OPERATORS), "{" + jsonValue + ":43}", singletonMap(nonCanonicalValue, 43L), DEFAULT_OPEN_X_JSON_OPTIONS); + // starburst version does not allow for non-canonical field names, but original version does + assertValueFailsHive(new MapType(type, BIGINT, TYPE_OPERATORS), "{" + jsonValue + ":43}", DEFAULT_OPEN_X_JSON_OPTIONS, false); + + // unquoted values are canonicalized (using slightly different rules) + internalAssertValueTrino(type, jsonValue, trinoCanonicalValue, DEFAULT_OPEN_X_JSON_OPTIONS); + internalAssertValueTrino(new ArrayType(type), "[" + jsonValue + "]", singletonList(trinoCanonicalValue), DEFAULT_OPEN_X_JSON_OPTIONS); + internalAssertValueTrino(new MapType(BIGINT, type, TYPE_OPERATORS), "{43:" + jsonValue + "}", singletonMap(43L, trinoCanonicalValue), DEFAULT_OPEN_X_JSON_OPTIONS); + + if (type instanceof VarcharType varcharType && varcharType.isUnbounded()) { + internalAssertValueHive(type, jsonValue, hiveCanonicalValue, DEFAULT_OPEN_X_JSON_OPTIONS); + internalAssertValueHive(new ArrayType(type), "[" + jsonValue + "]", singletonList(hiveCanonicalValue), DEFAULT_OPEN_X_JSON_OPTIONS); + internalAssertValueHive(new MapType(BIGINT, type, TYPE_OPERATORS), "{\"43\":" + jsonValue + "}", singletonMap(43L, hiveCanonicalValue), DEFAULT_OPEN_X_JSON_OPTIONS); + } + else { + // varchar and char type in Starburst doesn't support canonicalization (the code above uses Hive string type) + assertValueFailsHive(type, jsonValue, DEFAULT_OPEN_X_JSON_OPTIONS, false); + assertValueFailsHive(new ArrayType(type), "[" + jsonValue + "]", DEFAULT_OPEN_X_JSON_OPTIONS, false); + assertValueFailsHive(new MapType(BIGINT, type, TYPE_OPERATORS), "{\"43\":" + jsonValue + "}", DEFAULT_OPEN_X_JSON_OPTIONS, false); + } + } + + @Test + public void testVarbinary() + { + assertValue(VARBINARY, "null", null); + + byte[] allBytes = new byte[255]; + for (int i = 0; i < allBytes.length; i++) { + allBytes[i] = (byte) (Byte.MIN_VALUE + i); + } + String base64 = Base64.getEncoder().encodeToString(allBytes); + assertValue( + VARBINARY, + "\"" + base64 + "\"", + new SqlVarbinary(allBytes)); + + // all other json fails + assertValueFails(VARBINARY, "true", DEFAULT_OPEN_X_JSON_OPTIONS, false); + assertValueFails(VARBINARY, "false", DEFAULT_OPEN_X_JSON_OPTIONS, false); + + assertValueFails(VARBINARY, "-1", DEFAULT_OPEN_X_JSON_OPTIONS, false); + assertValueFails(VARBINARY, "1.23", DEFAULT_OPEN_X_JSON_OPTIONS, false); + assertValueFails(VARBINARY, "1.23e45", DEFAULT_OPEN_X_JSON_OPTIONS, false); + + assertValueFails(VARBINARY, "\"value\"", DEFAULT_OPEN_X_JSON_OPTIONS, false); + assertValueFails(VARBINARY, "[ \"value\" ]", DEFAULT_OPEN_X_JSON_OPTIONS, false); + assertValueFails(VARBINARY, "{ \"x\" : \"value\" }", DEFAULT_OPEN_X_JSON_OPTIONS, false); + } + + @Test + public void testBoolean() + { + assertValue(BOOLEAN, "null", null); + + assertValue(BOOLEAN, "true", true); + assertValue(BOOLEAN, "false", false); + + assertValue(BOOLEAN, "\"true\"", true); + assertValue(BOOLEAN, "\"tRuE\"", true); + assertValue(BOOLEAN, "\"unknown\"", false); + assertValue(BOOLEAN, "\"null\"", false); + + assertValue(BOOLEAN, "-1", false); + assertValue(BOOLEAN, "0", false); + assertValue(BOOLEAN, "1", false); + assertValue(BOOLEAN, "1.23", false); + assertValue(BOOLEAN, "1.23e45", false); + + assertValue(BOOLEAN, "invalid", false); + assertValue(BOOLEAN, "[]", false); + assertValue(BOOLEAN, "[ 123 ]", false); + assertValue(BOOLEAN, "[ true ]", false); + assertValue(BOOLEAN, "[ \"true\" ]", false, false); + assertValue(BOOLEAN, "{ \"x\" : false }", false, false); + } + + @Test + public void testBigint() + { + testIntegralNumber(BIGINT, Long.MAX_VALUE, Long.MIN_VALUE, value -> value); + } + + @Test + public void testInteger() + { + testIntegralNumber(INTEGER, Integer.MAX_VALUE, Integer.MIN_VALUE, Ints::checkedCast); + } + + @Test + public void testSmallInt() + { + testIntegralNumber(SMALLINT, Short.MAX_VALUE, Short.MIN_VALUE, Shorts::checkedCast); + } + + @Test + public void testTinyint() + { + testIntegralNumber(TINYINT, Byte.MAX_VALUE, Byte.MIN_VALUE, SignedBytes::checkedCast); + } + + private static void testIntegralNumber(Type type, long maxValue, long minValue, LongFunction coercion) + { + assertValue(type, "null", null); + + assertNumber(type, "0", coercion.apply(0L)); + assertNumber(type, "-0", coercion.apply(0L)); + assertNumber(type, "+0", coercion.apply(0L)); + assertNumber(type, "1", coercion.apply(1L)); + assertNumber(type, "+1", coercion.apply(1L)); + assertNumber(type, "-1", coercion.apply(-1L)); + assertNumber(type, String.valueOf(maxValue), coercion.apply(maxValue)); + assertNumber(type, "+" + maxValue, coercion.apply(maxValue)); + assertNumber(type, String.valueOf(minValue), coercion.apply(minValue)); + + assertNumberOutOfBounds(type, BigInteger.valueOf(maxValue).add(BigInteger.ONE).toString(), coercion.apply(minValue)); + assertNumberOutOfBounds(type, BigInteger.valueOf(minValue).subtract(BigInteger.ONE).toString(), coercion.apply(maxValue)); + + // Decimal values are truncated + assertNumber(type, "1.23", coercion.apply(1L), true); + assertNumber(type, "1.56", coercion.apply(1L), true); + assertNumber(type, maxValue + ".9999", coercion.apply(maxValue), true); + assertNumber(type, minValue + ".9999", coercion.apply(minValue), true); + + // Exponents are expanded, and decimals are truncated + assertNumber(type, "1.2345e2", coercion.apply(123L), true); + assertNumber(type, "%1.18e".formatted(new BigDecimal(maxValue)).trim(), coercion.apply(maxValue), true); + assertNumber(type, "%+1.18e".formatted(new BigDecimal(maxValue)).trim(), coercion.apply(maxValue), true); + assertNumber(type, "%1.18e".formatted(new BigDecimal(minValue)).trim(), coercion.apply(minValue), true); + assertNumberOutOfBounds(type, "%1.18e".formatted(new BigDecimal(maxValue).add(BigDecimal.ONE)).trim(), coercion.apply(minValue)); + assertNumberOutOfBounds(type, "%1.18e".formatted(new BigDecimal(minValue).subtract(BigDecimal.ONE)).trim(), coercion.apply(maxValue)); + + // Hex is supported + assertNumber(type, "0x0", coercion.apply(0)); + assertNumber(type, "0x1", coercion.apply(1)); + assertNumber(type, "0x" + Long.toUnsignedString(maxValue, 16), coercion.apply(maxValue)); + // But negative values are not allowed + assertInvalidNumber(type, "0x8000000000000000"); + assertInvalidNumber(type, "0xFFFFFFFFFFFFFFFF"); + assertInvalidNumber(type, "0x" + Long.toUnsignedString(minValue, 16)); + + // Octal is supported + assertNumber(type, "00", coercion.apply(0)); + assertNumber(type, "01", coercion.apply(1)); + assertNumber(type, "0" + Long.toUnsignedString(maxValue, 8), coercion.apply(maxValue), true); + // But negative values are not allowed + // Only test with Trino here, as Hive handling of octal is very broken + assertValueTrino(type, "01777777777777777777777", null, DEFAULT_OPEN_X_JSON_OPTIONS, true); + assertValueTrino(type, "07777777777777777777777", null, DEFAULT_OPEN_X_JSON_OPTIONS, true); + assertValueTrino(type, "0" + Long.toUnsignedString(minValue, 8), null, DEFAULT_OPEN_X_JSON_OPTIONS, true); + + // all other string values are invalid + assertInvalidNumber(type, "\"\""); + assertInvalidNumber(type, "invalid"); + assertInvalidNumber(type, "true"); + assertInvalidNumber(type, "false"); + assertInvalidNumber(type, "\"null\""); + + // array and object cause failures + assertValueFails(type, "[ 42 ]", DEFAULT_OPEN_X_JSON_OPTIONS, false); + assertValueFails(type, "{ \"x\" : 42 }", DEFAULT_OPEN_X_JSON_OPTIONS, false); + } + + private static void assertNumber(Type type, String jsonValue, Number expectedValue) + { + assertNumber(type, jsonValue, expectedValue, false); + } + + private static void assertNumber(Type type, String jsonValue, Number expectedValue, boolean hiveQuotedValueFails) + { + assertValueTrino(type, jsonValue, expectedValue, DEFAULT_OPEN_X_JSON_OPTIONS, true); + assertValueTrino(type, "\"" + jsonValue + "\"", expectedValue, DEFAULT_OPEN_X_JSON_OPTIONS, true); + + assertValueHive(type, jsonValue, expectedValue, DEFAULT_OPEN_X_JSON_OPTIONS, !hiveQuotedValueFails); + if (hiveQuotedValueFails) { + assertValueFailsHive(type, "\"" + jsonValue + "\"", DEFAULT_OPEN_X_JSON_OPTIONS, false); + } + else { + assertValueHive(type, "\"" + jsonValue + "\"", expectedValue, DEFAULT_OPEN_X_JSON_OPTIONS, true); + } + } + + private static void assertNumberOutOfBounds(Type type, String jsonValue, Number hiveExpectedValue) + { + assertValueTrino(type, jsonValue, null, DEFAULT_OPEN_X_JSON_OPTIONS, true); + assertValueTrino(type, "\"" + jsonValue + "\"", null, DEFAULT_OPEN_X_JSON_OPTIONS, true); + + Object hiveActualValue = readHiveValue(type, jsonValue, DEFAULT_OPEN_X_JSON_OPTIONS); + if (!hiveActualValue.equals(hiveExpectedValue)) { + // Hive reads values using double which will lose precision, so Integer.MAX_VALUE + 1 round trips + assertThat(hiveActualValue).isInstanceOf(Integer.class); + if (hiveExpectedValue.intValue() < 0) { + assertThat(hiveActualValue).isEqualTo(Integer.MAX_VALUE); + } + else { + assertThat(hiveActualValue).isEqualTo(Integer.MIN_VALUE); + } + } + } + + private static void assertInvalidNumber(Type type, String jsonValue) + { + assertValueTrino(type, jsonValue, null, DEFAULT_OPEN_X_JSON_OPTIONS, true); + if (!jsonValue.startsWith("\"")) { + assertValueTrino(type, "\"" + jsonValue + "\"", null, DEFAULT_OPEN_X_JSON_OPTIONS, true); + } + + assertValueFailsHive(type, jsonValue, DEFAULT_OPEN_X_JSON_OPTIONS, false); + if (!jsonValue.startsWith("\"")) { + assertValueFailsHive(type, "\"" + jsonValue + "\"", DEFAULT_OPEN_X_JSON_OPTIONS, false); + } + } + + @Test + public void testDecimalShort() + { + assertValue(SHORT_DECIMAL, "10000000000000000.00", null); + assertValue(SHORT_DECIMAL, "null", null); + + // allowed range for JsonSerDe + assertDecimal(SHORT_DECIMAL, "0"); + assertDecimal(SHORT_DECIMAL, "1"); + assertDecimal(SHORT_DECIMAL, "-1"); + assertDecimal(SHORT_DECIMAL, "9999999999999999.99"); + assertDecimal(SHORT_DECIMAL, "-9999999999999999.99"); + + assertDecimal(SHORT_DECIMAL, "1.2345e2"); + assertDecimal(SHORT_DECIMAL, "1.5645e15"); + + // Hive does not enforce size bounds + assertValue(SHORT_DECIMAL, "10000000000000000.00", null); + assertValue(SHORT_DECIMAL, "-10000000000000000.00", null); + assertValue(SHORT_DECIMAL, "1e19", null); + assertValue(SHORT_DECIMAL, "-1e19", null); + + // test rounding + DecimalType roundingType = createDecimalType(4, 2); + assertValue(roundingType, "10.001", SqlDecimal.decimal("10.00", roundingType)); + assertValue(roundingType, "10.005", SqlDecimal.decimal("10.01", roundingType)); + assertValue(roundingType, "99.999", null); + + assertValue(SHORT_DECIMAL, "invalid", null); + assertValue(SHORT_DECIMAL, "true", null); + assertValue(SHORT_DECIMAL, "false", null); + assertValue(SHORT_DECIMAL, "\"string\"", null); + assertValue(SHORT_DECIMAL, "\"null\"", null); + + assertValueFailsTrino(SHORT_DECIMAL, "[ 42 ]", DEFAULT_OPEN_X_JSON_OPTIONS, false); + assertValueFailsTrino(SHORT_DECIMAL, "{ \"x\" : 42 }", DEFAULT_OPEN_X_JSON_OPTIONS, false); + } + + @Test + public void testDecimalLong() + { + assertValue(LONG_DECIMAL, "null", null); + + // allowed range for JsonSerDe + assertDecimal(LONG_DECIMAL, "0"); + assertDecimal(LONG_DECIMAL, "1"); + assertDecimal(LONG_DECIMAL, "-1"); + assertDecimal(LONG_DECIMAL, "9999999999999999.99"); + assertDecimal(LONG_DECIMAL, "-9999999999999999.99"); + assertDecimal(LONG_DECIMAL, "10000000000000000.00"); + assertDecimal(LONG_DECIMAL, "-10000000000000000.00"); + assertDecimal(LONG_DECIMAL, "999999999999999999999999999999999999.99"); + assertDecimal(LONG_DECIMAL, "-999999999999999999999999999999999999.99"); + + assertDecimal(LONG_DECIMAL, "1.2345e2"); + assertDecimal(LONG_DECIMAL, "1.5645e15"); + assertDecimal(LONG_DECIMAL, "1.5645e35"); + + // Hive does not enforce size bounds + assertValue(LONG_DECIMAL, "1000000000000000000000000000000000000.00", null); + assertValue(LONG_DECIMAL, "-1000000000000000000000000000000000000.00", null); + assertValue(LONG_DECIMAL, "1e39", null); + assertValue(LONG_DECIMAL, "-1e39", null); + + // test rounding (Hive doesn't seem to enforce scale) + DecimalType roundingType = createDecimalType(38, 2); + assertValue(roundingType, "10.001", SqlDecimal.decimal("10.00", roundingType)); + assertValue(roundingType, "10.005", SqlDecimal.decimal("10.01", roundingType)); + + assertValue(LONG_DECIMAL, "invalid", null); + assertValue(LONG_DECIMAL, "true", null); + assertValue(LONG_DECIMAL, "false", null); + assertValue(LONG_DECIMAL, "\"string\"", null); + assertValue(LONG_DECIMAL, "\"null\"", null); + + assertValueFailsTrino(LONG_DECIMAL, "[ 42 ]", DEFAULT_OPEN_X_JSON_OPTIONS, false); + assertValueFailsTrino(LONG_DECIMAL, "{ \"x\" : 42 }", DEFAULT_OPEN_X_JSON_OPTIONS, false); + } + + private static void assertDecimal(DecimalType decimalType, String jsonValue) + { + SqlDecimal expectedValue = toSqlDecimal(decimalType, jsonValue); + assertValue(decimalType, jsonValue, expectedValue); + // value can be passed as a JSON string + assertValue(decimalType, "\"" + jsonValue + "\"", expectedValue); + } + + private static SqlDecimal toSqlDecimal(DecimalType decimalType, String expectedValueString) + { + BigDecimal bigDecimal = new BigDecimal(expectedValueString); + BigDecimal newBigDecimal = Decimals.rescale(bigDecimal, decimalType); + SqlDecimal expectedValue = new SqlDecimal(newBigDecimal.unscaledValue(), decimalType.getPrecision(), decimalType.getScale()); + return expectedValue; + } + + @Test + public void testReal() + { + assertValue(REAL, "null", null); + + // allowed range for JsonSerDe + assertValue(REAL, "0", 0.0f); + assertValue(REAL, "123", 123.0f); + assertValue(REAL, "-123", -123.0f); + assertValue(REAL, "1.23", 1.23f); + assertValue(REAL, "-1.23", -1.23f); + assertValue(REAL, "1.5645e33", 1.5645e33f); + + assertValue(REAL, "NaN", Float.NaN); + assertValue(REAL, "Infinity", Float.POSITIVE_INFINITY); + assertValue(REAL, "+Infinity", Float.POSITIVE_INFINITY); + assertValue(REAL, "-Infinity", Float.NEGATIVE_INFINITY); + assertValueTrinoOnly(REAL, "+Inf", null); + assertValueTrinoOnly(REAL, "-Inf", null); + + assertValueTrinoOnly(REAL, "invalid", null); + assertValueTrinoOnly(REAL, "true", null); + assertValueTrinoOnly(REAL, "false", null); + assertValue(REAL, "\"123.45\"", 123.45f); + assertValueTrinoOnly(REAL, "\"string\"", null); + assertValueTrinoOnly(REAL, "\"null\"", null); + + assertValueFails(REAL, "[ 42 ]", DEFAULT_OPEN_X_JSON_OPTIONS, false); + assertValueFails(REAL, "{ \"x\" : 42 }", DEFAULT_OPEN_X_JSON_OPTIONS, false); + } + + @Test + public void testDouble() + { + assertValue(DOUBLE, "null", null); + + // allowed range for JsonSerDe + assertValue(DOUBLE, "0", 0.0); + assertValueTrino(DOUBLE, "-0", -0.0, DEFAULT_OPEN_X_JSON_OPTIONS, true); + assertValue(DOUBLE, "123", 123.0); + assertValue(DOUBLE, "-123", -123.0); + assertValue(DOUBLE, "1.23", 1.23); + assertValue(DOUBLE, "-1.23", -1.23); + assertValue(DOUBLE, "1.5645e33", 1.5645e33); + + assertValue(DOUBLE, "NaN", Double.NaN); + assertValue(DOUBLE, "Infinity", Double.POSITIVE_INFINITY); + assertValue(DOUBLE, "+Infinity", Double.POSITIVE_INFINITY); + assertValue(DOUBLE, "-Infinity", Double.NEGATIVE_INFINITY); + assertValueTrinoOnly(DOUBLE, "+Inf", null); + assertValueTrinoOnly(DOUBLE, "-Inf", null); + + assertValueTrinoOnly(DOUBLE, "invalid", null); + assertValueTrinoOnly(DOUBLE, "true", null); + assertValueTrinoOnly(DOUBLE, "false", null); + assertValue(DOUBLE, "\"123.45\"", 123.45); + assertValueTrinoOnly(DOUBLE, "\"string\"", null); + assertValueTrinoOnly(DOUBLE, "\"null\"", null); + + assertValueFails(DOUBLE, "[ 42 ]", DEFAULT_OPEN_X_JSON_OPTIONS, false); + assertValueFails(DOUBLE, "{ \"x\" : 42 }", DEFAULT_OPEN_X_JSON_OPTIONS, false); + } + + @Test + public void testDate() + { + assertValue(DATE, "null", null); + + // allowed range for JsonSerDe + assertDate("\"1970-01-01\"", 0); + assertDate("\"1970-01-02\"", 1); + assertDate("\"1969-12-31\"", -1); + + // Hive ignores everything after the first space + assertDate("\"1986-01-01 anything is allowed here\"", LocalDate.of(1986, 1, 1).toEpochDay()); + + assertDate("\"1986-01-01\"", LocalDate.of(1986, 1, 1).toEpochDay()); + assertDate("\"1986-01-33\"", LocalDate.of(1986, 2, 2).toEpochDay()); + + assertDate("\"5881580-07-11\"", Integer.MAX_VALUE); + assertDate("\"-5877641-06-23\"", Integer.MIN_VALUE); + + // Hive does not enforce size bounds and truncates the results in Date.toEpochDay + assertValueTrinoOnly(DATE, "\"5881580-07-12\"", null); + assertValueTrinoOnly(DATE, "\"-5877641-06-22\"", null); + + // numbers are translated into epoch days + assertDate("0", 0, false); + assertDate("1", 1, false); + assertDate("-1", -1, false); + assertDate("123", 123, false); + assertDate(String.valueOf(Integer.MAX_VALUE), Integer.MAX_VALUE, false); + assertDate(String.valueOf(Integer.MIN_VALUE), Integer.MIN_VALUE, false); + + // hex + assertDate("0x0", 0, false); + assertDate("0x1", 1, false); + assertDate("0x123", 0x123, false); + assertDate("0x" + Long.toUnsignedString(Integer.MAX_VALUE, 16), Integer.MAX_VALUE, false); + + // octal + assertDate("00", 0, false); + assertDate("01", 1, false); + assertDate("0123", 83, false); + assertDate("0" + Long.toUnsignedString(Integer.MAX_VALUE, 8), Integer.MAX_VALUE, false); + + // out of bounds + assertValueTrinoOnly(DATE, String.valueOf(Integer.MAX_VALUE + 1L), null); + assertValueTrinoOnly(DATE, String.valueOf(Integer.MIN_VALUE - 1L), null); + + // unsupported values are null + assertValueTrinoOnly(DATE, "1.23", null); + assertValueTrinoOnly(DATE, "1.2345e2", null); + assertValueTrinoOnly(DATE, "1.56", null); + assertValueTrinoOnly(DATE, "1.5645e2", null); + assertValueTrinoOnly(DATE, "1.5645e300", null); + + assertValueTrinoOnly(DATE, "true", null); + assertValueTrinoOnly(DATE, "false", null); + assertValueTrinoOnly(DATE, "\"string\"", null); + assertValueTrinoOnly(DATE, "\"null\"", null); + + assertValueFails(DATE, "[ 42 ]", DEFAULT_OPEN_X_JSON_OPTIONS, false); + assertValueFails(DATE, "{ \"x\" : 42 }", DEFAULT_OPEN_X_JSON_OPTIONS, false); + } + + private static void assertDate(String jsonValue, long days) + { + assertDate(jsonValue, days, true); + } + + private static void assertDate(String jsonValue, long days, boolean testHiveMapKey) + { + Object expectedValue = new SqlDate(toIntExact(days)); + assertValueHive(DATE, jsonValue, expectedValue, DEFAULT_OPEN_X_JSON_OPTIONS, testHiveMapKey); + assertValueTrino(DATE, jsonValue, expectedValue, DEFAULT_OPEN_X_JSON_OPTIONS, true); + } + + @Test + public void testTimestamp() + { + testTimestamp(TIMESTAMP_NANOS); + testTimestamp(TIMESTAMP_MICROS); + testTimestamp(TIMESTAMP_MILLIS); + testTimestamp(TIMESTAMP_SECONDS); + } + + private static void testTimestamp(TimestampType timestampType) + { + assertValue(timestampType, "null", null); + + // Standard SQL format + assertTimestamp(timestampType, "2020-05-10 12:34:56.123456789", LocalDateTime.of(2020, 5, 10, 12, 34, 56, 123_456_789)); + assertTrinoTimestamp(timestampType, "2020-5-6 7:8:9.123456789", LocalDateTime.of(2020, 5, 6, 7, 8, 9, 123_456_789)); + assertTimestamp(timestampType, "2020-05-10 12:34:56.123456", LocalDateTime.of(2020, 5, 10, 12, 34, 56, 123_456_000)); + assertTimestamp(timestampType, "2020-05-10 12:34:56.123", LocalDateTime.of(2020, 5, 10, 12, 34, 56, 123_000_000)); + assertTimestamp(timestampType, "2020-05-10 12:34:56", LocalDateTime.of(2020, 5, 10, 12, 34, 56)); + assertInvalidTimestamp(timestampType, "2020-05-10 12:34"); + assertInvalidTimestamp(timestampType, "2020-05-10 12"); + assertInvalidTimestamp(timestampType, "2020-05-10"); + + // Lenient timestamp parsing is used and values wrap around + assertTrinoTimestamp(timestampType, "2020-13-10 12:34:56.123456789", LocalDateTime.of(2021, 1, 10, 12, 34, 56, 123_456_789)); + assertTrinoTimestamp(timestampType, "2020-05-35 12:34:56.123456789", LocalDateTime.of(2020, 6, 4, 12, 34, 56, 123_456_789)); + assertTrinoTimestamp(timestampType, "2020-05-10 12:65:56.123456789", LocalDateTime.of(2020, 5, 10, 13, 5, 56, 123_456_789)); + + assertTimestamp(timestampType, "1970-01-01 00:00:00.000000000", LocalDateTime.of(1970, 1, 1, 0, 0, 0, 0)); + assertTimestamp(timestampType, "1960-05-10 12:34:56.123456789", LocalDateTime.of(1960, 5, 10, 12, 34, 56, 123_456_789)); + + // test bounds + assertTimestamp(timestampType, + "294247-01-10 04:00:54" + truncateNanosFor(timestampType, ".775807999"), + LocalDateTime.of(294247, 1, 10, 4, 0, 54, truncateNanosFor(timestampType, 775_807_999))); + assertTrinoTimestamp(timestampType, "-290308-12-21 19:59:06.224192000", LocalDateTime.of(-290308, 12, 21, 19, 59, 6, 224_192_000)); + + assertInvalidTimestamp(timestampType, "294247-01-10 04:00:54.775808000"); + assertInvalidTimestamp(timestampType, "-290308-12-21 19:59:05.224192000"); + + // custom formats + assertTimestamp( + timestampType, + "05/10/2020 12.34.56.123", + LocalDateTime.of(2020, 5, 10, 12, 34, 56, 123_000_000), + "MM/dd/yyyy HH.mm.ss.SSS"); + assertTimestamp( + timestampType, + "10.05.2020 12:34", + LocalDateTime.of(2020, 5, 10, 12, 34, 0, 0), + "dd.MM.yyyy HH:mm"); + assertTimestamp( + timestampType, + "05/10/2020 12.34.56.123", + LocalDateTime.of(2020, 5, 10, 12, 34, 56, 123_000_000), + "yyyy", + "MM/dd/yyyy HH.mm.ss.SSS", + "dd.MM.yyyy HH:mm"); + assertTimestamp( + timestampType, + "10.05.2020 12:34", + LocalDateTime.of(2020, 5, 10, 12, 34, 0, 0), + "yyyy", + "MM/dd/yyyy HH.mm.ss.SSS", + "dd.MM.yyyy HH:mm"); + + // Default timestamp formats are always supported when formats are supplied + assertTrinoTimestamp( + timestampType, + "2020-05-10 12:34:56.123456789", + LocalDateTime.of(2020, 5, 10, 12, 34, 56, 123_456_789), + "dd.MM.yyyy HH:mm"); + assertTrinoTimestamp( + timestampType, + "2020-05-10T12:34:56.123456789", + LocalDateTime.of(2020, 5, 10, 12, 34, 56, 123_456_789), + "dd.MM.yyyy HH:mm"); + + // fixed offset time zone is allowed + assertValueTrino(timestampType, + "\"2020-05-10T12:34:56.123456789-0800\"", + toSqlTimestamp(timestampType, LocalDateTime.of(2020, 5, 10, 12 + 8, 34, 56, 123_456_789)), + DEFAULT_OPEN_X_JSON_OPTIONS, + true); + + // seconds.millis + assertTimestampNumeric(timestampType, "456.123", LocalDateTime.ofEpochSecond(456, 123_000_000, UTC), false); + assertTimestampNumeric(timestampType, "456.1239", LocalDateTime.ofEpochSecond(456, 123_000_000, UTC), false); + assertTimestampNumeric(timestampType, "0.123", LocalDateTime.ofEpochSecond(0, 123_000_000, UTC), false); + assertTimestampNumeric(timestampType, "00.123", LocalDateTime.ofEpochSecond(0, 123_000_000, UTC), false); + assertTimestampNumeric(timestampType, ".123", LocalDateTime.ofEpochSecond(0, 123_000_000, UTC), false); + + // due to bugs, Starburst supports exponents, but Rcongiu does not + assertTimestampNumeric(timestampType, "1.2345E2", null, true); + + // 13 digits or more is millis + assertTimestampNumeric(timestampType, "1234567890123", LocalDateTime.ofEpochSecond(1234567890L, 123_000_000, UTC), false); + assertTimestampNumeric(timestampType, "12345678901234", LocalDateTime.ofEpochSecond(12345678901L, 234_000_000, UTC), false); + // leading zeros are counted (and not interpreted as octal) + assertTimestampNumeric(timestampType, "0034567890123", LocalDateTime.ofEpochSecond(34567890L, 123_000_000, UTC), true); + // sign is counted + assertTimestampNumeric(timestampType, "+234567890123", LocalDateTime.ofEpochSecond(234567890L, 123_000_000, UTC), true); + assertTimestampNumeric(timestampType, "-234567890123", LocalDateTime.ofEpochSecond(-234567891L, 877_000_000, UTC), false); + + // 12 digits or fewer is seconds + assertTimestampNumeric(timestampType, "123456789012", LocalDateTime.ofEpochSecond(123456789012L, 0, UTC), false); + assertTimestampNumeric(timestampType, "12345678901", LocalDateTime.ofEpochSecond(12345678901L, 0, UTC), false); + + // hex is not supported + assertInvalidTimestamp(timestampType, "0x123"); + + // values that don't parse are null + assertInvalidTimestamp(timestampType, "true"); + assertInvalidTimestamp(timestampType, "false"); + assertInvalidTimestamp(timestampType, "string"); + + assertValueFails(timestampType, "[ 42 ]", DEFAULT_OPEN_X_JSON_OPTIONS, false); + assertValueFails(timestampType, "{ x : 42 }", DEFAULT_OPEN_X_JSON_OPTIONS, false); + } + + private static void assertTimestamp(TimestampType timestampType, String jsonValue, LocalDateTime expectedDateTime, String... timestampFormats) + { + assertTimestamp(timestampType, jsonValue, expectedDateTime, ImmutableList.copyOf(timestampFormats), true); + } + + private static void assertTrinoTimestamp(TimestampType timestampType, String jsonValue, LocalDateTime expectedDateTime, String... timestampFormats) + { + assertTimestamp(timestampType, jsonValue, expectedDateTime, ImmutableList.copyOf(timestampFormats), false); + } + + private static void assertTimestamp(TimestampType timestampType, String timestampString, LocalDateTime expectedDateTime, List timestampFormats, boolean testHive) + { + SqlTimestamp expectedTimestamp = toSqlTimestamp(timestampType, expectedDateTime); + OpenXJsonOptions options = OpenXJsonOptions.builder().timestampFormats(timestampFormats).build(); + + List testValues = new ArrayList<>(); + testValues.add(timestampString); + if (timestampFormats.isEmpty()) { + for (Character separator : ImmutableList.of('T', 't')) { + for (String zone : ImmutableList.of("", "z", "Z", "-0000", "+0000", "-00:00", "+00:00")) { + testValues.add(timestampString.replace(' ', separator) + zone); + } + } + } + for (String testValue : testValues) { + try { + if (!testHive) { + assertValueTrino(timestampType, "\"" + testValue + "\"", expectedTimestamp, options, true); + } + else if (!isSupportedByHiveTimestamp(expectedDateTime, testValue)) { + // Hive code can not parse very negative dates + assertValueTrinoOnly(timestampType, "\"" + testValue + "\"", expectedTimestamp, options); + } + else { + assertValue(timestampType, "\"" + testValue + "\"", expectedTimestamp, options, true); + } + } + catch (Throwable e) { + throw new RuntimeException(testValue, e); + } + } + } + + private static boolean isSupportedByHiveTimestamp(LocalDateTime expectedDateTime, String testValue) + { + if (testValue.endsWith("z")) { + return false; + } + if (expectedDateTime.getYear() < -10_000) { + // ends with Z or fixed zone + return !testValue.matches("(?i).*([-+]\\d{4}|z)"); + } + if (expectedDateTime.getYear() > 10_000) { + return !testValue.toLowerCase(Locale.ROOT).contains("t"); + } + return true; + } + + private static void assertTimestampNumeric(TimestampType timestampType, String jsonValue, LocalDateTime expectedDateTime, boolean trinoOnly) + { + SqlTimestamp expectedTimestamp = toSqlTimestamp(timestampType, expectedDateTime); + + for (String testJson : ImmutableList.of(jsonValue, jsonValue.toLowerCase(Locale.ROOT))) { + if (trinoOnly) { + assertValueTrinoOnly(timestampType, testJson, expectedTimestamp); + + // In Starburst, quoted octal is parsed as a decimal, but unquoted octal is parsed as octal + // Trino always parses as decimal + if ((testJson.startsWith("0") || testJson.startsWith("+")) && CharMatcher.inRange('0', '9').matchesAllOf(jsonValue.substring(1))) { + assertValue(timestampType, "\"" + testJson + "\"", expectedTimestamp); + } + else { + assertValueTrinoOnly(timestampType, "\"" + testJson + "\"", expectedTimestamp); + } + } + else { + assertValue(timestampType, testJson, expectedTimestamp); + assertValue(timestampType, "\"" + testJson + "\"", expectedTimestamp); + } + } + } + + private static void assertInvalidTimestamp(TimestampType timestampType, String jsonValue, String... timestampFormats) + { + OpenXJsonOptions options = OpenXJsonOptions.builder().timestampFormats(timestampFormats).build(); + assertValueTrinoOnly(timestampType, "\"" + jsonValue + "\"", null, options); + + if (CharMatcher.whitespace().or(CharMatcher.is(':')).matchesNoneOf(jsonValue)) { + assertValueTrinoOnly(timestampType, jsonValue, null, options); + } + } + + private static String truncateNanosFor(TimestampType timestampType, String nanoString) + { + if (timestampType.getPrecision() == 0) { + return ""; + } + return nanoString.substring(0, timestampType.getPrecision() + 1); + } + + private static int truncateNanosFor(TimestampType timestampType, int nanos) + { + long nanoRescale = (long) Math.pow(10, 9 - timestampType.getPrecision()); + return toIntExact((nanos / nanoRescale) * nanoRescale); + } + + private static void assertValue(Type type, String jsonValue, Object expectedValue) + { + assertValue(type, jsonValue, expectedValue, true); + } + + private static void assertValue(Type type, String jsonValue, Object expectedValue, boolean testMapKey) + { + assertValue(type, jsonValue, expectedValue, DEFAULT_OPEN_X_JSON_OPTIONS, testMapKey); + } + + private static void assertValue(Type type, String jsonValue, Object expectedValue, OpenXJsonOptions options, boolean testMapKey) + { + assertValueHive(type, jsonValue, expectedValue, options, testMapKey); + assertValueTrino(type, jsonValue, expectedValue, options, testMapKey); + } + + private static void assertLine(List columns, String line, List expectedValues, OpenXJsonOptions options) + { + internalAssertLineHive(columns, line, expectedValues, options); + internalAssertLineTrino(columns, line, expectedValues, options); + } + + private static void assertValueTrinoOnly(Type type, String jsonValue, Object expectedValue) + { + assertValueTrinoOnly(type, jsonValue, expectedValue, DEFAULT_OPEN_X_JSON_OPTIONS); + } + + private static void assertValueTrinoOnly(Type type, String jsonValue, Object expectedValue, OpenXJsonOptions defaultOpenXJsonOptions) + { + assertValueTrino(type, jsonValue, expectedValue, defaultOpenXJsonOptions, true); + assertThatThrownBy(() -> assertValueHive(type, jsonValue, expectedValue, defaultOpenXJsonOptions, true)); + } + + private static void assertValueTrino(Type type, String jsonValue, Object expectedValue, OpenXJsonOptions options, boolean testMapKey) + { + internalAssertValueTrino(type, jsonValue, expectedValue, options); + internalAssertValueTrino(new ArrayType(type), "[" + jsonValue + "]", singletonList(expectedValue), options); + internalAssertValueTrino( + rowType(field("a", type), field("nested", type), field("b", type)), + "{ \"nested\" : " + jsonValue + " }", + Arrays.asList(null, expectedValue, null), + options); + if (expectedValue != null) { + internalAssertValueTrino( + new MapType(BIGINT, type, TYPE_OPERATORS), + "{ \"1234\" : " + jsonValue + " }", + singletonMap(1234L, expectedValue), + options); + } + if (expectedValue != null && isScalarType(type)) { + if (testMapKey) { + internalAssertValueTrino(toMapKeyType(type), toMapKeyJson(jsonValue), toMapKeyExpectedValue(expectedValue), options); + } + else { + internalAssertValueFailsTrino(toMapKeyType(type), toMapKeyJson(jsonValue), options); + } + } + } + + private static void internalAssertValueTrino(Type type, String jsonValue, Object expectedValue, OpenXJsonOptions options) + { + List columns = ImmutableList.of(new Column("test", type, 33)); + internalAssertLineTrino(columns, "{\"test\" : " + jsonValue + "}", singletonList(expectedValue), options); + internalAssertLineTrino(columns, "[" + jsonValue + "]", singletonList(expectedValue), options); + } + + private static void internalAssertLineTrino(List columns, String line, List expectedValues, OpenXJsonOptions options) + { + // read normal json + List actualValues = readTrinoLine(columns, line, options); + assertColumnValuesEquals(columns, actualValues, expectedValues); + + // if type is not supported (e.g., Map with complex key), skip round trip test + if (!columns.stream().map(Column::type).allMatch(OpenXJsonSerializer::isSupportedType)) { + return; + } + + // write with Trino and verify that trino reads the value back correctly + String trinoLine = writeTrinoLine(columns, expectedValues, options); + List trinoValues = readTrinoLine(columns, trinoLine, DEFAULT_OPEN_X_JSON_OPTIONS); + assertColumnValuesEquals(columns, trinoValues, expectedValues); + + // verify that Hive can read the value back + List hiveValues; + try { + hiveValues = readHiveLine(columns, trinoLine, DEFAULT_OPEN_X_JSON_OPTIONS); + } + catch (Exception e) { + // Hive can not read back timestamps that start with `+` + assertThat(e.getMessage()).isEqualTo("Cannot create timestamp, parsing error"); + return; + } + assertColumnValuesEquals(columns, hiveValues, expectedValues); + + // Check if Hive is capable of writing the values + String hiveLine; + try { + hiveLine = writeHiveLine(columns, expectedValues, options); + } + catch (Exception e) { + return; + } + // Check if Hive can read back the line it wrote + try { + hiveValues = readHiveLine(columns, hiveLine, DEFAULT_OPEN_X_JSON_OPTIONS); + assertColumnValuesEquals(columns, hiveValues, expectedValues); + } + catch (RuntimeException | AssertionError ignored) { + // If Hive does not round trip values correctly, then stop testing + // this happens for types like Char, Date, and Decimal which are not rendered + return; + } + + // verify that Trino and Hive wrote the exact same line + assertThat(trinoLine).isEqualTo(hiveLine); + } + + private static void assertExactLine(List columns, List values, String expectedLine, OpenXJsonOptions options) + { + // verify Hive produces the exact line + String hiveLine = writeHiveLine(columns, values, options); + assertThat(hiveLine).isEqualTo(expectedLine); + + // verify Trino produces the exact line + String trinoLine = writeTrinoLine(columns, values, options); + assertThat(trinoLine).isEqualTo(expectedLine); + } + + private static List readTrinoLine(List columns, String jsonLine, OpenXJsonOptions options) + { + try { + LineDeserializer deserializer = new OpenXJsonDeserializerFactory().create(columns, options.toSchema()); + PageBuilder pageBuilder = new PageBuilder(1, deserializer.getTypes()); + deserializer.deserialize(createLineBuffer(jsonLine), pageBuilder); + Page page = pageBuilder.build(); + return readTrinoValues(columns, page, 0); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private static String writeTrinoLine(List columns, List values, OpenXJsonOptions options) + { + try { + Page page = toSingleRowPage(columns, values); + + // write the data to json + LineSerializer serializer = new OpenXJsonSerializerFactory().create(columns, options.toSchema()); + SliceOutput sliceOutput = new DynamicSliceOutput(1024); + serializer.write(page, 0, sliceOutput); + return sliceOutput.slice().toStringUtf8(); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private static void assertValueHive(Type type, String jsonValue, Object expectedValue, OpenXJsonOptions options, boolean testMapKey) + { + // The non-hcatalog version of JsonSerDe has a broken implementation of ordinal fields that always fails + internalAssertValueHive(type, jsonValue, expectedValue, options); + if (expectedValue != null && isScalarType(type)) { + if (testMapKey) { + internalAssertValueHive(toMapKeyType(type), toMapKeyJson(jsonValue), toMapKeyExpectedValue(expectedValue), options); + } + else { + internalAssertValueFailsHive(toMapKeyType(type), toMapKeyJson(jsonValue), options); + } + } + } + + private static void internalAssertValueHive(Type type, String jsonValue, Object expectedValue, OpenXJsonOptions options) + { + Object actualValue = readHiveValue(type, jsonValue, options); + assertColumnValueEquals(type, actualValue, expectedValue); + } + + private static void internalAssertLineHive(List columns, String jsonLine, List expectedValues, OpenXJsonOptions options) + { + List actualValues = readHiveLine(columns, jsonLine, options); + assertColumnValuesEquals(columns, actualValues, expectedValues); + } + + private static Object readHiveValue(Type type, String jsonValue, OpenXJsonOptions options) + { + List columns = ImmutableList.of( + new Column("a", BIGINT, 0), + new Column("test", type, 1), + new Column("b", BIGINT, 2)); + String jsonLine = "{\"test\" : " + jsonValue + "}"; + return readHiveLine(columns, jsonLine, options).get(1); + } + + private static List readHiveLine(List columns, String jsonLine, OpenXJsonOptions options) + { + try { + Deserializer deserializer = createHiveSerDe(columns, options); + + Object rowData = deserializer.deserialize(new Text(jsonLine)); + + List fieldValues = new ArrayList<>(); + StructObjectInspector rowInspector = (StructObjectInspector) deserializer.getObjectInspector(); + for (Column column : columns) { + StructField field = rowInspector.getStructFieldRef(column.name()); + Object fieldValue = rowInspector.getStructFieldData(rowData, field); + fieldValue = unwrapJsonObject(fieldValue, field.getFieldObjectInspector()); + fieldValue = decodeRecordReaderValue(column.type(), fieldValue); + fieldValues.add(fieldValue); + } + return fieldValues; + } + catch (SerDeException e) { + throw new RuntimeException(e); + } + } + + private static JsonSerDe createHiveSerDe(List columns, OpenXJsonOptions options) + { + try { + JobConf configuration = new JobConf(newEmptyConfiguration()); + + Properties schema = new Properties(); + schema.putAll(createOpenXJsonSerDeProperties(columns, options)); + + JsonSerDe jsonSerDe = new JsonSerDe(); + jsonSerDe.initialize(configuration, schema); + configuration.set(SERIALIZATION_LIB, jsonSerDe.getClass().getName()); + return jsonSerDe; + } + catch (SerDeException e) { + throw new RuntimeException(e); + } + } + + private static Object unwrapJsonObject(Object object, ObjectInspector inspector) + { + if (object == null) { + return null; + } + return switch (inspector.getCategory()) { + case PRIMITIVE -> ((PrimitiveObjectInspector) inspector).getPrimitiveJavaObject(object); + case LIST -> serializeList(object, (ListObjectInspector) inspector); + case MAP -> serializeMap(object, (MapObjectInspector) inspector, false); + case STRUCT -> serializeStruct(object, (StructObjectInspector) inspector); + case UNION -> throw new UnsupportedOperationException("Union not implemented"); + }; + } + + private static List serializeList(Object object, ListObjectInspector inspector) + { + List list = inspector.getList(object); + if (list == null) { + return null; + } + + ObjectInspector elementInspector = inspector.getListElementObjectInspector(); + return list.stream() + .map(element -> unwrapJsonObject(element, elementInspector)) + .collect(Collectors.toCollection(ArrayList::new)); + } + + private static Map serializeMap(Object object, MapObjectInspector inspector, boolean filterNullMapKeys) + { + Map map = inspector.getMap(object); + if (map == null) { + return null; + } + + ObjectInspector keyInspector = inspector.getMapKeyObjectInspector(); + ObjectInspector valueInspector = inspector.getMapValueObjectInspector(); + Map values = new HashMap<>(); + for (Map.Entry entry : map.entrySet()) { + // Hive skips map entries with null keys + if (!filterNullMapKeys || entry.getKey() != null) { + values.put( + unwrapJsonObject(entry.getKey(), keyInspector), + unwrapJsonObject(entry.getValue(), valueInspector)); + } + } + return values; + } + + private static List serializeStruct(Object object, StructObjectInspector inspector) + { + if (object == null) { + return null; + } + + List fields = inspector.getAllStructFieldRefs(); + List values = new ArrayList<>(); + for (StructField field : fields) { + values.add(unwrapJsonObject(inspector.getStructFieldData(object, field), field.getFieldObjectInspector())); + } + return values; + } + + private static String writeHiveLine(List columns, List values, OpenXJsonOptions options) + { + SettableStructObjectInspector objectInspector = getStandardStructObjectInspector( + columns.stream().map(Column::name).collect(toImmutableList()), + columns.stream().map(Column::type).map(FormatTestUtils::getJavaObjectInspector).collect(toImmutableList())); + + Object row = objectInspector.create(); + for (int i = 0; i < columns.size(); i++) { + Object value = toHiveWriteValue(columns.get(i).type(), values.get(i), Optional.empty()); + objectInspector.setStructFieldData(row, objectInspector.getAllStructFieldRefs().get(i), value); + } + + JsonSerDe serializer = createHiveSerDe(columns, options); + try { + return serializer.serialize(row, objectInspector).toString(); + } + catch (SerDeException e) { + throw new RuntimeException(e); + } + } + + private static void assertLineFails(List columns, String jsonValue, OpenXJsonOptions options) + { + assertLineFailsHive(columns, jsonValue, options); + assertLineFailsTrino(columns, jsonValue, options); + } + + private static void assertValueFails(Type type, String jsonValue) + { + assertValueFails(type, jsonValue, DEFAULT_OPEN_X_JSON_OPTIONS, true); + } + + private static void assertValueFails(Type type, String jsonValue, OpenXJsonOptions options, boolean testMapKey) + { + assertValueFailsHive(type, jsonValue, options, testMapKey); + assertValueFailsTrino(type, jsonValue, options, testMapKey); + } + + private static void assertValueFailsTrino(Type type, String jsonValue, OpenXJsonOptions options, boolean testMapKey) + { + internalAssertValueFailsTrino(type, jsonValue, options); + + // ignore array and object json + if (testMapKey && isScalarType(type)) { + internalAssertValueFailsTrino(toMapKeyType(type), toMapKeyJson(jsonValue), options); + } + } + + private static void internalAssertValueFailsTrino(Type type, String jsonValue, OpenXJsonOptions options) + { + String jsonLine = "{\"test\" : " + jsonValue + "}"; + List columns = ImmutableList.of(new Column("test", type, 33)); + assertLineFailsTrino(columns, jsonLine, options); + } + + private static void assertLineFailsTrino(List columns, String jsonLine, OpenXJsonOptions options) + { + LineDeserializer deserializer = new OpenXJsonDeserializerFactory().create(columns, options.toSchema()); + assertThatThrownBy(() -> deserializer.deserialize(createLineBuffer(jsonLine), new PageBuilder(1, deserializer.getTypes()))) + .isInstanceOf(Exception.class); + } + + private static void assertValueFailsHive(Type type, String jsonValue, OpenXJsonOptions options, boolean testMapKey) + { + internalAssertValueFailsHive(type, jsonValue, options); + if (testMapKey && isScalarType(type)) { + internalAssertValueFailsHive(toMapKeyType(type), toMapKeyJson(jsonValue), options); + } + } + + private static void internalAssertValueFailsHive(Type type, String jsonValue, OpenXJsonOptions options) + { + List columns = ImmutableList.of(new Column("test", type, 0)); + String jsonLine = "{\"test\" : " + jsonValue + "}"; + assertLineFailsHive(columns, jsonLine, options); + } + + private static void assertLineFailsHive(List columns, String jsonLine, OpenXJsonOptions options) + { + assertThatThrownBy(() -> { + JobConf configuration = new JobConf(newEmptyConfiguration()); + + Properties schema = new Properties(); + schema.putAll(createOpenXJsonSerDeProperties(columns, options)); + + Deserializer deserializer = new JsonSerDe(); + deserializer.initialize(configuration, schema); + configuration.set(SERIALIZATION_LIB, deserializer.getClass().getName()); + + Object rowData = deserializer.deserialize(new Text(jsonLine)); + + StructObjectInspector rowInspector = (StructObjectInspector) deserializer.getObjectInspector(); + for (Column column : columns) { + StructField field = rowInspector.getStructFieldRef(column.name()); + Object actualValue = rowInspector.getStructFieldData(rowData, field); + actualValue = unwrapJsonObject(actualValue, field.getFieldObjectInspector()); + decodeRecordReaderValue(column.type(), actualValue); + } + }); + } + + private static MapType toMapKeyType(Type type) + { + assertTrue(isScalarType(type)); + return new MapType(type, BIGINT, TYPE_OPERATORS); + } + + private static String toMapKeyJson(String jsonValue) + { + if (jsonValue.startsWith("\"")) { + jsonValue = jsonValue.substring(1, jsonValue.length() - 1); + } + return "{ \"" + jsonValue + "\" : 8675309 }"; + } + + private static Map toMapKeyExpectedValue(Object value) + { + return ImmutableMap.of(value, 8675309L); + } + + private static Map createOpenXJsonSerDeProperties(List columns, OpenXJsonOptions options) + { + ImmutableMap.Builder schema = ImmutableMap.builder(); + schema.put(LIST_COLUMNS, columns.stream() + .sorted(Comparator.comparing(Column::ordinal)) + .map(Column::name) + .collect(joining(","))); + schema.put(LIST_COLUMN_TYPES, columns.stream() + .sorted(Comparator.comparing(Column::ordinal)) + .map(Column::type) + .map(FormatTestUtils::getJavaObjectInspector) + .map(ObjectInspector::getTypeName) + .collect(joining(","))); + schema.putAll(options.toSchema()); + return schema.buildOrThrow(); + } +} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveFormatsConfig.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveFormatsConfig.java index 0bae9579277a..5a396802b633 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveFormatsConfig.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveFormatsConfig.java @@ -22,6 +22,8 @@ public class HiveFormatsConfig private boolean csvNativeWriterEnabled = true; private boolean jsonNativeReaderEnabled = true; private boolean jsonNativeWriterEnabled = true; + private boolean openXJsonNativeReaderEnabled = true; + private boolean openXJsonNativeWriterEnabled = true; private boolean regexNativeReaderEnabled = true; private boolean textFileNativeReaderEnabled = true; private boolean textFileNativeWriterEnabled = true; @@ -80,6 +82,32 @@ public HiveFormatsConfig setJsonNativeWriterEnabled(boolean jsonNativeWriterEnab return this; } + public boolean isOpenXJsonNativeReaderEnabled() + { + return openXJsonNativeReaderEnabled; + } + + @Config("openx-json.native-reader.enabled") + @ConfigDescription("Use native OpenXJson reader") + public HiveFormatsConfig setOpenXJsonNativeReaderEnabled(boolean openXJsonNativeReaderEnabled) + { + this.openXJsonNativeReaderEnabled = openXJsonNativeReaderEnabled; + return this; + } + + public boolean isOpenXJsonNativeWriterEnabled() + { + return openXJsonNativeWriterEnabled; + } + + @Config("openx-json.native-writer.enabled") + @ConfigDescription("Use native OpenXJson writer") + public HiveFormatsConfig setOpenXJsonNativeWriterEnabled(boolean openXJsonNativeWriterEnabled) + { + this.openXJsonNativeWriterEnabled = openXJsonNativeWriterEnabled; + return this; + } + public boolean isRegexNativeReaderEnabled() { return regexNativeReaderEnabled; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveModule.java index 3d203431b9a0..89df99798772 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveModule.java @@ -30,6 +30,8 @@ import io.trino.plugin.hive.line.CsvPageSourceFactory; import io.trino.plugin.hive.line.JsonFileWriterFactory; import io.trino.plugin.hive.line.JsonPageSourceFactory; +import io.trino.plugin.hive.line.OpenXJsonFileWriterFactory; +import io.trino.plugin.hive.line.OpenXJsonPageSourceFactory; import io.trino.plugin.hive.line.RegexFileWriterFactory; import io.trino.plugin.hive.line.RegexPageSourceFactory; import io.trino.plugin.hive.line.SimpleSequenceFilePageSourceFactory; @@ -130,6 +132,7 @@ public void configure(Binder binder) Multibinder pageSourceFactoryBinder = newSetBinder(binder, HivePageSourceFactory.class); pageSourceFactoryBinder.addBinding().to(CsvPageSourceFactory.class).in(Scopes.SINGLETON); pageSourceFactoryBinder.addBinding().to(JsonPageSourceFactory.class).in(Scopes.SINGLETON); + pageSourceFactoryBinder.addBinding().to(OpenXJsonPageSourceFactory.class).in(Scopes.SINGLETON); pageSourceFactoryBinder.addBinding().to(RegexPageSourceFactory.class).in(Scopes.SINGLETON); pageSourceFactoryBinder.addBinding().to(SimpleTextFilePageSourceFactory.class).in(Scopes.SINGLETON); pageSourceFactoryBinder.addBinding().to(SimpleSequenceFilePageSourceFactory.class).in(Scopes.SINGLETON); @@ -150,6 +153,7 @@ public void configure(Binder binder) fileWriterFactoryBinder.addBinding().to(CsvFileWriterFactory.class).in(Scopes.SINGLETON); fileWriterFactoryBinder.addBinding().to(JsonFileWriterFactory.class).in(Scopes.SINGLETON); fileWriterFactoryBinder.addBinding().to(RegexFileWriterFactory.class).in(Scopes.SINGLETON); + fileWriterFactoryBinder.addBinding().to(OpenXJsonFileWriterFactory.class).in(Scopes.SINGLETON); fileWriterFactoryBinder.addBinding().to(SimpleTextFileWriterFactory.class).in(Scopes.SINGLETON); fileWriterFactoryBinder.addBinding().to(SimpleSequenceFileWriterFactory.class).in(Scopes.SINGLETON); fileWriterFactoryBinder.addBinding().to(OrcFileWriterFactory.class).in(Scopes.SINGLETON); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSessionProperties.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSessionProperties.java index bebeecad22ca..31ecc2bc631d 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSessionProperties.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSessionProperties.java @@ -65,6 +65,8 @@ public final class HiveSessionProperties private static final String CSV_NATIVE_WRITER_ENABLED = "csv_native_writer_enabled"; private static final String JSON_NATIVE_READER_ENABLED = "json_native_reader_enabled"; private static final String JSON_NATIVE_WRITER_ENABLED = "json_native_writer_enabled"; + private static final String OPENX_JSON_NATIVE_READER_ENABLED = "openx_json_native_reader_enabled"; + private static final String OPENX_JSON_NATIVE_WRITER_ENABLED = "openx_json_native_writer_enabled"; private static final String REGEX_NATIVE_READER_ENABLED = "regex_native_reader_enabled"; private static final String TEXT_FILE_NATIVE_READER_ENABLED = "text_file_native_reader_enabled"; private static final String TEXT_FILE_NATIVE_WRITER_ENABLED = "text_file_native_writer_enabled"; @@ -219,6 +221,16 @@ public HiveSessionProperties( "Use native JSON writer", hiveFormatsConfig.isJsonNativeWriterEnabled(), false), + booleanProperty( + OPENX_JSON_NATIVE_READER_ENABLED, + "Use native OpenX JSON reader", + hiveFormatsConfig.isOpenXJsonNativeReaderEnabled(), + false), + booleanProperty( + OPENX_JSON_NATIVE_WRITER_ENABLED, + "Use native OpenX JSON writer", + hiveFormatsConfig.isOpenXJsonNativeWriterEnabled(), + false), booleanProperty( REGEX_NATIVE_READER_ENABLED, "Use native REGEX reader", @@ -670,6 +682,16 @@ public static boolean isJsonNativeWriterEnabled(ConnectorSession session) return session.getProperty(JSON_NATIVE_WRITER_ENABLED, Boolean.class); } + public static boolean isOpenXJsonNativeReaderEnabled(ConnectorSession session) + { + return session.getProperty(OPENX_JSON_NATIVE_READER_ENABLED, Boolean.class); + } + + public static boolean isOpenXJsonNativeWriterEnabled(ConnectorSession session) + { + return session.getProperty(OPENX_JSON_NATIVE_WRITER_ENABLED, Boolean.class); + } + public static boolean isRegexNativeReaderEnabled(ConnectorSession session) { return session.getProperty(REGEX_NATIVE_READER_ENABLED, Boolean.class); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveStorageFormat.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveStorageFormat.java index a239f8f784a8..e8ff5e61d3f3 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveStorageFormat.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveStorageFormat.java @@ -43,6 +43,7 @@ import static io.trino.plugin.hive.util.HiveClassNames.MAPRED_PARQUET_INPUT_FORMAT_CLASS; import static io.trino.plugin.hive.util.HiveClassNames.MAPRED_PARQUET_OUTPUT_FORMAT_CLASS; import static io.trino.plugin.hive.util.HiveClassNames.OPENCSV_SERDE_CLASS; +import static io.trino.plugin.hive.util.HiveClassNames.OPENX_JSON_SERDE_CLASS; import static io.trino.plugin.hive.util.HiveClassNames.ORC_INPUT_FORMAT_CLASS; import static io.trino.plugin.hive.util.HiveClassNames.ORC_OUTPUT_FORMAT_CLASS; import static io.trino.plugin.hive.util.HiveClassNames.ORC_SERDE_CLASS; @@ -93,6 +94,11 @@ public enum HiveStorageFormat TEXT_INPUT_FORMAT_CLASS, HIVE_IGNORE_KEY_OUTPUT_FORMAT_CLASS, DataSize.of(8, Unit.MEGABYTE)), + OPENX_JSON( + OPENX_JSON_SERDE_CLASS, + TEXT_INPUT_FORMAT_CLASS, + HIVE_IGNORE_KEY_OUTPUT_FORMAT_CLASS, + DataSize.of(8, Unit.MEGABYTE)), TEXTFILE( LAZY_SIMPLE_SERDE_CLASS, TEXT_INPUT_FORMAT_CLASS, diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/line/OpenXJsonFileWriterFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/line/OpenXJsonFileWriterFactory.java new file mode 100644 index 000000000000..4b77e34a57bd --- /dev/null +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/line/OpenXJsonFileWriterFactory.java @@ -0,0 +1,37 @@ +/* + * 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.trino.plugin.hive.line; + +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.hive.formats.line.openxjson.OpenXJsonSerializerFactory; +import io.trino.hive.formats.line.text.TextLineWriterFactory; +import io.trino.plugin.hive.HiveSessionProperties; +import io.trino.spi.type.TypeManager; + +import javax.inject.Inject; + +public class OpenXJsonFileWriterFactory + extends LineFileWriterFactory +{ + @Inject + public OpenXJsonFileWriterFactory(TrinoFileSystemFactory trinoFileSystemFactory, TypeManager typeManager) + { + super(trinoFileSystemFactory, + typeManager, + new OpenXJsonSerializerFactory(), + new TextLineWriterFactory(), + HiveSessionProperties::isOpenXJsonNativeWriterEnabled, + true); + } +} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/line/OpenXJsonPageSourceFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/line/OpenXJsonPageSourceFactory.java new file mode 100644 index 000000000000..0426bee32361 --- /dev/null +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/line/OpenXJsonPageSourceFactory.java @@ -0,0 +1,39 @@ +/* + * 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.trino.plugin.hive.line; + +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.hive.formats.line.openxjson.OpenXJsonDeserializerFactory; +import io.trino.hive.formats.line.text.TextLineReaderFactory; +import io.trino.plugin.hive.FileFormatDataSourceStats; +import io.trino.plugin.hive.HiveConfig; +import io.trino.plugin.hive.HiveSessionProperties; + +import javax.inject.Inject; + +import static java.lang.Math.toIntExact; + +public class OpenXJsonPageSourceFactory + extends LinePageSourceFactory +{ + @Inject + public OpenXJsonPageSourceFactory(TrinoFileSystemFactory trinoFileSystemFactory, FileFormatDataSourceStats stats, HiveConfig config) + { + super(trinoFileSystemFactory, + stats, + new OpenXJsonDeserializerFactory(), + new TextLineReaderFactory(1024, 1024, toIntExact(config.getTextMaxLineLength().toBytes())), + HiveSessionProperties::isOpenXJsonNativeReaderEnabled); + } +} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveClassNames.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveClassNames.java index 5d53b58092f0..4d21453cc7fc 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveClassNames.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveClassNames.java @@ -24,6 +24,7 @@ public final class HiveClassNames public static final String HIVE_IGNORE_KEY_OUTPUT_FORMAT_CLASS = "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"; public static final String HIVE_SEQUENCEFILE_OUTPUT_FORMAT_CLASS = "org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat"; public static final String JSON_SERDE_CLASS = "org.apache.hive.hcatalog.data.JsonSerDe"; + public static final String OPENX_JSON_SERDE_CLASS = "org.openx.data.jsonserde.JsonSerDe"; public static final String LAZY_BINARY_COLUMNAR_SERDE_CLASS = "org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe"; public static final String LAZY_SIMPLE_SERDE_CLASS = "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"; public static final String MAPRED_PARQUET_INPUT_FORMAT_CLASS = "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java index a1a4177f8624..0323495d1d45 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java @@ -5291,6 +5291,7 @@ private static Class pageSourceType(HiveStorageFo return ParquetPageSource.class; case CSV: case JSON: + case OPENX_JSON: case TEXTFILE: case SEQUENCEFILE: return LinePageSource.class; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java index d69f9979f682..fb2fd06ba2fa 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java @@ -35,6 +35,8 @@ import io.trino.plugin.hive.line.CsvPageSourceFactory; import io.trino.plugin.hive.line.JsonFileWriterFactory; import io.trino.plugin.hive.line.JsonPageSourceFactory; +import io.trino.plugin.hive.line.OpenXJsonFileWriterFactory; +import io.trino.plugin.hive.line.OpenXJsonPageSourceFactory; import io.trino.plugin.hive.line.RegexFileWriterFactory; import io.trino.plugin.hive.line.RegexPageSourceFactory; import io.trino.plugin.hive.line.SimpleSequenceFilePageSourceFactory; @@ -198,6 +200,7 @@ public static Set getDefaultHivePageSourceFactories(HdfsE return ImmutableSet.builder() .add(new CsvPageSourceFactory(fileSystemFactory, stats, hiveConfig)) .add(new JsonPageSourceFactory(fileSystemFactory, stats, hiveConfig)) + .add(new OpenXJsonPageSourceFactory(fileSystemFactory, stats, hiveConfig)) .add(new RegexPageSourceFactory(fileSystemFactory, stats, hiveConfig)) .add(new SimpleTextFilePageSourceFactory(fileSystemFactory, stats, hiveConfig)) .add(new SimpleSequenceFilePageSourceFactory(fileSystemFactory, stats, hiveConfig)) @@ -220,6 +223,7 @@ public static Set getDefaultHiveFileWriterFactories(HiveC .add(new CsvFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER)) .add(new JsonFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER)) .add(new RegexFileWriterFactory()) + .add(new OpenXJsonFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER)) .add(new SimpleTextFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER)) .add(new SimpleSequenceFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER, nodeVersion)) .add(new RcFileFileWriterFactory(hdfsEnvironment, TESTING_TYPE_MANAGER, nodeVersion, hiveConfig)) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java index ae1fad8b4129..5ffc0c9b0157 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java @@ -27,6 +27,8 @@ import io.trino.plugin.hive.line.CsvPageSourceFactory; import io.trino.plugin.hive.line.JsonFileWriterFactory; import io.trino.plugin.hive.line.JsonPageSourceFactory; +import io.trino.plugin.hive.line.OpenXJsonFileWriterFactory; +import io.trino.plugin.hive.line.OpenXJsonPageSourceFactory; import io.trino.plugin.hive.line.SimpleSequenceFilePageSourceFactory; import io.trino.plugin.hive.line.SimpleSequenceFileWriterFactory; import io.trino.plugin.hive.line.SimpleTextFilePageSourceFactory; @@ -85,6 +87,7 @@ import static io.trino.plugin.hive.HiveStorageFormat.AVRO; import static io.trino.plugin.hive.HiveStorageFormat.CSV; import static io.trino.plugin.hive.HiveStorageFormat.JSON; +import static io.trino.plugin.hive.HiveStorageFormat.OPENX_JSON; import static io.trino.plugin.hive.HiveStorageFormat.ORC; import static io.trino.plugin.hive.HiveStorageFormat.PARQUET; import static io.trino.plugin.hive.HiveStorageFormat.RCBINARY; @@ -246,6 +249,25 @@ public void testJson(int rowCount, long fileSizePadding) .isReadableByPageSource(new JsonPageSourceFactory(HDFS_FILE_SYSTEM_FACTORY, STATS, new HiveConfig())); } + @Test(dataProvider = "validRowAndFileSizePadding") + public void testOpenXJson(int rowCount, long fileSizePadding) + throws Exception + { + List testColumns = TEST_COLUMNS.stream() + // null map keys are not supported + .filter(TestHiveFileFormats::withoutNullMapKeyTests) + .collect(toList()); + + assertThatFileFormat(OPENX_JSON) + .withColumns(testColumns) + .withRowsCount(rowCount) + .withFileSizePadding(fileSizePadding) + // openx serde is not available for testing + .withSkipGenericWriterTest() + .withFileWriterFactory(new OpenXJsonFileWriterFactory(HDFS_FILE_SYSTEM_FACTORY, TESTING_TYPE_MANAGER)) + .isReadableByPageSource(new OpenXJsonPageSourceFactory(HDFS_FILE_SYSTEM_FACTORY, STATS, new HiveConfig())); + } + @Test(dataProvider = "validRowAndFileSizePadding") public void testRcTextPageSource(int rowCount, long fileSizePadding) throws Exception diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFormatsConfig.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFormatsConfig.java index 12329bc310ea..d2ec74bdb540 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFormatsConfig.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFormatsConfig.java @@ -32,6 +32,8 @@ public void testDefaults() .setCsvNativeWriterEnabled(true) .setJsonNativeReaderEnabled(true) .setJsonNativeWriterEnabled(true) + .setOpenXJsonNativeReaderEnabled(true) + .setOpenXJsonNativeWriterEnabled(true) .setRegexNativeReaderEnabled(true) .setTextFileNativeReaderEnabled(true) .setTextFileNativeWriterEnabled(true) @@ -47,6 +49,8 @@ public void testExplicitPropertyMappings() .put("csv.native-writer.enabled", "false") .put("json.native-reader.enabled", "false") .put("json.native-writer.enabled", "false") + .put("openx-json.native-reader.enabled", "false") + .put("openx-json.native-writer.enabled", "false") .put("regex.native-reader.enabled", "false") .put("text-file.native-reader.enabled", "false") .put("text-file.native-writer.enabled", "false") @@ -59,6 +63,8 @@ public void testExplicitPropertyMappings() .setCsvNativeWriterEnabled(false) .setJsonNativeReaderEnabled(false) .setJsonNativeWriterEnabled(false) + .setOpenXJsonNativeReaderEnabled(false) + .setOpenXJsonNativeWriterEnabled(false) .setRegexNativeReaderEnabled(false) .setTextFileNativeReaderEnabled(false) .setTextFileNativeWriterEnabled(false) diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveStorageFormats.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveStorageFormats.java index 609245952289..2c1b96d2f739 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveStorageFormats.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveStorageFormats.java @@ -308,6 +308,8 @@ public void verifyDataProviderCompleteness() .filter(format -> !"REGEX".equals(format)) // TODO when using JSON serde Hive fails with ClassNotFoundException: org.apache.hive.hcatalog.data.JsonSerDe .filter(format -> !"JSON".equals(format)) + // OPENX is not supported in Hive by default + .filter(format -> !"OPENX_JSON".equals(format)) .collect(toImmutableSet()); Assertions.assertThat(ImmutableSet.copyOf(storageFormats()))