http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalAggFuncHolder.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalAggFuncHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalAggFuncHolder.java new file mode 100644 index 0000000..a5de0cd --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalAggFuncHolder.java @@ -0,0 +1,46 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.drill.exec.expr.fn; + +import org.apache.drill.common.expression.LogicalExpression; +import org.apache.drill.common.types.TypeProtos; +import org.apache.drill.exec.expr.annotations.FunctionTemplate; + +import java.util.List; +import java.util.Map; + +public class DrillDecimalAggFuncHolder extends DrillAggFuncHolder { + public DrillDecimalAggFuncHolder(FunctionTemplate.FunctionScope scope, FunctionTemplate.NullHandling nullHandling, boolean isBinaryCommutative, boolean isRandom, String[] registeredNames, ValueReference[] parameters, ValueReference returnValue, WorkspaceReference[] workspaceVars, Map<String, String> methods, List<String> imports) { + super(scope, nullHandling, isBinaryCommutative, isRandom, registeredNames, parameters, returnValue, workspaceVars, methods, imports); + } + + @Override + public TypeProtos.MajorType getReturnType(List<LogicalExpression> args) { + + int scale = 0; + int precision = 0; + + // Get the max scale and precision from the inputs + for (LogicalExpression e : args) { + scale = Math.max(scale, e.getMajorType().getScale()); + precision = Math.max(precision, e.getMajorType().getPrecision()); + } + + return (TypeProtos.MajorType.newBuilder().setMinorType(returnValue.type.getMinorType()).setScale(scale).setPrecision(precision).setMode(TypeProtos.DataMode.REQUIRED).build()); + } +}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalDivScaleFuncHolder.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalDivScaleFuncHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalDivScaleFuncHolder.java new file mode 100644 index 0000000..af239e4 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalDivScaleFuncHolder.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.drill.exec.expr.fn; + +import java.util.List; +import java.util.Map; + +import org.apache.drill.common.expression.LogicalExpression; +import org.apache.drill.common.types.TypeProtos; +import org.apache.drill.common.types.TypeProtos.MajorType; + +import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope; +import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling; + +public class DrillDecimalDivScaleFuncHolder extends DrillSimpleFuncHolder{ + + + public DrillDecimalDivScaleFuncHolder(FunctionScope scope, NullHandling nullHandling, boolean isBinaryCommutative, boolean isRandom, + String[] registeredNames, ValueReference[] parameters, ValueReference returnValue, WorkspaceReference[] workspaceVars, + Map<String, String> methods, List<String> imports) { + super(scope, nullHandling, isBinaryCommutative, isRandom, registeredNames, parameters, returnValue, workspaceVars, methods, imports); + } + + @Override + public MajorType getReturnType(List<LogicalExpression> args) { + + TypeProtos.DataMode mode = returnValue.type.getMode(); + int scale = 0; + int precision = 0; + + if (nullHandling == NullHandling.NULL_IF_NULL) { + // if any one of the input types is nullable, then return nullable return type + for (LogicalExpression e : args) { + if (e.getMajorType().getMode() == TypeProtos.DataMode.OPTIONAL) { + mode = TypeProtos.DataMode.OPTIONAL; + break; + } + } + } + + /* Set the scale to be the same as the fist input's scale + * Used by divide and modulo functions + */ + scale = args.get(0).getMajorType().getScale(); + precision = args.get(0).getMajorType().getPrecision(); + + return (TypeProtos.MajorType.newBuilder().setMinorType(returnValue.type.getMinorType()).setScale(scale).setPrecision(precision).setMode(mode).build()); + } +} http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalSetScaleFuncHolder.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalSetScaleFuncHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalSetScaleFuncHolder.java new file mode 100644 index 0000000..b7df04f --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalSetScaleFuncHolder.java @@ -0,0 +1,69 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.drill.exec.expr.fn; + +import java.util.List; +import java.util.Map; + +import org.apache.drill.common.expression.LogicalExpression; +import org.apache.drill.common.expression.ValueExpressions; +import org.apache.drill.common.types.TypeProtos; +import org.apache.drill.common.types.TypeProtos.MajorType; + +import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope; +import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling; + +public class DrillDecimalSetScaleFuncHolder extends DrillSimpleFuncHolder{ + + + public DrillDecimalSetScaleFuncHolder(FunctionScope scope, NullHandling nullHandling, boolean isBinaryCommutative, boolean isRandom, + String[] registeredNames, ValueReference[] parameters, ValueReference returnValue, WorkspaceReference[] workspaceVars, + Map<String, String> methods, List<String> imports) { + super(scope, nullHandling, isBinaryCommutative, isRandom, registeredNames, parameters, returnValue, workspaceVars, methods, imports); + } + + @Override + public MajorType getReturnType(List<LogicalExpression> args) { + + TypeProtos.DataMode mode = returnValue.type.getMode(); + int scale = 0; + int precision = 0; + int i = 0; + + if (nullHandling == NullHandling.NULL_IF_NULL) { + // if any one of the input types is nullable, then return nullable return type + for (LogicalExpression e : args) { + + precision = Math.max(precision, e.getMajorType().getPrecision()); + if (e.getMajorType().getMode() == TypeProtos.DataMode.OPTIONAL) { + mode = TypeProtos.DataMode.OPTIONAL; + } + } + + /* Used by functions like round, truncate which specify the scale for + * the output as the second argument + */ + assert (args.size() == 2) && (args.get(1) instanceof ValueExpressions.IntExpression); + + // Get the scale from the second argument which should be a constant + scale = ((ValueExpressions.IntExpression) args.get(1)).getInt(); + } + + return (TypeProtos.MajorType.newBuilder().setMinorType(returnValue.type.getMinorType()).setScale(scale).setPrecision(precision).setMode(mode).build()); + } +} http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalZeroScaleFuncHolder.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalZeroScaleFuncHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalZeroScaleFuncHolder.java new file mode 100644 index 0000000..2ee60bf --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalZeroScaleFuncHolder.java @@ -0,0 +1,60 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.drill.exec.expr.fn; + +import java.util.List; +import java.util.Map; + +import org.apache.drill.common.expression.LogicalExpression; +import org.apache.drill.common.types.TypeProtos; +import org.apache.drill.common.types.TypeProtos.MajorType; + +import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope; +import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling; + +public class DrillDecimalZeroScaleFuncHolder extends DrillSimpleFuncHolder{ + + + public DrillDecimalZeroScaleFuncHolder(FunctionScope scope, NullHandling nullHandling, boolean isBinaryCommutative, boolean isRandom, + String[] registeredNames, ValueReference[] parameters, ValueReference returnValue, WorkspaceReference[] workspaceVars, + Map<String, String> methods, List<String> imports) { + super(scope, nullHandling, isBinaryCommutative, isRandom, registeredNames, parameters, returnValue, workspaceVars, methods, imports); + } + + /* This function scope is used when we need to remove the scale part. + * trunc and round functions with single argument use this + */ + @Override + public MajorType getReturnType(List<LogicalExpression> args) { + + int precision = 0; + TypeProtos.DataMode mode = returnValue.type.getMode(); + + if (nullHandling == NullHandling.NULL_IF_NULL) { + // if any one of the input types is nullable, then return nullable return type + for (LogicalExpression e : args) { + if (e.getMajorType().getMode() == TypeProtos.DataMode.OPTIONAL) { + mode = TypeProtos.DataMode.OPTIONAL; + } + precision = Math.max(precision, e.getMajorType().getPrecision()); + } + } + + return (TypeProtos.MajorType.newBuilder().setMinorType(returnValue.type.getMinorType()).setScale(0).setPrecision(precision).setMode(mode).build()); + } +} http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java index 888829d..4421091 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java @@ -206,6 +206,9 @@ public class FunctionConverter { case POINT_AGGREGATE: return new DrillAggFuncHolder(template.scope(), template.nulls(), template.isBinaryCommutative(), template.isRandom(), registeredNames, ps, outputField, works, methods, imports); + case DECIMAL_AGGREGATE: + return new DrillDecimalAggFuncHolder(template.scope(), template.nulls(), template.isBinaryCommutative(), + template.isRandom(), registeredNames, ps, outputField, works, methods, imports); case SIMPLE: return new DrillSimpleFuncHolder(template.scope(), template.nulls(), template.isBinaryCommutative(), template.isRandom(), registeredNames, ps, outputField, works, methods, imports); @@ -218,6 +221,15 @@ public class FunctionConverter { case DECIMAL_CAST: return new DrillDecimalCastFuncHolder(template.scope(), template.nulls(), template.isBinaryCommutative(), template.isRandom(), registeredNames, ps, outputField, works, methods, imports); + case DECIMAL_DIV_SCALE: + return new DrillDecimalDivScaleFuncHolder(template.scope(), template.nulls(), template.isBinaryCommutative(), + template.isRandom(), registeredNames, ps, outputField, works, methods, imports); + case DECIMAL_SET_SCALE: + return new DrillDecimalSetScaleFuncHolder(template.scope(), template.nulls(), template.isBinaryCommutative(), + template.isRandom(), registeredNames, ps, outputField, works, methods, imports); + case DECIMAL_ZERO_SCALE: + return new DrillDecimalZeroScaleFuncHolder(template.scope(), template.nulls(), template.isBinaryCommutative(), + template.isRandom(), registeredNames, ps, outputField, works, methods, imports); case HOLISTIC_AGGREGATE: case RANGE_AGGREGATE: default: http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java index 59be472..7d5a3a6 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java @@ -23,12 +23,18 @@ import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope; import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling; import org.apache.drill.exec.expr.annotations.Output; import org.apache.drill.exec.expr.annotations.Param; +import org.apache.drill.exec.expr.annotations.Workspace; import org.apache.drill.exec.expr.holders.BigIntHolder; import org.apache.drill.exec.expr.holders.Float4Holder; import org.apache.drill.exec.expr.holders.Float8Holder; import org.apache.drill.exec.expr.holders.IntHolder; +import org.apache.drill.exec.expr.holders.VarCharHolder; import org.apache.drill.exec.record.RecordBatch; +import java.math.BigDecimal; +import java.text.DecimalFormat; +import java.text.ParseException; + public class MathFunctions{ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MathFunctions.class); @@ -74,6 +80,36 @@ public class MathFunctions{ out.value = java.lang.Math.random(); } - } - + } + + @FunctionTemplate(name = "to_number", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL) + public static class ToNumber implements DrillSimpleFunc { + @Param VarCharHolder left; + @Param VarCharHolder right; + @Workspace java.text.DecimalFormat inputFormat; + @Workspace int decimalDigits; + @Output Float8Holder out; + + public void setup(RecordBatch b) { + byte[] buf = new byte[right.end - right.start]; + right.buffer.getBytes(right.start, buf, 0, right.end - right.start); + inputFormat = new DecimalFormat(new String(buf)); + decimalDigits = inputFormat.getMaximumFractionDigits(); + } + + public void eval() { + byte[] buf1 = new byte[left.end - left.start]; + left.buffer.getBytes(left.start, buf1, 0, left.end - left.start); + String input = new String(buf1); + try { + out.value = inputFormat.parse(input).doubleValue(); + } catch(java.text.ParseException e) { + throw new UnsupportedOperationException("Cannot parse input: " + input + " with pattern : " + inputFormat.toPattern()); + } + + // Round the value + java.math.BigDecimal roundedValue = new java.math.BigDecimal(out.value); + out.value = (roundedValue.setScale(decimalDigits, java.math.BigDecimal.ROUND_HALF_UP)).doubleValue(); + } + } } http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java index aca5933..cebe491 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java @@ -31,6 +31,7 @@ import org.apache.drill.exec.expr.holders.BigIntHolder; import org.apache.drill.exec.expr.holders.BitHolder; import org.apache.drill.exec.expr.holders.VarBinaryHolder; import org.apache.drill.exec.expr.holders.VarCharHolder; +import org.apache.drill.exec.expr.holders.NullableVarCharHolder; import org.apache.drill.exec.record.RecordBatch; public class StringFunctions{ @@ -115,7 +116,19 @@ public class StringFunctions{ public void eval(){ out.value = org.apache.drill.exec.expr.fn.impl.StringFunctionUtil.getUTF8CharLength(input.buffer, input.start, input.end); } - + } + + @FunctionTemplate(name = "lengthUtf8", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL) + public static class ByteLength implements DrillSimpleFunc{ + + @Param VarBinaryHolder input; + @Output BigIntHolder out; + + public void setup(RecordBatch incoming){} + + public void eval(){ + out.value = org.apache.drill.exec.expr.fn.impl.StringFunctionUtil.getUTF8CharLength(input.buffer, input.start, input.end); + } } @FunctionTemplate(name = "octet_length", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL) @@ -781,6 +794,92 @@ public class StringFunctions{ } + @FunctionTemplate(name = "concat", scope = FunctionScope.SIMPLE, nulls = NullHandling.INTERNAL) + public static class ConcatRightNullInput implements DrillSimpleFunc{ + + @Param VarCharHolder left; + @Param NullableVarCharHolder right; + @Output VarCharHolder out; + @Workspace ByteBuf buffer; + + + public void setup(RecordBatch incoming){ + buffer = io.netty.buffer.Unpooled.wrappedBuffer(new byte [8000]); + } + + public void eval(){ + out.buffer = buffer; + out.start = out.end = 0; + + int id = 0; + for (id = left.start; id < left.end; id++) + out.buffer.setByte(out.end++, left.buffer.getByte(id)); + + if (right.isSet == 1) { + for (id = right.start; id < right.end; id++) + out.buffer.setByte(out.end++, right.buffer.getByte(id)); + } + } + } + + @FunctionTemplate(name = "concat", scope = FunctionScope.SIMPLE, nulls = NullHandling.INTERNAL) + public static class ConcatLeftNullInput implements DrillSimpleFunc{ + + @Param NullableVarCharHolder left; + @Param VarCharHolder right; + @Output VarCharHolder out; + @Workspace ByteBuf buffer; + + + public void setup(RecordBatch incoming){ + buffer = io.netty.buffer.Unpooled.wrappedBuffer(new byte [8000]); + } + + public void eval(){ + out.buffer = buffer; + out.start = out.end = 0; + + int id = 0; + if (left.isSet == 1) { + for (id = left.start; id < left.end; id++) + out.buffer.setByte(out.end++, left.buffer.getByte(id)); + } + + for (id = right.start; id < right.end; id++) + out.buffer.setByte(out.end++, right.buffer.getByte(id)); + } + } + + @FunctionTemplate(name = "concat", scope = FunctionScope.SIMPLE, nulls = NullHandling.INTERNAL) + public static class ConcatBothNullInput implements DrillSimpleFunc{ + + @Param NullableVarCharHolder left; + @Param NullableVarCharHolder right; + @Output VarCharHolder out; + @Workspace ByteBuf buffer; + + + public void setup(RecordBatch incoming){ + buffer = io.netty.buffer.Unpooled.wrappedBuffer(new byte [8000]); + } + + public void eval(){ + out.buffer = buffer; + out.start = out.end = 0; + + int id = 0; + if (left.isSet == 1) { + for (id = left.start; id < left.end; id++) + out.buffer.setByte(out.end++, left.buffer.getByte(id)); + } + + if (right.isSet == 1) { + for (id = right.start; id < right.end; id++) + out.buffer.setByte(out.end++, right.buffer.getByte(id)); + } + } + } + // Converts a hex encoded string into a varbinary type. // "\xca\xfe\xba\xbe" => (byte[]) {(byte)0xca, (byte)0xfe, (byte)0xba, (byte)0xbe} @FunctionTemplate(name = "binary_string", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL) http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java index aee6da4..73e835a 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java @@ -257,13 +257,14 @@ public class DrillOptiq { } - private LogicalExpression getDrillFunctionFromOptiqCall(RexCall call){ + private LogicalExpression getDrillFunctionFromOptiqCall(RexCall call) { List<LogicalExpression> args = Lists.newArrayList(); for(RexNode n : call.getOperands()){ args.add(n.accept(this)); } String functionName = call.getOperator().getName().toLowerCase(); + // TODO: once we have more function rewrites and a patter emerges from different rewrites, factor this out in a better fashion /* Rewrite extract functions in the following manner * extract(year, date '2008-2-23') ---> extractYear(date '2008-2-23') */ @@ -288,11 +289,8 @@ public class DrillOptiq { default: throw new UnsupportedOperationException("extract function supports the following time units: YEAR, MONTH, DAY, HOUR, MINUTE, SECOND"); } - } - - // Rewrite DATE_PART functions as extract functions - if (call.getOperator().getName().equalsIgnoreCase("DATE_PART")) { - + } else if (functionName.equals("date_part")) { + // Rewrite DATE_PART functions as extract functions // assert that the function has exactly two arguments assert args.size() == 2; @@ -305,9 +303,39 @@ public class DrillOptiq { QuotedString extractString = (QuotedString) args.get(0); String functionPostfix = extractString.value.substring(0, 1).toUpperCase() + extractString.value.substring(1).toLowerCase(); return FunctionCallFactory.createExpression("extract" + functionPostfix, args.subList(1, 2)); + } else if (functionName.equals("concat")) { + + // Cast arguments to VARCHAR + List<LogicalExpression> concatArgs = Lists.newArrayList(); + MajorType castType = Types.required(MinorType.VARCHAR).toBuilder().setWidth(64000).build(); + concatArgs.add(FunctionCallFactory.createCast(castType, ExpressionPosition.UNKNOWN, args.get(0))); + concatArgs.add(FunctionCallFactory.createCast(castType, ExpressionPosition.UNKNOWN, args.get(1))); + + LogicalExpression first = FunctionCallFactory.createExpression(functionName, concatArgs); + + for (int i = 2; i < args.size(); i++) { + concatArgs = Lists.newArrayList(); + concatArgs.add(first); + concatArgs.add(FunctionCallFactory.createCast(castType, ExpressionPosition.UNKNOWN, args.get(i))); + first = FunctionCallFactory.createExpression(functionName, concatArgs); + } + + return first; + } else if (functionName.equals("length")) { + + if (args.size() == 2) { + + // Second argument should always be a literal specifying the encoding format + assert args.get(1) instanceof ValueExpressions.QuotedString; + + String encodingType = ((ValueExpressions.QuotedString) args.get(1)).value; + functionName += encodingType.substring(0, 1).toUpperCase() + encodingType.substring(1).toLowerCase(); + + return FunctionCallFactory.createExpression(functionName, args.subList(0, 1)); + } } - return FunctionCallFactory.createExpression(call.getOperator().getName().toLowerCase(), args); + return FunctionCallFactory.createExpression(functionName, args); } @Override http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/ResolverTypePrecedence.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/ResolverTypePrecedence.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/ResolverTypePrecedence.java index 71bf616..9d83941 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/ResolverTypePrecedence.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/ResolverTypePrecedence.java @@ -63,13 +63,13 @@ public class ResolverTypePrecedence { precedenceMap.put(MinorType.UINT8, i += 2); precedenceMap.put(MinorType.MONEY, i += 2); precedenceMap.put(MinorType.FLOAT4, i += 2); - precedenceMap.put(MinorType.FLOAT8, i += 2); - precedenceMap.put(MinorType.DECIMAL9, i += 2); - precedenceMap.put(MinorType.DECIMAL18, i += 2); - precedenceMap.put(MinorType.DECIMAL28DENSE, i += 2); - precedenceMap.put(MinorType.DECIMAL28SPARSE, i += 2); + precedenceMap.put(MinorType.DECIMAL9, i += 2); + precedenceMap.put(MinorType.DECIMAL18, i += 2); + precedenceMap.put(MinorType.DECIMAL28DENSE, i += 2); + precedenceMap.put(MinorType.DECIMAL28SPARSE, i += 2); precedenceMap.put(MinorType.DECIMAL38DENSE, i += 2); precedenceMap.put(MinorType.DECIMAL38SPARSE, i += 2); + precedenceMap.put(MinorType.FLOAT8, i += 2); precedenceMap.put(MinorType.TIME, i += 2); precedenceMap.put(MinorType.DATE, i += 2); precedenceMap.put(MinorType.TIMESTAMP, i += 2); http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestAggregateFunctionsQuery.java ---------------------------------------------------------------------- diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestAggregateFunctionsQuery.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestAggregateFunctionsQuery.java new file mode 100644 index 0000000..10c36cb --- /dev/null +++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestAggregateFunctionsQuery.java @@ -0,0 +1,98 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.drill.jdbc.test; + +import java.nio.file.Paths; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.Statement; +import java.util.concurrent.TimeUnit; + +import org.apache.drill.common.util.TestTools; +import org.apache.drill.exec.store.hive.HiveTestDataGenerator; +import org.apache.drill.jdbc.Driver; +import org.apache.drill.jdbc.JdbcTest; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestRule; + +import com.google.common.base.Function; +import com.google.common.base.Stopwatch; + +public class TestAggregateFunctionsQuery { + + public static final String WORKING_PATH; + static{ + Driver.load(); + WORKING_PATH = Paths.get("").toAbsolutePath().toString(); + + } + @Test + public void testDateAggFunction() throws Exception{ + String query = new String("SELECT max(cast(HIRE_DATE as date)) as MAX_DATE, min(cast(HIRE_DATE as date)) as MIN_DATE" + + " FROM `employee.json`"); + + JdbcAssert.withFull("cp") + .sql(query) + .returns( + "MAX_DATE=1998-01-01; " + + "MIN_DATE=1993-05-01\n" + ); + } + + @Test + public void testIntervalAggFunction() throws Exception{ + String query = new String("select max(date_diff(date'2014-5-2', cast(HIRE_DATE as date))) as MAX_DAYS, min(date_diff(date'2014-5-2', cast(HIRE_DATE as date))) MIN_DAYS" + + " FROM `employee.json`"); + + JdbcAssert.withFull("cp") + .sql(query) + .returns( + "MAX_DAYS=7671 days 0:0:0.0; " + + "MIN_DAYS=5965 days 0:0:0.0\n" + ); + } + + @Test + public void testDecimalAggFunction() throws Exception{ + String query = new String("SELECT " + + "max(cast(EMPLOYEE_ID as decimal(9, 2))) as MAX_DEC9, min(cast(EMPLOYEE_ID as decimal(9, 2))) as MIN_DEC9," + + "max(cast(EMPLOYEE_ID as decimal(18, 4))) as MAX_DEC18, min(cast(EMPLOYEE_ID as decimal(18, 4))) as MIN_DEC18," + + "max(cast(EMPLOYEE_ID as decimal(28, 9))) as MAX_DEC28, min(cast(EMPLOYEE_ID as decimal(28, 9))) as MIN_DEC28," + + "max(cast(EMPLOYEE_ID as decimal(38, 11))) as MAX_DEC38, min(cast(EMPLOYEE_ID as decimal(38, 11))) as MIN_DEC38" + + " FROM `employee.json`"); + + JdbcAssert.withFull("cp") + .sql(query) + .returns( + "MAX_DEC9=1156.00; " + + "MIN_DEC9=1.00; " + + "MAX_DEC18=1156.0000; " + + "MIN_DEC18=1.0000; " + + "MAX_DEC28=1156.000000000; " + + "MIN_DEC28=1.000000000; " + + "MAX_DEC38=1156.00000000000; " + + "MIN_DEC38=1.00000000000\n " + ); + } +} http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestDateAggregateFunction.java ---------------------------------------------------------------------- diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestDateAggregateFunction.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestDateAggregateFunction.java deleted file mode 100644 index 05c3fae..0000000 --- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestDateAggregateFunction.java +++ /dev/null @@ -1,75 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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 org.apache.drill.jdbc.test; - -import java.nio.file.Paths; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.ResultSet; -import java.sql.ResultSetMetaData; -import java.sql.Statement; -import java.util.concurrent.TimeUnit; - -import org.apache.drill.common.util.TestTools; -import org.apache.drill.exec.store.hive.HiveTestDataGenerator; -import org.apache.drill.jdbc.Driver; -import org.apache.drill.jdbc.JdbcTest; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Ignore; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TestRule; - -import com.google.common.base.Function; -import com.google.common.base.Stopwatch; - -public class TestDateAggregateFunction { - - public static final String WORKING_PATH; - static{ - Driver.load(); - WORKING_PATH = Paths.get("").toAbsolutePath().toString(); - - } - @Test - public void testDateAggFunction() throws Exception{ - String query = new String("SELECT max(cast(HIRE_DATE as date)) as MAX_DATE, min(cast(HIRE_DATE as date)) as MIN_DATE" + - " FROM `employee.json`"); - - JdbcAssert.withFull("cp") - .sql(query) - .returns( - "MAX_DATE=1998-01-01; " + - "MIN_DATE=1993-05-01\n" - ); - } - - @Test - public void testIntervalAggFunction() throws Exception{ - String query = new String("select max(date_diff(date'2014-5-2', cast(HIRE_DATE as date))) as MAX_DAYS, min(date_diff(date'2014-5-2', cast(HIRE_DATE as date))) MIN_DAYS" + - " FROM `employee.json`"); - - JdbcAssert.withFull("cp") - .sql(query) - .returns( - "MAX_DAYS=7671 days 0:0:0.0; " + - "MIN_DAYS=5965 days 0:0:0.0\n" - ); - } -} http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestFunctionsQuery.java ---------------------------------------------------------------------- diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestFunctionsQuery.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestFunctionsQuery.java new file mode 100644 index 0000000..5ef25dc --- /dev/null +++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestFunctionsQuery.java @@ -0,0 +1,404 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.drill.jdbc.test; + +import java.nio.file.Paths; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.Statement; +import java.util.concurrent.TimeUnit; + +import org.apache.drill.common.util.TestTools; +import org.apache.drill.exec.store.hive.HiveTestDataGenerator; +import org.apache.drill.jdbc.Driver; +import org.apache.drill.jdbc.JdbcTest; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestRule; + +import com.google.common.base.Function; +import com.google.common.base.Stopwatch; + +public class TestFunctionsQuery { + + public static final String WORKING_PATH; + static{ + Driver.load(); + WORKING_PATH = Paths.get("").toAbsolutePath().toString(); + + } + @Test + public void testAbsDecimalFunction() throws Exception{ + String query = String.format("SELECT abs(cast('1234.4567' as decimal(9, 5))) as DEC9_ABS_1, " + + "abs(cast('-1234.4567' as decimal(9, 5))) DEC9_ABS_2, " + + "abs(cast('99999912399.4567' as decimal(18, 5))) DEC18_ABS_1, " + + "abs(cast('-99999912399.4567' as decimal(18, 5))) DEC18_ABS_2, " + + "abs(cast('12345678912345678912.4567' as decimal(28, 5))) DEC28_ABS_1, " + + "abs(cast('-12345678912345678912.4567' as decimal(28, 5))) DEC28_ABS_2, " + + "abs(cast('1234567891234567891234567891234567891.4' as decimal(38, 1))) DEC38_ABS_1, " + + "abs(cast('-1234567891234567891234567891234567891.4' as decimal(38, 1))) DEC38_ABS_2 " + + "FROM dfs.`%s/../../sample-data/region.parquet` limit 1", WORKING_PATH); + + JdbcAssert.withNoDefaultSchema() + .sql(query) + .returns( + "DEC9_ABS_1=1234.45670; " + + "DEC9_ABS_2=1234.45670; " + + "DEC18_ABS_1=99999912399.45670; " + + "DEC18_ABS_2=99999912399.45670; " + + "DEC28_ABS_1=12345678912345678912.45670; " + + "DEC28_ABS_2=12345678912345678912.45670; " + + "DEC38_ABS_1=1234567891234567891234567891234567891.4; " + + "DEC38_ABS_2=1234567891234567891234567891234567891.4\n" + ); + + } + + @Test + public void testCeilDecimalFunction() throws Exception { + String query = String.format("SELECT " + + "ceil(cast('1234.4567' as decimal(9, 5))) as DEC9_1, " + + "ceil(cast('1234.0000' as decimal(9, 5))) as DEC9_2, " + + "ceil(cast('-1234.4567' as decimal(9, 5))) as DEC9_3, " + + "ceil(cast('-1234.000' as decimal(9, 5))) as DEC9_4, " + + "ceil(cast('99999912399.4567' as decimal(18, 5))) DEC18_1, " + + "ceil(cast('99999912399.0000' as decimal(18, 5))) DEC18_2, " + + "ceil(cast('-99999912399.4567' as decimal(18, 5))) DEC18_3, " + + "ceil(cast('-99999912399.0000' as decimal(18, 5))) DEC18_4, " + + "ceil(cast('12345678912345678912.4567' as decimal(28, 5))) DEC28_1, " + + "ceil(cast('999999999999999999.4567' as decimal(28, 5))) DEC28_2, " + + "ceil(cast('12345678912345678912.0000' as decimal(28, 5))) DEC28_3, " + + "ceil(cast('-12345678912345678912.4567' as decimal(28, 5))) DEC28_4, " + + "ceil(cast('-12345678912345678912.0000' as decimal(28, 5))) DEC28_5, " + + "ceil(cast('1234567891234567891234567891234567891.4' as decimal(38, 1))) DEC38_1, " + + "ceil(cast('999999999999999999999999999999999999.4' as decimal(38, 1))) DEC38_2, " + + "ceil(cast('1234567891234567891234567891234567891.0' as decimal(38, 1))) DEC38_3, " + + "ceil(cast('-1234567891234567891234567891234567891.4' as decimal(38, 1))) DEC38_4, " + + "ceil(cast('-1234567891234567891234567891234567891.0' as decimal(38, 1))) DEC38_5 " + + "FROM dfs.`%s/../../sample-data/region.parquet` limit 1", WORKING_PATH); + + JdbcAssert.withNoDefaultSchema() + .sql(query) + .returns( + "DEC9_1=1235; " + + "DEC9_2=1234; " + + "DEC9_3=-1234; " + + "DEC9_4=-1234; " + + "DEC18_1=99999912400; " + + "DEC18_2=99999912399; " + + "DEC18_3=-99999912399; " + + "DEC18_4=-99999912399; " + + "DEC28_1=12345678912345678913; " + + "DEC28_2=1000000000000000000; " + + "DEC28_3=12345678912345678912; " + + "DEC28_4=-12345678912345678912; " + + "DEC28_5=-12345678912345678912; " + + "DEC38_1=1234567891234567891234567891234567892; " + + "DEC38_2=1000000000000000000000000000000000000; " + + "DEC38_3=1234567891234567891234567891234567891; " + + "DEC38_4=-1234567891234567891234567891234567891; " + + "DEC38_5=-1234567891234567891234567891234567891\n" + ); + + } + + @Test + public void testFloorDecimalFunction() throws Exception { + String query = String.format("SELECT " + + "floor(cast('1234.4567' as decimal(9, 5))) as DEC9_1, " + + "floor(cast('1234.0000' as decimal(9, 5))) as DEC9_2, " + + "floor(cast('-1234.4567' as decimal(9, 5))) as DEC9_3, " + + "floor(cast('-1234.000' as decimal(9, 5))) as DEC9_4, " + + "floor(cast('99999912399.4567' as decimal(18, 5))) DEC18_1, " + + "floor(cast('99999912399.0000' as decimal(18, 5))) DEC18_2, " + + "floor(cast('-99999912399.4567' as decimal(18, 5))) DEC18_3, " + + "floor(cast('-99999912399.0000' as decimal(18, 5))) DEC18_4, " + + "floor(cast('12345678912345678912.4567' as decimal(28, 5))) DEC28_1, " + + "floor(cast('999999999999999999.4567' as decimal(28, 5))) DEC28_2, " + + "floor(cast('12345678912345678912.0000' as decimal(28, 5))) DEC28_3, " + + "floor(cast('-12345678912345678912.4567' as decimal(28, 5))) DEC28_4, " + + "floor(cast('-12345678912345678912.0000' as decimal(28, 5))) DEC28_5, " + + "floor(cast('1234567891234567891234567891234567891.4' as decimal(38, 1))) DEC38_1, " + + "floor(cast('999999999999999999999999999999999999.4' as decimal(38, 1))) DEC38_2, " + + "floor(cast('1234567891234567891234567891234567891.0' as decimal(38, 1))) DEC38_3, " + + "floor(cast('-1234567891234567891234567891234567891.4' as decimal(38, 1))) DEC38_4, " + + "floor(cast('-999999999999999999999999999999999999.4' as decimal(38, 1))) DEC38_5 " + + "FROM dfs.`%s/../../sample-data/region.parquet` limit 1", WORKING_PATH); + + JdbcAssert.withNoDefaultSchema() + .sql(query) + .returns( + "DEC9_1=1234; " + + "DEC9_2=1234; " + + "DEC9_3=-1235; " + + "DEC9_4=-1234; " + + "DEC18_1=99999912399; " + + "DEC18_2=99999912399; " + + "DEC18_3=-99999912400; " + + "DEC18_4=-99999912399; " + + "DEC28_1=12345678912345678912; " + + "DEC28_2=999999999999999999; " + + "DEC28_3=12345678912345678912; " + + "DEC28_4=-12345678912345678913; " + + "DEC28_5=-12345678912345678912; " + + "DEC38_1=1234567891234567891234567891234567891; " + + "DEC38_2=999999999999999999999999999999999999; " + + "DEC38_3=1234567891234567891234567891234567891; " + + "DEC38_4=-1234567891234567891234567891234567892; " + + "DEC38_5=-1000000000000000000000000000000000000\n" + ); + } + + @Test + public void testTruncateDecimalFunction() throws Exception { + String query = String.format("SELECT " + + "trunc(cast('1234.4567' as decimal(9, 5))) as DEC9_1, " + + "trunc(cast('1234.0000' as decimal(9, 5))) as DEC9_2, " + + "trunc(cast('-1234.4567' as decimal(9, 5))) as DEC9_3, " + + "trunc(cast('0.111' as decimal(9, 5))) as DEC9_4, " + + "trunc(cast('99999912399.4567' as decimal(18, 5))) DEC18_1, " + + "trunc(cast('99999912399.0000' as decimal(18, 5))) DEC18_2, " + + "trunc(cast('-99999912399.4567' as decimal(18, 5))) DEC18_3, " + + "trunc(cast('-99999912399.0000' as decimal(18, 5))) DEC18_4, " + + "trunc(cast('12345678912345678912.4567' as decimal(28, 5))) DEC28_1, " + + "trunc(cast('999999999999999999.4567' as decimal(28, 5))) DEC28_2, " + + "trunc(cast('12345678912345678912.0000' as decimal(28, 5))) DEC28_3, " + + "trunc(cast('-12345678912345678912.4567' as decimal(28, 5))) DEC28_4, " + + "trunc(cast('-12345678912345678912.0000' as decimal(28, 5))) DEC28_5, " + + "trunc(cast('1234567891234567891234567891234567891.4' as decimal(38, 1))) DEC38_1, " + + "trunc(cast('999999999999999999999999999999999999.4' as decimal(38, 1))) DEC38_2, " + + "trunc(cast('1234567891234567891234567891234567891.0' as decimal(38, 1))) DEC38_3, " + + "trunc(cast('-1234567891234567891234567891234567891.4' as decimal(38, 1))) DEC38_4, " + + "trunc(cast('-999999999999999999999999999999999999.4' as decimal(38, 1))) DEC38_5 " + + "FROM dfs.`%s/../../sample-data/region.parquet` limit 1", WORKING_PATH); + + JdbcAssert.withNoDefaultSchema() + .sql(query) + .returns( + "DEC9_1=1234; " + + "DEC9_2=1234; " + + "DEC9_3=-1234; " + + "DEC9_4=0; " + + "DEC18_1=99999912399; " + + "DEC18_2=99999912399; " + + "DEC18_3=-99999912399; " + + "DEC18_4=-99999912399; " + + "DEC28_1=12345678912345678912; " + + "DEC28_2=999999999999999999; " + + "DEC28_3=12345678912345678912; " + + "DEC28_4=-12345678912345678912; " + + "DEC28_5=-12345678912345678912; " + + "DEC38_1=1234567891234567891234567891234567891; " + + "DEC38_2=999999999999999999999999999999999999; " + + "DEC38_3=1234567891234567891234567891234567891; " + + "DEC38_4=-1234567891234567891234567891234567891; " + + "DEC38_5=-999999999999999999999999999999999999\n" + ); + } + + @Test + public void testTruncateWithParamDecimalFunction() throws Exception { + String query = String.format("SELECT " + + "trunc(cast('1234.4567' as decimal(9, 5)), 2) as DEC9_1, " + + "trunc(cast('1234.45' as decimal(9, 2)), 4) as DEC9_2, " + + "trunc(cast('-1234.4567' as decimal(9, 5)), 0) as DEC9_3, " + + "trunc(cast('0.111' as decimal(9, 5)), 2) as DEC9_4, " + + "trunc(cast('99999912399.4567' as decimal(18, 5)), 2) DEC18_1, " + + "trunc(cast('99999912399.0000' as decimal(18, 5)), 2) DEC18_2, " + + "trunc(cast('-99999912399.45' as decimal(18, 2)), 6) DEC18_3, " + + "trunc(cast('-99999912399.0000' as decimal(18, 5)), 4) DEC18_4, " + + "trunc(cast('12345678912345678912.4567' as decimal(28, 5)), 1) DEC28_1, " + + "trunc(cast('999999999999999999.456' as decimal(28, 3)), 6) DEC28_2, " + + "trunc(cast('12345678912345678912.0000' as decimal(28, 5)), 2) DEC28_3, " + + "trunc(cast('-12345678912345678912.45' as decimal(28, 2)), 0) DEC28_4, " + + "trunc(cast('-12345678912345678912.0000' as decimal(28, 5)), 1) DEC28_5, " + + "trunc(cast('999999999.123456789' as decimal(38, 9)), 7) DEC38_1, " + + "trunc(cast('999999999.4' as decimal(38, 1)), 8) DEC38_2, " + + "trunc(cast('999999999.1234' as decimal(38, 4)), 12) DEC38_3, " + + "trunc(cast('-123456789123456789.4' as decimal(38, 1)), 10) DEC38_4, " + + "trunc(cast('-999999999999999999999999999999999999.4' as decimal(38, 1)), 1) DEC38_5 " + + "FROM dfs.`%s/../../sample-data/region.parquet` limit 1", WORKING_PATH); + + JdbcAssert.withNoDefaultSchema() + .sql(query) + .returns( + "DEC9_1=1234.45; " + + "DEC9_2=1234.4500; " + + "DEC9_3=-1234; " + + "DEC9_4=0.11; " + + "DEC18_1=99999912399.45; " + + "DEC18_2=99999912399.00; " + + "DEC18_3=-99999912399.450000; " + + "DEC18_4=-99999912399.0000; " + + "DEC28_1=12345678912345678912.4; " + + "DEC28_2=999999999999999999.456000; " + + "DEC28_3=12345678912345678912.00; " + + "DEC28_4=-12345678912345678912; " + + "DEC28_5=-12345678912345678912.0; " + + "DEC38_1=999999999.1234567; " + + "DEC38_2=999999999.40000000; " + + "DEC38_3=999999999.123400000000; " + + "DEC38_4=-123456789123456789.4000000000; " + + "DEC38_5=-999999999999999999999999999999999999.4\n" + ); + } + + @Test + public void testRoundDecimalFunction() throws Exception { + String query = String.format("SELECT " + + "round(cast('1234.5567' as decimal(9, 5))) as DEC9_1, " + + "round(cast('1234.1000' as decimal(9, 5))) as DEC9_2, " + + "round(cast('-1234.5567' as decimal(9, 5))) as DEC9_3, " + + "round(cast('-1234.1234' as decimal(9, 5))) as DEC9_4, " + + "round(cast('99999912399.9567' as decimal(18, 5))) DEC18_1, " + + "round(cast('99999912399.0000' as decimal(18, 5))) DEC18_2, " + + "round(cast('-99999912399.5567' as decimal(18, 5))) DEC18_3, " + + "round(cast('-99999912399.0000' as decimal(18, 5))) DEC18_4, " + + "round(cast('12345678912345678912.5567' as decimal(28, 5))) DEC28_1, " + + "round(cast('999999999999999999.5567' as decimal(28, 5))) DEC28_2, " + + "round(cast('12345678912345678912.0000' as decimal(28, 5))) DEC28_3, " + + "round(cast('-12345678912345678912.5567' as decimal(28, 5))) DEC28_4, " + + "round(cast('-12345678912345678912.0000' as decimal(28, 5))) DEC28_5, " + + "round(cast('999999999999999999999999999.5' as decimal(38, 1))) DEC38_1, " + + "round(cast('99999999.512345678123456789' as decimal(38, 18))) DEC38_2, " + + "round(cast('999999999999999999999999999999999999.5' as decimal(38, 1))) DEC38_3, " + + "round(cast('1234567891234567891234567891234567891.2' as decimal(38, 1))) DEC38_4, " + + "round(cast('-1234567891234567891234567891234567891.4' as decimal(38, 1))) DEC38_5, " + + "round(cast('-999999999999999999999999999999999999.9' as decimal(38, 1))) DEC38_6 " + + "FROM dfs.`%s/../../sample-data/region.parquet` limit 1", WORKING_PATH); + + JdbcAssert.withNoDefaultSchema() + .sql(query) + .returns( + "DEC9_1=1235; " + + "DEC9_2=1234; " + + "DEC9_3=-1235; " + + "DEC9_4=-1234; " + + "DEC18_1=99999912400; " + + "DEC18_2=99999912399; " + + "DEC18_3=-99999912400; " + + "DEC18_4=-99999912399; " + + "DEC28_1=12345678912345678913; " + + "DEC28_2=1000000000000000000; " + + "DEC28_3=12345678912345678912; " + + "DEC28_4=-12345678912345678913; " + + "DEC28_5=-12345678912345678912; " + + "DEC38_1=1000000000000000000000000000; " + + "DEC38_2=100000000; " + + "DEC38_3=1000000000000000000000000000000000000; " + + "DEC38_4=1234567891234567891234567891234567891; " + + "DEC38_5=-1234567891234567891234567891234567891; " + + "DEC38_6=-1000000000000000000000000000000000000\n" + ); + } + + @Test + public void testRoundWithScaleDecimalFunction() throws Exception { + String query = String.format("SELECT " + + "round(cast('1234.5567' as decimal(9, 5)), 3) as DEC9_1, " + + "round(cast('1234.1000' as decimal(9, 5)), 2) as DEC9_2, " + + "round(cast('-1234.5567' as decimal(9, 5)), 4) as DEC9_3, " + + "round(cast('-1234.1234' as decimal(9, 5)), 3) as DEC9_4, " + + "round(cast('-1234.1234' as decimal(9, 2)), 4) as DEC9_5, " + + "round(cast('99999912399.9567' as decimal(18, 5)), 3) DEC18_1, " + + "round(cast('99999912399.0000' as decimal(18, 5)), 2) DEC18_2, " + + "round(cast('-99999912399.5567' as decimal(18, 5)), 2) DEC18_3, " + + "round(cast('-99999912399.0000' as decimal(18, 5)), 0) DEC18_4, " + + "round(cast('12345678912345678912.5567' as decimal(28, 5)), 2) DEC28_1, " + + "round(cast('999999999999999999.5567' as decimal(28, 5)), 1) DEC28_2, " + + "round(cast('12345678912345678912.0000' as decimal(28, 5)), 8) DEC28_3, " + + "round(cast('-12345678912345678912.5567' as decimal(28, 5)), 3) DEC28_4, " + + "round(cast('-12345678912345678912.0000' as decimal(28, 5)), 0) DEC28_5, " + + "round(cast('999999999999999999999999999.5' as decimal(38, 1)), 1) DEC38_1, " + + "round(cast('99999999.512345678923456789' as decimal(38, 18)), 9) DEC38_2, " + + "round(cast('999999999.9999999995678' as decimal(38, 18)), 9) DEC38_3, " + + "round(cast('999999999.9999999995678' as decimal(38, 18)), 11) DEC38_4, " + + "round(cast('999999999.9999999995678' as decimal(38, 18)), 21) DEC38_5, " + + "round(cast('-1234567891234567891234567891234567891.4' as decimal(38, 1)), 1) DEC38_6, " + + "round(cast('-999999999999999999999999999999999999.9' as decimal(38, 1)), 0) DEC38_7 " + + "FROM dfs.`%s/../../sample-data/region.parquet` limit 1", WORKING_PATH); + + JdbcAssert.withNoDefaultSchema() + .sql(query) + .returns( + "DEC9_1=1234.557; " + + "DEC9_2=1234.10; " + + "DEC9_3=-1234.5567; " + + "DEC9_4=-1234.123; " + + "DEC9_5=-1234.1200; " + + "DEC18_1=99999912399.957; " + + "DEC18_2=99999912399.00; " + + "DEC18_3=-99999912399.56; " + + "DEC18_4=-99999912399; " + + "DEC28_1=12345678912345678912.56; " + + "DEC28_2=999999999999999999.6; " + + "DEC28_3=12345678912345678912.00000000; " + + "DEC28_4=-12345678912345678912.557; " + + "DEC28_5=-12345678912345678912; " + + "DEC38_1=999999999999999999999999999.5; " + + "DEC38_2=99999999.512345679; " + + "DEC38_3=1000000000.000000000; " + + "DEC38_4=999999999.99999999957; " + + "DEC38_5=999999999.999999999567800000000; " + + "DEC38_6=-1234567891234567891234567891234567891.4; " + + "DEC38_7=-1000000000000000000000000000000000000\n" + ); + } + + @Test + public void testToCharFunction() throws Exception { + String query = String.format("SELECT " + + "to_char(1234.5567, '#,###.##') as FLOAT8_1, " + + "to_char(1234.5, '$#,###.00') as FLOAT8_2, " + + "to_char(cast('1234.5567' as decimal(9, 5)), '#,###.##') as DEC9_1, " + + "to_char(cast('99999912399.9567' as decimal(18, 5)), '#.#####') DEC18_1, " + + "to_char(cast('12345678912345678912.5567' as decimal(28, 5)), '#,###.#####') DEC28_1, " + + "to_char(cast('999999999999999999999999999.5' as decimal(38, 1)), '#.#') DEC38_1 " + + "FROM dfs.`%s/../../sample-data/region.parquet` limit 1", WORKING_PATH); + + JdbcAssert.withNoDefaultSchema() + .sql(query) + .returns( + "FLOAT8_1=1,234.56; " + + "FLOAT8_2=$1,234.50; " + + "DEC9_1=1,234.56; " + + "DEC18_1=99999912399.9567; " + + "DEC28_1=12,345,678,912,345,678,912.5567; " + + "DEC38_1=999999999999999999999999999.5\n" + ); + } + + + @Test + public void testConcatFunction() throws Exception { + String query = String.format("SELECT " + + "concat('1234', ' COL_VALUE ', R_REGIONKEY, ' - STRING') as STR_1 " + + "FROM dfs.`%s/../../sample-data/region.parquet` limit 1", WORKING_PATH); + + JdbcAssert.withNoDefaultSchema() + .sql(query) + .returns( + "STR_1=1234 COL_VALUE 0 - STRING\n" + ); + } +}
