+ * 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:
+ *
+{
+ /**
+ * 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 sequences
+ if (previousChar == '<') {
+ out.append('\\');
+ }
+ out.append(currentChar);
+ }
+ case '\t' -> 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 extends Comparable>> 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 extends Type> 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 extends Number> 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 extends StructField> 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 extends ConnectorPageSource> 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()))