mirror of https://github.com/apache/druid.git
stricter behavior for parse_json, add try_parse_json, remove to_json (#12920)
This commit is contained in:
parent
6fec1d4c95
commit
289e43281e
|
@ -80,7 +80,7 @@ public class NestedDataExpressions
|
|||
ExprEval value = args.get(i + 1).eval(bindings);
|
||||
|
||||
Preconditions.checkArgument(field.type().is(ExprType.STRING), "field name must be a STRING");
|
||||
theMap.put(field.asString(), maybeUnwrapStructuredData(value.value()));
|
||||
theMap.put(field.asString(), unwrap(value));
|
||||
}
|
||||
|
||||
return ExprEval.ofComplex(TYPE, theMap);
|
||||
|
@ -115,54 +115,6 @@ public class NestedDataExpressions
|
|||
}
|
||||
}
|
||||
|
||||
public static class ToJsonExprMacro implements ExprMacroTable.ExprMacro
|
||||
{
|
||||
public static final String NAME = "to_json";
|
||||
|
||||
@Override
|
||||
public String name()
|
||||
{
|
||||
return NAME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Expr apply(List<Expr> args)
|
||||
{
|
||||
class ToJsonExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
|
||||
{
|
||||
public ToJsonExpr(List<Expr> args)
|
||||
{
|
||||
super(name(), args);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ExprEval eval(ObjectBinding bindings)
|
||||
{
|
||||
ExprEval input = args.get(0).eval(bindings);
|
||||
return ExprEval.ofComplex(
|
||||
TYPE,
|
||||
maybeUnwrapStructuredData(input)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Expr visit(Shuttle shuttle)
|
||||
{
|
||||
List<Expr> newArgs = args.stream().map(x -> x.visit(shuttle)).collect(Collectors.toList());
|
||||
return shuttle.visit(new ToJsonExpr(newArgs));
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return TYPE;
|
||||
}
|
||||
}
|
||||
return new ToJsonExpr(args);
|
||||
}
|
||||
}
|
||||
|
||||
public static class ToJsonStringExprMacro implements ExprMacroTable.ExprMacro
|
||||
{
|
||||
public static final String NAME = "to_json_string";
|
||||
|
@ -198,7 +150,7 @@ public class NestedDataExpressions
|
|||
{
|
||||
ExprEval input = args.get(0).eval(bindings);
|
||||
try {
|
||||
final Object unwrapped = maybeUnwrapStructuredData(input);
|
||||
final Object unwrapped = unwrap(input);
|
||||
final String stringify = unwrapped == null ? null : jsonMapper.writeValueAsString(unwrapped);
|
||||
return ExprEval.ofType(
|
||||
ExpressionType.STRING,
|
||||
|
@ -262,18 +214,98 @@ public class NestedDataExpressions
|
|||
public ExprEval eval(ObjectBinding bindings)
|
||||
{
|
||||
ExprEval arg = args.get(0).eval(bindings);
|
||||
Object parsed = maybeUnwrapStructuredData(arg);
|
||||
if (arg.type().is(ExprType.STRING) && arg.value() != null && maybeJson(arg.asString())) {
|
||||
if (arg.value() == null) {
|
||||
return ExprEval.ofComplex(TYPE, null);
|
||||
}
|
||||
if (arg.type().is(ExprType.STRING)) {
|
||||
try {
|
||||
parsed = jsonMapper.readValue(arg.asString(), Object.class);
|
||||
return ExprEval.ofComplex(
|
||||
TYPE,
|
||||
jsonMapper.readValue(arg.asString(), Object.class)
|
||||
);
|
||||
}
|
||||
catch (JsonProcessingException e) {
|
||||
throw new IAE("Bad string input [%s] to [%s]", arg.asString(), name());
|
||||
}
|
||||
}
|
||||
throw new IAE(
|
||||
"Invalid input [%s] of type [%s] to [%s], expected [%s]",
|
||||
arg.asString(),
|
||||
arg.type(),
|
||||
name(),
|
||||
ExpressionType.STRING
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Expr visit(Shuttle shuttle)
|
||||
{
|
||||
List<Expr> newArgs = args.stream().map(x -> x.visit(shuttle)).collect(Collectors.toList());
|
||||
return shuttle.visit(new ParseJsonExpr(newArgs));
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return TYPE;
|
||||
}
|
||||
}
|
||||
return new ParseJsonExpr(args);
|
||||
}
|
||||
}
|
||||
|
||||
public static class TryParseJsonExprMacro implements ExprMacroTable.ExprMacro
|
||||
{
|
||||
public static final String NAME = "try_parse_json";
|
||||
|
||||
private final ObjectMapper jsonMapper;
|
||||
|
||||
@Inject
|
||||
public TryParseJsonExprMacro(
|
||||
@Json ObjectMapper jsonMapper
|
||||
)
|
||||
{
|
||||
this.jsonMapper = jsonMapper;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String name()
|
||||
{
|
||||
return NAME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Expr apply(List<Expr> args)
|
||||
{
|
||||
class ParseJsonExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
|
||||
{
|
||||
public ParseJsonExpr(List<Expr> args)
|
||||
{
|
||||
super(name(), args);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ExprEval eval(ObjectBinding bindings)
|
||||
{
|
||||
ExprEval arg = args.get(0).eval(bindings);
|
||||
if (arg.type().is(ExprType.STRING) && arg.value() != null) {
|
||||
try {
|
||||
return ExprEval.ofComplex(
|
||||
TYPE,
|
||||
jsonMapper.readValue(arg.asString(), Object.class)
|
||||
);
|
||||
}
|
||||
catch (JsonProcessingException e) {
|
||||
return ExprEval.ofComplex(
|
||||
TYPE,
|
||||
null
|
||||
);
|
||||
}
|
||||
}
|
||||
return ExprEval.ofComplex(
|
||||
TYPE,
|
||||
parsed
|
||||
null
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -323,7 +355,7 @@ public class NestedDataExpressions
|
|||
{
|
||||
ExprEval input = args.get(0).eval(bindings);
|
||||
return ExprEval.bestEffortOf(
|
||||
NestedPathFinder.findLiteral(maybeUnwrapStructuredData(input), parts)
|
||||
NestedPathFinder.findLiteral(unwrap(input), parts)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -373,7 +405,7 @@ public class NestedDataExpressions
|
|||
ExprEval input = args.get(0).eval(bindings);
|
||||
return ExprEval.ofComplex(
|
||||
TYPE,
|
||||
NestedPathFinder.find(maybeUnwrapStructuredData(input), parts)
|
||||
NestedPathFinder.find(unwrap(input), parts)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -422,7 +454,7 @@ public class NestedDataExpressions
|
|||
{
|
||||
ExprEval input = args.get(0).eval(bindings);
|
||||
return ExprEval.bestEffortOf(
|
||||
NestedPathFinder.findLiteral(maybeUnwrapStructuredData(input), parts)
|
||||
NestedPathFinder.findLiteral(unwrap(input), parts)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -479,7 +511,7 @@ public class NestedDataExpressions
|
|||
public ExprEval eval(ObjectBinding bindings)
|
||||
{
|
||||
ExprEval input = args.get(0).eval(bindings);
|
||||
StructuredDataProcessor.ProcessResults info = processor.processFields(maybeUnwrapStructuredData(input));
|
||||
StructuredDataProcessor.ProcessResults info = processor.processFields(unwrap(input));
|
||||
return ExprEval.ofType(
|
||||
ExpressionType.STRING_ARRAY,
|
||||
ImmutableList.copyOf(info.getLiteralFields())
|
||||
|
@ -539,7 +571,7 @@ public class NestedDataExpressions
|
|||
{
|
||||
ExprEval input = args.get(0).eval(bindings);
|
||||
// maybe in the future ProcessResults should deal in PathFinder.PathPart instead of strings for fields
|
||||
StructuredDataProcessor.ProcessResults info = processor.processFields(maybeUnwrapStructuredData(input));
|
||||
StructuredDataProcessor.ProcessResults info = processor.processFields(unwrap(input));
|
||||
List<String> transformed = info.getLiteralFields()
|
||||
.stream()
|
||||
.map(p -> NestedPathFinder.toNormalizedJsonPath(NestedPathFinder.parseJqPath(p)))
|
||||
|
@ -595,7 +627,7 @@ public class NestedDataExpressions
|
|||
ExprEval input = args.get(0).eval(bindings);
|
||||
return ExprEval.ofType(
|
||||
ExpressionType.STRING_ARRAY,
|
||||
NestedPathFinder.findKeys(maybeUnwrapStructuredData(input), parts)
|
||||
NestedPathFinder.findKeys(unwrap(input), parts)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -630,21 +662,17 @@ public class NestedDataExpressions
|
|||
}
|
||||
|
||||
@Nullable
|
||||
static Object maybeUnwrapStructuredData(ExprEval input)
|
||||
static Object unwrap(ExprEval input)
|
||||
{
|
||||
return maybeUnwrapStructuredData(input.value());
|
||||
return unwrap(input.value());
|
||||
}
|
||||
|
||||
static Object maybeUnwrapStructuredData(Object input)
|
||||
static Object unwrap(Object input)
|
||||
{
|
||||
if (input instanceof StructuredData) {
|
||||
StructuredData data = (StructuredData) input;
|
||||
return data.getValue();
|
||||
}
|
||||
if (input instanceof Object[]) {
|
||||
return Arrays.stream((Object[]) input).map(x -> maybeUnwrapStructuredData(x)).toArray();
|
||||
return Arrays.stream((Object[]) input).map(NestedDataExpressions::unwrap).toArray();
|
||||
}
|
||||
return input;
|
||||
return StructuredData.unwrap(input);
|
||||
}
|
||||
|
||||
|
||||
|
@ -684,15 +712,4 @@ public class NestedDataExpressions
|
|||
);
|
||||
return parts;
|
||||
}
|
||||
|
||||
static boolean maybeJson(@Nullable String val)
|
||||
{
|
||||
if (val == null) {
|
||||
return false;
|
||||
}
|
||||
if (val.isEmpty()) {
|
||||
return false;
|
||||
}
|
||||
return val.startsWith("[") || val.startsWith("{") || val.startsWith("\"") || Character.isDigit(val.charAt(0));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -36,8 +36,8 @@ public class NestedDataDimensionHandler implements DimensionHandler<StructuredDa
|
|||
{
|
||||
private static Comparator<ColumnValueSelector> COMPARATOR = (s1, s2) ->
|
||||
StructuredData.COMPARATOR.compare(
|
||||
StructuredData.possiblyWrap(s1.getObject()),
|
||||
StructuredData.possiblyWrap(s2.getObject())
|
||||
StructuredData.wrap(s1.getObject()),
|
||||
StructuredData.wrap(s2.getObject())
|
||||
);
|
||||
|
||||
private final String name;
|
||||
|
|
|
@ -102,7 +102,7 @@ public class NestedDataComplexTypeSerde extends ComplexMetricSerde
|
|||
public int compare(Object o1, Object o2)
|
||||
{
|
||||
return Comparators.<StructuredData>naturalNullsFirst()
|
||||
.compare(StructuredData.possiblyWrap(o1), StructuredData.possiblyWrap(o2));
|
||||
.compare(StructuredData.wrap(o1), StructuredData.wrap(o2));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -52,7 +52,7 @@ public class StructuredData implements Comparable<StructuredData>
|
|||
}
|
||||
|
||||
@Nullable
|
||||
public static StructuredData possiblyWrap(@Nullable Object value)
|
||||
public static StructuredData wrap(@Nullable Object value)
|
||||
{
|
||||
if (value == null || value instanceof StructuredData) {
|
||||
return (StructuredData) value;
|
||||
|
@ -60,6 +60,15 @@ public class StructuredData implements Comparable<StructuredData>
|
|||
return new StructuredData(value);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public static Object unwrap(@Nullable Object value)
|
||||
{
|
||||
if (value instanceof StructuredData) {
|
||||
return ((StructuredData) value).getValue();
|
||||
}
|
||||
return value;
|
||||
}
|
||||
|
||||
@JsonCreator
|
||||
public static StructuredData create(Object value)
|
||||
{
|
||||
|
|
|
@ -47,6 +47,7 @@ public abstract class StructuredDataProcessor
|
|||
public ProcessResults processFields(Object raw)
|
||||
{
|
||||
Queue<Field> toProcess = new ArrayDeque<>();
|
||||
raw = StructuredData.unwrap(raw);
|
||||
if (raw instanceof Map) {
|
||||
toProcess.add(new MapField("", (Map<String, ?>) raw));
|
||||
} else if (raw instanceof List) {
|
||||
|
@ -76,15 +77,16 @@ public abstract class StructuredDataProcessor
|
|||
// add estimated size of string key
|
||||
processResults.addSize(estimateStringSize(entry.getKey()));
|
||||
final String fieldName = map.getName() + ".\"" + entry.getKey() + "\"";
|
||||
Object value = StructuredData.unwrap(entry.getValue());
|
||||
// lists and maps go back in the queue
|
||||
if (entry.getValue() instanceof List) {
|
||||
List<?> theList = (List<?>) entry.getValue();
|
||||
if (value instanceof List) {
|
||||
List<?> theList = (List<?>) value;
|
||||
toProcess.add(new ListField(fieldName, theList));
|
||||
} else if (entry.getValue() instanceof Map) {
|
||||
toProcess.add(new MapField(fieldName, (Map<String, ?>) entry.getValue()));
|
||||
} else if (value instanceof Map) {
|
||||
toProcess.add(new MapField(fieldName, (Map<String, ?>) value));
|
||||
} else {
|
||||
// literals get processed
|
||||
processResults.addLiteralField(fieldName, processLiteralField(fieldName, entry.getValue()));
|
||||
processResults.addLiteralField(fieldName, processLiteralField(fieldName, value));
|
||||
}
|
||||
}
|
||||
return processResults;
|
||||
|
@ -97,7 +99,7 @@ public abstract class StructuredDataProcessor
|
|||
final List<?> theList = list.getList();
|
||||
for (int i = 0; i < theList.size(); i++) {
|
||||
final String listFieldName = list.getName() + "[" + i + "]";
|
||||
final Object element = theList.get(i);
|
||||
final Object element = StructuredData.unwrap(theList.get(i));
|
||||
// maps and lists go back into the queue
|
||||
if (element instanceof Map) {
|
||||
toProcess.add(new MapField(listFieldName, (Map<String, ?>) element));
|
||||
|
|
|
@ -615,7 +615,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn
|
|||
@Override
|
||||
public Object getObject()
|
||||
{
|
||||
StructuredData data = StructuredData.possiblyWrap(baseSelector.getObject());
|
||||
StructuredData data = StructuredData.wrap(baseSelector.getObject());
|
||||
return NestedPathFinder.findLiteral(data == null ? null : data.getValue(), parts);
|
||||
}
|
||||
|
||||
|
@ -689,8 +689,8 @@ public class NestedFieldVirtualColumn implements VirtualColumn
|
|||
@Override
|
||||
public Object getObject()
|
||||
{
|
||||
StructuredData data = StructuredData.possiblyWrap(baseSelector.getObject());
|
||||
return StructuredData.possiblyWrap(NestedPathFinder.find(data == null ? null : data.getValue(), parts));
|
||||
StructuredData data = StructuredData.wrap(baseSelector.getObject());
|
||||
return StructuredData.wrap(NestedPathFinder.find(data == null ? null : data.getValue(), parts));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -744,8 +744,8 @@ public class NestedFieldVirtualColumn implements VirtualColumn
|
|||
|
||||
private Object compute(Object input)
|
||||
{
|
||||
StructuredData data = StructuredData.possiblyWrap(input);
|
||||
return StructuredData.possiblyWrap(NestedPathFinder.find(data == null ? null : data.getValue(), parts));
|
||||
StructuredData data = StructuredData.wrap(input);
|
||||
return StructuredData.wrap(NestedPathFinder.find(data == null ? null : data.getValue(), parts));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,11 +19,13 @@
|
|||
|
||||
package org.apache.druid.query.expression;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.math.expr.Expr;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
|
@ -52,9 +54,9 @@ public class NestedDataExpressionsTest extends InitializedNullHandlingTest
|
|||
new NestedDataExpressions.JsonObjectExprMacro(),
|
||||
new NestedDataExpressions.JsonValueExprMacro(),
|
||||
new NestedDataExpressions.JsonQueryExprMacro(),
|
||||
new NestedDataExpressions.ToJsonExprMacro(),
|
||||
new NestedDataExpressions.ToJsonStringExprMacro(JSON_MAPPER),
|
||||
new NestedDataExpressions.ParseJsonExprMacro(JSON_MAPPER)
|
||||
new NestedDataExpressions.ParseJsonExprMacro(JSON_MAPPER),
|
||||
new NestedDataExpressions.TryParseJsonExprMacro(JSON_MAPPER)
|
||||
)
|
||||
);
|
||||
private static final Map<String, Object> NEST = ImmutableMap.of(
|
||||
|
@ -324,19 +326,59 @@ public class NestedDataExpressionsTest extends InitializedNullHandlingTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testToJsonToStringParseJson()
|
||||
public void testParseJsonTryParseJson() throws JsonProcessingException
|
||||
{
|
||||
Expr expr = Parser.parse("to_json(long)", MACRO_TABLE);
|
||||
Expr expr = Parser.parse("parse_json(null)", MACRO_TABLE);
|
||||
ExprEval eval = expr.eval(inputBindings);
|
||||
Assert.assertEquals(1234L, eval.value());
|
||||
Assert.assertEquals(null, eval.value());
|
||||
Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
|
||||
|
||||
expr = Parser.parse("to_json_string(nest)", MACRO_TABLE);
|
||||
expr = Parser.parse("parse_json('null')", MACRO_TABLE);
|
||||
eval = expr.eval(inputBindings);
|
||||
Assert.assertEquals(null, eval.value());
|
||||
Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
|
||||
|
||||
Assert.assertThrows(IAE.class, () -> Parser.parse("parse_json('{')", MACRO_TABLE));
|
||||
expr = Parser.parse("try_parse_json('{')", MACRO_TABLE);
|
||||
eval = expr.eval(inputBindings);
|
||||
Assert.assertEquals(null, eval.value());
|
||||
Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
|
||||
|
||||
Assert.assertThrows(IAE.class, () -> Parser.parse("parse_json('hello world')", MACRO_TABLE));
|
||||
expr = Parser.parse("try_parse_json('hello world')", MACRO_TABLE);
|
||||
eval = expr.eval(inputBindings);
|
||||
Assert.assertEquals(null, eval.value());
|
||||
Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
|
||||
|
||||
expr = Parser.parse("parse_json('\"hello world\"')", MACRO_TABLE);
|
||||
eval = expr.eval(inputBindings);
|
||||
Assert.assertEquals("hello world", eval.value());
|
||||
Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
|
||||
|
||||
expr = Parser.parse("parse_json('1')", MACRO_TABLE);
|
||||
eval = expr.eval(inputBindings);
|
||||
Assert.assertEquals(1, eval.value());
|
||||
Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
|
||||
|
||||
expr = Parser.parse("parse_json('true')", MACRO_TABLE);
|
||||
eval = expr.eval(inputBindings);
|
||||
Assert.assertEquals(true, eval.value());
|
||||
Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
|
||||
|
||||
expr = Parser.parse("parse_json('{\"foo\":1}')", MACRO_TABLE);
|
||||
eval = expr.eval(inputBindings);
|
||||
Assert.assertEquals("{\"foo\":1}", JSON_MAPPER.writeValueAsString(eval.value()));
|
||||
Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testToJsonStringParseJson()
|
||||
{
|
||||
Expr expr = Parser.parse("to_json_string(nest)", MACRO_TABLE);
|
||||
ExprEval eval = expr.eval(inputBindings);
|
||||
Assert.assertEquals("{\"x\":100,\"y\":200,\"z\":300}", eval.value());
|
||||
Assert.assertEquals(ExpressionType.STRING, eval.type());
|
||||
|
||||
|
||||
expr = Parser.parse("parse_json(to_json_string(nest))", MACRO_TABLE);
|
||||
eval = expr.eval(inputBindings);
|
||||
// round trip ends up as integers initially...
|
||||
|
|
|
@ -62,9 +62,9 @@ public class TestExprMacroTable extends ExprMacroTable
|
|||
new NestedDataExpressions.JsonPathsExprMacro(),
|
||||
new NestedDataExpressions.JsonValueExprMacro(),
|
||||
new NestedDataExpressions.JsonQueryExprMacro(),
|
||||
new NestedDataExpressions.ToJsonExprMacro(),
|
||||
new NestedDataExpressions.ToJsonStringExprMacro(jsonMapper),
|
||||
new NestedDataExpressions.ParseJsonExprMacro(jsonMapper)
|
||||
new NestedDataExpressions.ParseJsonExprMacro(jsonMapper),
|
||||
new NestedDataExpressions.TryParseJsonExprMacro(jsonMapper)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
2021-01-01 hello {"x":100,"y":200,"z":300} {"x":["a","b","c"],"y":{"a":"a","b":"b","c":[1,2,3]}} {"a":["hello","world"],"b":{"x":"hello","y":"world"}} [{"x":5,"y":10},{"x":15,"y":22}]
|
||||
2021-01-01 hello {"x":["x","y","z"]} [{"x":35,"y":310},{"x":315,"y":322}]
|
||||
2021-01-01 hello {"x":300,"y":800} hello
|
||||
2021-01-01 hello {"x":300,"y":800} "hello"
|
||||
2021-01-01 hello {"y":500} [{"x":115,"y":410},{"x":415,"y":422}]
|
||||
2021-01-02 hello {"x":200,"y":100,"z":101} {"x":["x","y","z"],"y":{"a":"b","b":"c","c":[4,5,6]}} {"b":["hello","world"],"c":{"x":["hello"],"y":"world"}}
|
||||
2021-01-02 hello {"x":["x","y","z"]}
|
||||
|
|
Can't render this file because it contains an unexpected character in line 1 and column 19.
|
|
@ -79,9 +79,9 @@ public class ExpressionModule implements Module
|
|||
.add(NestedDataExpressions.JsonPathsExprMacro.class)
|
||||
.add(NestedDataExpressions.JsonValueExprMacro.class)
|
||||
.add(NestedDataExpressions.JsonQueryExprMacro.class)
|
||||
.add(NestedDataExpressions.ToJsonExprMacro.class)
|
||||
.add(NestedDataExpressions.ToJsonStringExprMacro.class)
|
||||
.add(NestedDataExpressions.ParseJsonExprMacro.class)
|
||||
.add(NestedDataExpressions.TryParseJsonExprMacro.class)
|
||||
.build();
|
||||
|
||||
@Override
|
||||
|
|
|
@ -475,44 +475,6 @@ public class NestedDataOperatorConversions
|
|||
}
|
||||
}
|
||||
|
||||
public static class ToJsonOperatorConversion implements SqlOperatorConversion
|
||||
{
|
||||
private static final String FUNCTION_NAME = "to_json";
|
||||
private static final SqlFunction SQL_FUNCTION = OperatorConversions
|
||||
.operatorBuilder(StringUtils.toUpperCase(FUNCTION_NAME))
|
||||
.operandTypes(SqlTypeFamily.ANY)
|
||||
.returnTypeInference(NESTED_RETURN_TYPE_INFERENCE)
|
||||
.functionCategory(SqlFunctionCategory.USER_DEFINED_FUNCTION)
|
||||
.build();
|
||||
|
||||
|
||||
@Override
|
||||
public SqlOperator calciteOperator()
|
||||
{
|
||||
return SQL_FUNCTION;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public DruidExpression toDruidExpression(
|
||||
PlannerContext plannerContext,
|
||||
RowSignature rowSignature,
|
||||
RexNode rexNode
|
||||
)
|
||||
{
|
||||
return OperatorConversions.convertCall(
|
||||
plannerContext,
|
||||
rowSignature,
|
||||
rexNode,
|
||||
druidExpressions -> DruidExpression.ofExpression(
|
||||
NestedDataComplexTypeSerde.TYPE,
|
||||
DruidExpression.functionCall("to_json"),
|
||||
druidExpressions
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
public static class ToJsonStringOperatorConversion implements SqlOperatorConversion
|
||||
{
|
||||
private static final String FUNCTION_NAME = "to_json_string";
|
||||
|
@ -556,7 +518,7 @@ public class NestedDataOperatorConversions
|
|||
private static final String FUNCTION_NAME = "parse_json";
|
||||
private static final SqlFunction SQL_FUNCTION = OperatorConversions
|
||||
.operatorBuilder(StringUtils.toUpperCase(FUNCTION_NAME))
|
||||
.operandTypes(SqlTypeFamily.ANY)
|
||||
.operandTypes(SqlTypeFamily.STRING)
|
||||
.returnTypeInference(NESTED_RETURN_TYPE_INFERENCE)
|
||||
.functionCategory(SqlFunctionCategory.USER_DEFINED_FUNCTION)
|
||||
.build();
|
||||
|
@ -588,4 +550,42 @@ public class NestedDataOperatorConversions
|
|||
);
|
||||
}
|
||||
}
|
||||
|
||||
public static class TryParseJsonOperatorConversion implements SqlOperatorConversion
|
||||
{
|
||||
private static final String FUNCTION_NAME = "try_parse_json";
|
||||
private static final SqlFunction SQL_FUNCTION = OperatorConversions
|
||||
.operatorBuilder(StringUtils.toUpperCase(FUNCTION_NAME))
|
||||
.operandTypes(SqlTypeFamily.STRING)
|
||||
.returnTypeInference(NESTED_RETURN_TYPE_INFERENCE)
|
||||
.functionCategory(SqlFunctionCategory.USER_DEFINED_FUNCTION)
|
||||
.build();
|
||||
|
||||
|
||||
@Override
|
||||
public SqlOperator calciteOperator()
|
||||
{
|
||||
return SQL_FUNCTION;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public DruidExpression toDruidExpression(
|
||||
PlannerContext plannerContext,
|
||||
RowSignature rowSignature,
|
||||
RexNode rexNode
|
||||
)
|
||||
{
|
||||
return OperatorConversions.convertCall(
|
||||
plannerContext,
|
||||
rowSignature,
|
||||
rexNode,
|
||||
druidExpressions -> DruidExpression.ofExpression(
|
||||
NestedDataComplexTypeSerde.TYPE,
|
||||
DruidExpression.functionCall("try_parse_json"),
|
||||
druidExpressions
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -307,9 +307,9 @@ public class DruidOperatorTable implements SqlOperatorTable
|
|||
.add(new NestedDataOperatorConversions.JsonValueOperatorConversion())
|
||||
.add(new NestedDataOperatorConversions.JsonValueAnyOperatorConversion())
|
||||
.add(new NestedDataOperatorConversions.JsonObjectOperatorConversion())
|
||||
.add(new NestedDataOperatorConversions.ToJsonOperatorConversion())
|
||||
.add(new NestedDataOperatorConversions.ToJsonStringOperatorConversion())
|
||||
.add(new NestedDataOperatorConversions.ParseJsonOperatorConversion())
|
||||
.add(new NestedDataOperatorConversions.TryParseJsonOperatorConversion())
|
||||
.build();
|
||||
|
||||
private static final List<SqlOperatorConversion> STANDARD_OPERATOR_CONVERSIONS =
|
||||
|
|
|
@ -2362,7 +2362,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
public void testToJsonAndParseJson()
|
||||
{
|
||||
testQuery(
|
||||
"SELECT string, TO_JSON(string), PARSE_JSON(string), PARSE_JSON('{\"foo\":1}'), PARSE_JSON(TO_JSON_STRING(nester))\n"
|
||||
"SELECT string, TRY_PARSE_JSON(TO_JSON_STRING(string)), PARSE_JSON('{\"foo\":1}'), PARSE_JSON(TO_JSON_STRING(nester))\n"
|
||||
+ "FROM druid.nested",
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
|
@ -2371,30 +2371,24 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
.virtualColumns(
|
||||
new ExpressionVirtualColumn(
|
||||
"v0",
|
||||
"to_json(\"string\")",
|
||||
"try_parse_json(to_json_string(\"string\"))",
|
||||
NestedDataComplexTypeSerde.TYPE,
|
||||
macroTable
|
||||
),
|
||||
new ExpressionVirtualColumn(
|
||||
"v1",
|
||||
"parse_json(\"string\")",
|
||||
NestedDataComplexTypeSerde.TYPE,
|
||||
macroTable
|
||||
),
|
||||
new ExpressionVirtualColumn(
|
||||
"v2",
|
||||
"parse_json('{\\u0022foo\\u0022:1}')",
|
||||
NestedDataComplexTypeSerde.TYPE,
|
||||
macroTable
|
||||
),
|
||||
new ExpressionVirtualColumn(
|
||||
"v3",
|
||||
"v2",
|
||||
"parse_json(to_json_string(\"nester\"))",
|
||||
NestedDataComplexTypeSerde.TYPE,
|
||||
macroTable
|
||||
)
|
||||
)
|
||||
.columns("string", "v0", "v1", "v2", "v3")
|
||||
.columns("string", "v0", "v1", "v2")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.build()
|
||||
|
@ -2403,23 +2397,21 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
new Object[]{
|
||||
"aaa",
|
||||
"\"aaa\"",
|
||||
"\"aaa\"",
|
||||
"{\"foo\":1}",
|
||||
"{\"array\":[\"a\",\"b\"],\"n\":{\"x\":\"hello\"}}"
|
||||
},
|
||||
new Object[]{"bbb", "\"bbb\"", "\"bbb\"", "{\"foo\":1}", "\"hello\""},
|
||||
new Object[]{"ccc", "\"ccc\"", "\"ccc\"", "{\"foo\":1}", null},
|
||||
new Object[]{"ddd", "\"ddd\"", "\"ddd\"", "{\"foo\":1}", null},
|
||||
new Object[]{"eee", "\"eee\"", "\"eee\"", "{\"foo\":1}", null},
|
||||
new Object[]{"aaa", "\"aaa\"", "\"aaa\"", "{\"foo\":1}", "{\"array\":[\"a\",\"b\"],\"n\":{\"x\":1}}"},
|
||||
new Object[]{"ddd", "\"ddd\"", "\"ddd\"", "{\"foo\":1}", "2"}
|
||||
new Object[]{"bbb", "\"bbb\"", "{\"foo\":1}", "\"hello\""},
|
||||
new Object[]{"ccc", "\"ccc\"", "{\"foo\":1}", null},
|
||||
new Object[]{"ddd", "\"ddd\"", "{\"foo\":1}", null},
|
||||
new Object[]{"eee", "\"eee\"", "{\"foo\":1}", null},
|
||||
new Object[]{"aaa", "\"aaa\"", "{\"foo\":1}", "{\"array\":[\"a\",\"b\"],\"n\":{\"x\":1}}"},
|
||||
new Object[]{"ddd", "\"ddd\"", "{\"foo\":1}", "2"}
|
||||
),
|
||||
RowSignature.builder()
|
||||
.add("string", ColumnType.STRING)
|
||||
.add("EXPR$1", NestedDataComplexTypeSerde.TYPE)
|
||||
.add("EXPR$2", NestedDataComplexTypeSerde.TYPE)
|
||||
.add("EXPR$3", NestedDataComplexTypeSerde.TYPE)
|
||||
.add("EXPR$4", NestedDataComplexTypeSerde.TYPE)
|
||||
.build()
|
||||
);
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue