You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2018/03/09 23:03:40 UTC

[3/3] phoenix git commit: PHOENIX-4644 Array modification functions should require two arguments

PHOENIX-4644 Array modification functions should require two arguments


Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/c1391ba0
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/c1391ba0
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/c1391ba0

Branch: refs/heads/4.x-HBase-1.3
Commit: c1391ba03635d392f9ff230d3e6217e2d0c1ff2f
Parents: 1cf0744
Author: James Taylor <jt...@salesforce.com>
Authored: Fri Mar 9 15:00:03 2018 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Fri Mar 9 15:00:03 2018 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/ArrayAppendFunctionIT.java  | 43 ++++++++++++---
 .../function/ArrayAppendFunction.java           | 11 +++-
 .../function/ArrayConcatFunction.java           |  3 +-
 .../function/ArrayModifierFunction.java         | 55 ++++++++++++++++---
 .../function/ArrayPrependFunction.java          |  9 ++-
 .../phoenix/parse/ArrayModifierParseNode.java   | 36 ++++++++++++
 .../apache/phoenix/parse/FunctionParseNode.java | 12 +++-
 .../org/apache/phoenix/util/ExpressionUtil.java |  2 +-
 .../phoenix/compile/QueryCompilerTest.java      | 58 +++++++++++++++++++-
 9 files changed, 202 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c1391ba0/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayAppendFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayAppendFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayAppendFunctionIT.java
index 089cd41..caa17fe 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayAppendFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayAppendFunctionIT.java
@@ -81,26 +81,53 @@ public class ArrayAppendFunctionIT extends ParallelStatsDisabledIT {
         String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(nullVarChar,'34567'),ARRAY_PREPEND('34567',nullVarChar) FROM " + tableName + " LIMIT 1");
-        assertTrue(rs.next());
-
         String[] strings = new String[]{"34567"};
-
         Array array = conn.createArrayOf("VARCHAR", strings);
 
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(null,'34567') FROM " + tableName + " LIMIT 1");
+        assertTrue(rs.next());
+        assertEquals(array, rs.getArray(1));
+        assertFalse(rs.next());
+        
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND('34567',null) FROM " + tableName + " LIMIT 1");
+        assertTrue(rs.next());
+        assertEquals(array, rs.getArray(1));
+        assertFalse(rs.next());
+
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(nullVarChar,'34567') FROM " + tableName + " LIMIT 1");
+        assertTrue(rs.next());
         assertEquals(array, rs.getArray(1));
-        assertEquals(array, rs.getArray(2));
         assertFalse(rs.next());
 
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(nullBigint,123),ARRAY_PREPEND(123,nullBigint) FROM " + tableName + " LIMIT 1");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND('34567',nullVarChar) FROM " + tableName + " LIMIT 1");
         assertTrue(rs.next());
+        assertEquals(array, rs.getArray(1));
+        assertFalse(rs.next());
 
-        Long[] longs = new Long[]{123L};
+        Integer[] ints = new Integer[]{123};
+        array = conn.createArrayOf("INTEGER", ints);
 
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(null,123) FROM " + tableName + " LIMIT 1");
+        assertTrue(rs.next());
+        assertEquals(array, rs.getArray(1));
+        assertFalse(rs.next());
+
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(123,null) FROM " + tableName + " LIMIT 1");
+        assertTrue(rs.next());
+        assertEquals(array, rs.getArray(1));
+        assertFalse(rs.next());
+        
+        Long[] longs = new Long[]{123L};
         array = conn.createArrayOf("BIGINT", longs);
 
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(nullBigint,123) FROM " + tableName + " LIMIT 1");
+        assertTrue(rs.next());
+        assertEquals(array, rs.getArray(1));
+        assertFalse(rs.next());
+
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(123,nullBigint) FROM " + tableName + " LIMIT 1");
+        assertTrue(rs.next());
         assertEquals(array, rs.getArray(1));
-        assertEquals(array, rs.getArray(2));
         assertFalse(rs.next());
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c1391ba0/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayAppendFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayAppendFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayAppendFunction.java
index 8c7fa9f..d0c0b3b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayAppendFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayAppendFunction.java
@@ -21,13 +21,18 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.ArrayModifierParseNode;
 import org.apache.phoenix.parse.FunctionParseNode;
 import org.apache.phoenix.schema.TypeMismatchException;
-import org.apache.phoenix.schema.types.*;
+import org.apache.phoenix.schema.types.PArrayDataType;
+import org.apache.phoenix.schema.types.PBinaryArray;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PVarbinary;
+import org.apache.phoenix.schema.types.PVarbinaryArray;
 
-@FunctionParseNode.BuiltInFunction(name = ArrayAppendFunction.NAME, args = {
+@FunctionParseNode.BuiltInFunction(name = ArrayAppendFunction.NAME, nodeClass=ArrayModifierParseNode.class, args = {
         @FunctionParseNode.Argument(allowedTypes = {PBinaryArray.class, PVarbinaryArray.class}),
-        @FunctionParseNode.Argument(allowedTypes = {PVarbinary.class}, defaultValue = "null")})
+        @FunctionParseNode.Argument(allowedTypes = {PVarbinary.class})})
 public class ArrayAppendFunction extends ArrayModifierFunction {
 
     public static final String NAME = "ARRAY_APPEND";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c1391ba0/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayConcatFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayConcatFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayConcatFunction.java
index 28774cb..5e4dba9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayConcatFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayConcatFunction.java
@@ -21,6 +21,7 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.ArrayModifierParseNode;
 import org.apache.phoenix.parse.FunctionParseNode;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TypeMismatchException;
@@ -30,7 +31,7 @@ import org.apache.phoenix.schema.types.PBinaryArray;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PVarbinaryArray;
 
-@FunctionParseNode.BuiltInFunction(name = ArrayConcatFunction.NAME, args = {
+@FunctionParseNode.BuiltInFunction(name = ArrayConcatFunction.NAME, nodeClass=ArrayModifierParseNode.class, args = {
         @FunctionParseNode.Argument(allowedTypes = {PBinaryArray.class, PVarbinaryArray.class}),
         @FunctionParseNode.Argument(allowedTypes = {PBinaryArray.class, PVarbinaryArray.class})})
 public class ArrayConcatFunction extends ArrayModifierFunction {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c1391ba0/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayModifierFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayModifierFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayModifierFunction.java
index 2039690..2b399fe 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayModifierFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayModifierFunction.java
@@ -18,6 +18,8 @@
 
 package org.apache.phoenix.expression.function;
 
+import java.io.DataInput;
+import java.io.IOException;
 import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@@ -29,8 +31,10 @@ import org.apache.phoenix.schema.TypeMismatchException;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PArrayDataType;
 import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.util.ExpressionUtil;
 
 public abstract class ArrayModifierFunction extends ScalarFunction {
+    private boolean isNullArray;
 
     public ArrayModifierFunction() {
     }
@@ -72,6 +76,16 @@ public abstract class ArrayModifierFunction extends ScalarFunction {
             throw new DataExceedsCapacityException(baseDataType, arrayExpr.getMaxLength(),
                     arrayExpr.getScale());
         }
+        init();
+    }
+
+    private void init() {
+        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+        if (getLHSExpr().getDataType().isArrayType()) {
+            isNullArray = ExpressionUtil.isNull(getLHSExpr(), ptr);
+        } else {
+            isNullArray = ExpressionUtil.isNull(getRHSExpr(), ptr);
+        }
     }
 
     @Override
@@ -142,7 +156,8 @@ public abstract class ArrayModifierFunction extends ScalarFunction {
 
     public PDataType getLHSBaseType() {
         if (getLHSExpr().getDataType().isArrayType()) {
-            return PDataType.arrayBaseType(getLHSExpr().getDataType());
+            // Use RHS type if we have a null constant to get the correct array type
+            return isNullArray ? getRHSExpr().getDataType() : PDataType.arrayBaseType(getLHSExpr().getDataType());
         } else {
             return getLHSExpr().getDataType();
         }
@@ -150,7 +165,8 @@ public abstract class ArrayModifierFunction extends ScalarFunction {
 
     public PDataType getRHSBaseType() {
         if (getRHSExpr().getDataType().isArrayType()) {
-            return PDataType.arrayBaseType(getRHSExpr().getDataType());
+            // Use LHS type if we have a null constant to get the correct array type
+            return isNullArray ? getLHSExpr().getDataType() : PDataType.arrayBaseType(getRHSExpr().getDataType());
         } else {
             return getRHSExpr().getDataType();
         }
@@ -159,29 +175,52 @@ public abstract class ArrayModifierFunction extends ScalarFunction {
     @Override
     public PDataType getDataType() {
         if (getLHSExpr().getDataType().isArrayType()) {
-            return getLHSExpr().getDataType();
+            // Use array of RHS type if we have a null constant since otherwise we'd use binary
+            return isNullArray ? PDataType.fromTypeId(getRHSExpr().getDataType().getSqlType() + PDataType.ARRAY_TYPE_BASE) : getLHSExpr().getDataType();
         } else {
-            return getRHSExpr().getDataType();
+            return isNullArray ? PDataType.fromTypeId(getLHSExpr().getDataType().getSqlType() + PDataType.ARRAY_TYPE_BASE) : getRHSExpr().getDataType();
         }
     }
 
+    private Integer getMaxLength(Expression expression) {
+        PDataType type = expression.getDataType();
+        if (type.isFixedWidth() && type.getByteSize() != null) {
+            return type.getByteSize();
+        }
+        return expression.getMaxLength();
+    }
 
     @Override
     public Integer getMaxLength() {
         if (getLHSExpr().getDataType().isArrayType()) {
-            return getLHSExpr().getMaxLength();
+            // Use max length of RHS if we have a null constant since otherwise we'd use null (which breaks fixed types)
+            return getMaxLength(isNullArray ? getRHSExpr() : getLHSExpr());
         } else {
-            return getRHSExpr().getMaxLength();
+            return getMaxLength(isNullArray ? getLHSExpr() : getRHSExpr());
         }
     }
 
     @Override
     public SortOrder getSortOrder() {
         if (getLHSExpr().getDataType().isArrayType()) {
-            return getLHSExpr().getSortOrder();
+            return isNullArray ? getRHSExpr().getSortOrder() : getLHSExpr().getSortOrder();
         } else {
-            return getRHSExpr().getSortOrder();
+            return isNullArray ? getLHSExpr().getSortOrder() : getRHSExpr().getSortOrder();
         }
     }
 
+    @Override
+    public Integer getScale() {
+        if (getLHSExpr().getDataType().isArrayType()) {
+            return isNullArray ? getRHSExpr().getScale() : getLHSExpr().getScale();
+        } else {
+            return isNullArray ? getLHSExpr().getScale() : getRHSExpr().getScale();
+        }
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        init();
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c1391ba0/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayPrependFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayPrependFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayPrependFunction.java
index c2311fb..41fdf2a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayPrependFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayPrependFunction.java
@@ -22,11 +22,16 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.ArrayModifierParseNode;
 import org.apache.phoenix.parse.FunctionParseNode;
 import org.apache.phoenix.schema.TypeMismatchException;
-import org.apache.phoenix.schema.types.*;
+import org.apache.phoenix.schema.types.PArrayDataType;
+import org.apache.phoenix.schema.types.PBinaryArray;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PVarbinary;
+import org.apache.phoenix.schema.types.PVarbinaryArray;
 
-@FunctionParseNode.BuiltInFunction(name = ArrayPrependFunction.NAME, args = {
+@FunctionParseNode.BuiltInFunction(name = ArrayPrependFunction.NAME, nodeClass=ArrayModifierParseNode.class, args = {
         @FunctionParseNode.Argument(allowedTypes = {PVarbinary.class}),
         @FunctionParseNode.Argument(allowedTypes = {PBinaryArray.class, PVarbinaryArray.class})})
 public class ArrayPrependFunction extends ArrayModifierFunction {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c1391ba0/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayModifierParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayModifierParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayModifierParseNode.java
new file mode 100644
index 0000000..2ec1e27
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayModifierParseNode.java
@@ -0,0 +1,36 @@
+/*
+ * 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.phoenix.parse;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.compile.StatementContext;
+
+
+public class ArrayModifierParseNode extends FunctionParseNode {
+
+    public ArrayModifierParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public boolean evalToNullIfParamIsNull(StatementContext context, int index) throws SQLException {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c1391ba0/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java
index df9ad86..fc16156 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java
@@ -184,6 +184,10 @@ public class FunctionParseNode extends CompoundParseNode {
                     } else { // Use expression as is, since we already have the data type set
                         context.getBindManager().addParamMetaData(bindNode, child);
                     }
+                } else if (allowedTypes.length > 0) {
+                    // Switch null type with typed null
+                    children.set(i, LiteralExpression.newConstant(null, PDataTypeFactory.getInstance().instanceFromClass(
+                            allowedTypes[0]), Determinism.ALWAYS));
                 }
             } else {
                 validateFunctionArguement(info, i, child);
@@ -254,10 +258,14 @@ public class FunctionParseNode extends CompoundParseNode {
      */
     public Expression create(List<Expression> children, PFunction function, StatementContext context) throws SQLException {
         try {
+            Constructor<? extends FunctionExpression> fCtor = info.getFuncCtor();
+            if (fCtor == null) {
+                fCtor = getExpressionCtor(info.func, null);
+            }
             if(function == null) {
-                return info.getFuncCtor().newInstance(children);
+                return fCtor.newInstance(children);
             } else {
-                return info.getFuncCtor().newInstance(children, function);
+                return fCtor.newInstance(children, function);
             }
         } catch (InstantiationException e) {
             throw new SQLException(e);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c1391ba0/phoenix-core/src/main/java/org/apache/phoenix/util/ExpressionUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ExpressionUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ExpressionUtil.java
index fbd10fc..881b0e1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/ExpressionUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ExpressionUtil.java
@@ -36,7 +36,7 @@ public class ExpressionUtil {
         Object value = null;
         PDataType type = expression.getDataType();
         if (expression.evaluate(null, ptr) && ptr.getLength() != 0) {
-            value = type.toObject(ptr);
+            value = type.toObject(ptr.get(), ptr.getOffset(), ptr.getLength(), type, expression.getSortOrder(), expression.getMaxLength(), expression.getScale());
         }
         return LiteralExpression.newConstant(value, type, expression.getDeterminism());
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c1391ba0/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
index 2b5a8cb..11f5f22 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
@@ -43,7 +43,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Properties;
 
-import com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.Filter;
@@ -52,7 +51,18 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
 import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
 import org.apache.phoenix.exception.SQLExceptionCode;
-import org.apache.phoenix.execute.*;
+import org.apache.phoenix.execute.AggregatePlan;
+import org.apache.phoenix.execute.ClientAggregatePlan;
+import org.apache.phoenix.execute.ClientScanPlan;
+import org.apache.phoenix.execute.CorrelatePlan;
+import org.apache.phoenix.execute.CursorFetchPlan;
+import org.apache.phoenix.execute.HashJoinPlan;
+import org.apache.phoenix.execute.LiteralResultIterationPlan;
+import org.apache.phoenix.execute.ScanPlan;
+import org.apache.phoenix.execute.SortMergeJoinPlan;
+import org.apache.phoenix.execute.TupleProjectionPlan;
+import org.apache.phoenix.execute.UnionPlan;
+import org.apache.phoenix.execute.UnnestArrayPlan;
 import org.apache.phoenix.execute.visitor.QueryPlanVisitor;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.LiteralExpression;
@@ -88,6 +98,8 @@ import org.apache.phoenix.util.SchemaUtil;
 import org.junit.Ignore;
 import org.junit.Test;
 
+import com.google.common.collect.Lists;
+
 
 
 /**
@@ -2177,6 +2189,48 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
     }
 
     @Test
+    public void testArrayAppendSingleArg() throws SQLException {
+        Connection conn = DriverManager.getConnection(getUrl());
+        try {
+            conn.createStatement().execute("CREATE TABLE t (p INTEGER PRIMARY KEY, arr1 INTEGER ARRAY, arr2 INTEGER ARRAY)");
+            conn.createStatement().executeQuery("SELECT ARRAY_APPEND(arr2) from t");
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.FUNCTION_UNDEFINED.getErrorCode(),e.getErrorCode());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testArrayPrependSingleArg() throws SQLException {
+        Connection conn = DriverManager.getConnection(getUrl());
+        try {
+            conn.createStatement().execute("CREATE TABLE t (p INTEGER PRIMARY KEY, arr1 INTEGER ARRAY, arr2 INTEGER ARRAY)");
+            conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(arr2) from t");
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.FUNCTION_UNDEFINED.getErrorCode(),e.getErrorCode());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testArrayConcatSingleArg() throws SQLException {
+        Connection conn = DriverManager.getConnection(getUrl());
+        try {
+            conn.createStatement().execute("CREATE TABLE t (p INTEGER PRIMARY KEY, arr1 INTEGER ARRAY, arr2 INTEGER ARRAY)");
+            conn.createStatement().executeQuery("SELECT ARRAY_CAT(arr2) from t");
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.FUNCTION_UNDEFINED.getErrorCode(),e.getErrorCode());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
     public void testServerArrayElementProjection5() throws SQLException {
         Connection conn = DriverManager.getConnection(getUrl());
         try {