You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@phoenix.apache.org by xjodoin <gi...@git.apache.org> on 2018/03/06 15:54:18 UTC

[GitHub] phoenix pull request #294: PHOENIX-4643 Implement ARRAY_REMOVE built in func...

GitHub user xjodoin opened a pull request:

    https://github.com/apache/phoenix/pull/294

    PHOENIX-4643 Implement ARRAY_REMOVE built in function

    

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/xjodoin/phoenix PHOENIX-4643

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/phoenix/pull/294.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #294
    
----
commit ac939f8537dcf458cdeaaff2808eb7ef73b942b9
Author: Xavier Jodoin <xa...@...>
Date:   2018-03-06T15:46:21Z

    PHOENIX-4643 Implement ARRAY_REMOVE built in function

----


---

[GitHub] phoenix pull request #294: PHOENIX-4643 Implement ARRAY_REMOVE built in func...

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/294#discussion_r175225740
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayRemoveFunctionIT.java ---
    @@ -0,0 +1,425 @@
    +/*
    + * 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.end2end;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.assertNull;
    +import static org.junit.Assert.assertTrue;
    +
    +import java.sql.Array;
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +
    +import org.apache.phoenix.schema.TypeMismatchException;
    +import org.junit.Test;
    +
    +public class ArrayRemoveFunctionIT extends ParallelStatsDisabledIT {
    +    private String initTables(Connection conn) throws Exception {
    +        String tableName = generateUniqueName();
    +        String ddl = "CREATE TABLE " + tableName
    +                + " (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[],integers INTEGER[],doubles DOUBLE[],bigints BIGINT[],"
    +                + "chars CHAR(15)[],double1 DOUBLE,char1 CHAR(17),nullcheck INTEGER,chars2 CHAR(15)[], nullVarchar VARCHAR[], nullBigInt BIGINT[],double2 DOUBLE,integer1 INTEGER)";
    +        conn.createStatement().execute(ddl);
    +        String dml = "UPSERT INTO " + tableName + "(region_name,varchars,integers,doubles,bigints,chars,double1,char1,nullcheck,chars2,double2,integer1) VALUES('SF Bay Area'," +
    +                "ARRAY['2345','46345','23234']," +
    +                "ARRAY[2345,46345,23234,456]," +
    +                "ARRAY[23.45,46.345,23.234,45.6,5.78]," +
    +                "ARRAY[12,34,56,78,910]," +
    +                "ARRAY['a','bbbb','c','ddd','e']," +
    +                "23.45," +
    +                "'wert'," +
    +                "NULL," +
    +                "ARRAY['a','bbbb','c','ddd','e','foo']," +
    +                "12,"+
    +                "12"+
    +                ")";
    +        PreparedStatement stmt = conn.prepareStatement(dml);
    +        stmt.execute();
    +        conn.commit();
    +        return tableName;
    +    }
    +
    +    @Test
    +    public void testEmptyArrayModification() throws Exception {
    +        Connection conn = DriverManager.getConnection(getUrl());
    +        String tableName = initTables(conn);
    +
    +        ResultSet rs;
    +        rs = conn.createStatement().executeQuery("SELECT ARRAY_REMOVE(nullVarChar,'34567') FROM " + tableName + " LIMIT 1");
    +        assertTrue(rs.next());
    +
    +        assertNull(rs.getArray(1));
    +        assertFalse(rs.next());
    +    }
    +    
    +    @Test
    +    public void testArrayRemoveFunctionVarchar() throws Exception {
    +        Connection conn = DriverManager.getConnection(getUrl());
    +        String tableName = initTables(conn);
    +
    +        ResultSet rs;
    +        rs = conn.createStatement().executeQuery("SELECT ARRAY_REMOVE(varchars,'23234') FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +        assertTrue(rs.next());
    +
    +        String[] strings = new String[]{"2345", "46345"};
    +
    +        Array array = conn.createArrayOf("VARCHAR", strings);
    +
    +        assertEquals(array, rs.getArray(1));
    +        assertFalse(rs.next());
    +    }
    +
    +    @Test
    +    public void testArrayRemoveFunctionInteger() throws Exception {
    +        Connection conn = DriverManager.getConnection(getUrl());
    +        String tableName = initTables(conn);
    +
    +        ResultSet rs;
    +        rs = conn.createStatement().executeQuery("SELECT ARRAY_REMOVE(integers,456) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +        assertTrue(rs.next());
    +
    +        Integer[] integers = new Integer[]{2345, 46345, 23234};
    +
    +        Array array = conn.createArrayOf("INTEGER", integers);
    +
    +        assertEquals(array, rs.getArray(1));
    +        assertFalse(rs.next());
    +    }
    +
    +    @Test
    +    public void testArrayRemoveFunctionDouble() throws Exception {
    +        Connection conn = DriverManager.getConnection(getUrl());
    +        String tableName = initTables(conn);
    +
    +        ResultSet rs;
    +        rs = conn.createStatement().executeQuery("SELECT ARRAY_REMOVE(doubles,double1) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +        assertTrue(rs.next());
    +
    +        Double[] doubles = new Double[]{46.345, 23.234, 45.6, 5.78};
    +
    +        Array array = conn.createArrayOf("DOUBLE", doubles);
    +
    +        assertEquals(array, rs.getArray(1));
    +        assertFalse(rs.next());
    +    }
    +    
    +    @Test
    +    public void testArrayRemoveFunctionDoubleWithInt() throws Exception {
    +        Connection conn = DriverManager.getConnection(getUrl());
    +        String tableName = initTables(conn);
    +
    +        ResultSet rs;
    +        rs = conn.createStatement().executeQuery("SELECT ARRAY_REMOVE(doubles,10) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    --- End diff --
    
    Make sure to test when 10.0 is in the array and you’re removing with a 10 int argument.


---

[GitHub] phoenix pull request #294: PHOENIX-4643 Implement ARRAY_REMOVE built in func...

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/294#discussion_r172607179
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayRemoveFunction.java ---
    @@ -0,0 +1,102 @@
    +/*
    + * 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.expression.function;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.commons.lang.ArrayUtils;
    +import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
    +import org.apache.phoenix.expression.Expression;
    +import org.apache.phoenix.parse.FunctionParseNode;
    +import org.apache.phoenix.schema.SortOrder;
    +import org.apache.phoenix.schema.TypeMismatchException;
    +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;
    +import org.apache.phoenix.schema.types.PhoenixArray;
    +
    +@FunctionParseNode.BuiltInFunction(name = ArrayRemoveFunction.NAME, args = {
    +		@FunctionParseNode.Argument(allowedTypes = { PBinaryArray.class, PVarbinaryArray.class }),
    +		@FunctionParseNode.Argument(allowedTypes = { PVarbinary.class }, defaultValue = "null") })
    +public class ArrayRemoveFunction extends ArrayModifierFunction {
    +
    +	public static final String NAME = "ARRAY_REMOVE";
    +
    +	public ArrayRemoveFunction() {
    +	}
    +
    +	public ArrayRemoveFunction(List<Expression> children) throws TypeMismatchException {
    +		super(children);
    +	}
    +
    +	@Override
    +	protected boolean modifierFunction(ImmutableBytesWritable ptr, int length, int offset, byte[] arrayBytes,
    +			PDataType baseType, int arrayLength, Integer maxLength, Expression arrayExp) {
    +		SortOrder sortOrder = arrayExp.getSortOrder();
    +
    +		if (ptr.getLength() == 0 || arrayBytes.length == 0) {
    +			ptr.set(arrayBytes, offset, length);
    +			return true;
    +		}
    +
    +		PhoenixArray array = (PhoenixArray) getDataType().toObject(arrayBytes, offset, length, getDataType(), sortOrder,
    +				maxLength, null);
    +
    +		
    +		Object toCompare = baseType.toObject(ptr, sortOrder);
    +		
    +		if(baseType.isFixedWidth()) {
    +			toCompare = baseType.pad(toCompare, array.getMaxLength());
    +		}
    +
    +		int dimensions = array.getDimensions();
    +		List<Object> values = new ArrayList<>();
    +		for (int i = 0; i < dimensions; i++) {
    +			Object element = array.getElement(i);
    +			if (element != null && element.equals(toCompare)
    +					|| (element.getClass().isArray() && ArrayUtils.isEquals(element, toCompare))) {
    +				if (getDataType().isFixedWidth()) {
    --- End diff --
    
    When removing an element, you shouldn't replace it with null. I think you want to just not add the element to values in this case.


---

[GitHub] phoenix pull request #294: PHOENIX-4643 Implement ARRAY_REMOVE built in func...

Posted by xjodoin <gi...@git.apache.org>.
Github user xjodoin commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/294#discussion_r172642254
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayRemoveFunction.java ---
    @@ -0,0 +1,102 @@
    +/*
    + * 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.expression.function;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.commons.lang.ArrayUtils;
    +import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
    +import org.apache.phoenix.expression.Expression;
    +import org.apache.phoenix.parse.FunctionParseNode;
    +import org.apache.phoenix.schema.SortOrder;
    +import org.apache.phoenix.schema.TypeMismatchException;
    +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;
    +import org.apache.phoenix.schema.types.PhoenixArray;
    +
    +@FunctionParseNode.BuiltInFunction(name = ArrayRemoveFunction.NAME, args = {
    +		@FunctionParseNode.Argument(allowedTypes = { PBinaryArray.class, PVarbinaryArray.class }),
    +		@FunctionParseNode.Argument(allowedTypes = { PVarbinary.class }, defaultValue = "null") })
    --- End diff --
    
    When I remove the default value I have 3 failling tests I started my test from ArrayAppendFunctionIT because I assume the feature set should be similar.
    testArrayRemoveFunctionDoublesWithNull
    testArrayRemoveFunctionVarcharWithNull
    testArrayRemoveFunctionCharsWithNull


---

[GitHub] phoenix pull request #294: PHOENIX-4643 Implement ARRAY_REMOVE built in func...

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/294#discussion_r172603085
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayRemoveFunction.java ---
    @@ -0,0 +1,102 @@
    +/*
    + * 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.expression.function;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.commons.lang.ArrayUtils;
    +import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
    +import org.apache.phoenix.expression.Expression;
    +import org.apache.phoenix.parse.FunctionParseNode;
    +import org.apache.phoenix.schema.SortOrder;
    +import org.apache.phoenix.schema.TypeMismatchException;
    +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;
    +import org.apache.phoenix.schema.types.PhoenixArray;
    +
    +@FunctionParseNode.BuiltInFunction(name = ArrayRemoveFunction.NAME, args = {
    +		@FunctionParseNode.Argument(allowedTypes = { PBinaryArray.class, PVarbinaryArray.class }),
    +		@FunctionParseNode.Argument(allowedTypes = { PVarbinary.class }, defaultValue = "null") })
    --- End diff --
    
    Does it make sense to have a default value for the second parameter? Seems like no, as then you'd be missing the element to remove which is illegal.


---

[GitHub] phoenix pull request #294: PHOENIX-4643 Implement ARRAY_REMOVE built in func...

Posted by xjodoin <gi...@git.apache.org>.
Github user xjodoin commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/294#discussion_r172642518
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayRemoveFunction.java ---
    @@ -0,0 +1,102 @@
    +/*
    + * 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.expression.function;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.commons.lang.ArrayUtils;
    +import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
    +import org.apache.phoenix.expression.Expression;
    +import org.apache.phoenix.parse.FunctionParseNode;
    +import org.apache.phoenix.schema.SortOrder;
    +import org.apache.phoenix.schema.TypeMismatchException;
    +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;
    +import org.apache.phoenix.schema.types.PhoenixArray;
    +
    +@FunctionParseNode.BuiltInFunction(name = ArrayRemoveFunction.NAME, args = {
    +		@FunctionParseNode.Argument(allowedTypes = { PBinaryArray.class, PVarbinaryArray.class }),
    +		@FunctionParseNode.Argument(allowedTypes = { PVarbinary.class }, defaultValue = "null") })
    +public class ArrayRemoveFunction extends ArrayModifierFunction {
    +
    +	public static final String NAME = "ARRAY_REMOVE";
    +
    +	public ArrayRemoveFunction() {
    +	}
    +
    +	public ArrayRemoveFunction(List<Expression> children) throws TypeMismatchException {
    +		super(children);
    +	}
    +
    +	@Override
    +	protected boolean modifierFunction(ImmutableBytesWritable ptr, int length, int offset, byte[] arrayBytes,
    +			PDataType baseType, int arrayLength, Integer maxLength, Expression arrayExp) {
    +		SortOrder sortOrder = arrayExp.getSortOrder();
    +
    +		if (ptr.getLength() == 0 || arrayBytes.length == 0) {
    +			ptr.set(arrayBytes, offset, length);
    +			return true;
    +		}
    +
    +		PhoenixArray array = (PhoenixArray) getDataType().toObject(arrayBytes, offset, length, getDataType(), sortOrder,
    +				maxLength, null);
    +
    +		
    +		Object toCompare = baseType.toObject(ptr, sortOrder);
    +		
    +		if(baseType.isFixedWidth()) {
    +			toCompare = baseType.pad(toCompare, array.getMaxLength());
    +		}
    +
    +		int dimensions = array.getDimensions();
    +		List<Object> values = new ArrayList<>();
    +		for (int i = 0; i < dimensions; i++) {
    +			Object element = array.getElement(i);
    +			if (element != null && element.equals(toCompare)
    +					|| (element.getClass().isArray() && ArrayUtils.isEquals(element, toCompare))) {
    --- End diff --
    
    fix


---

[GitHub] phoenix pull request #294: PHOENIX-4643 Implement ARRAY_REMOVE built in func...

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/294#discussion_r175225898
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayRemoveFunctionIT.java ---
    @@ -0,0 +1,425 @@
    +/*
    + * 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.end2end;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.assertNull;
    +import static org.junit.Assert.assertTrue;
    +
    +import java.sql.Array;
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +
    +import org.apache.phoenix.schema.TypeMismatchException;
    +import org.junit.Test;
    +
    +public class ArrayRemoveFunctionIT extends ParallelStatsDisabledIT {
    +    private String initTables(Connection conn) throws Exception {
    +        String tableName = generateUniqueName();
    +        String ddl = "CREATE TABLE " + tableName
    +                + " (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[],integers INTEGER[],doubles DOUBLE[],bigints BIGINT[],"
    +                + "chars CHAR(15)[],double1 DOUBLE,char1 CHAR(17),nullcheck INTEGER,chars2 CHAR(15)[], nullVarchar VARCHAR[], nullBigInt BIGINT[],double2 DOUBLE,integer1 INTEGER)";
    +        conn.createStatement().execute(ddl);
    +        String dml = "UPSERT INTO " + tableName + "(region_name,varchars,integers,doubles,bigints,chars,double1,char1,nullcheck,chars2,double2,integer1) VALUES('SF Bay Area'," +
    +                "ARRAY['2345','46345','23234']," +
    +                "ARRAY[2345,46345,23234,456]," +
    +                "ARRAY[23.45,46.345,23.234,45.6,5.78]," +
    +                "ARRAY[12,34,56,78,910]," +
    +                "ARRAY['a','bbbb','c','ddd','e']," +
    +                "23.45," +
    +                "'wert'," +
    +                "NULL," +
    +                "ARRAY['a','bbbb','c','ddd','e','foo']," +
    +                "12,"+
    +                "12"+
    +                ")";
    +        PreparedStatement stmt = conn.prepareStatement(dml);
    +        stmt.execute();
    +        conn.commit();
    +        return tableName;
    +    }
    +
    +    @Test
    +    public void testEmptyArrayModification() throws Exception {
    +        Connection conn = DriverManager.getConnection(getUrl());
    +        String tableName = initTables(conn);
    +
    +        ResultSet rs;
    +        rs = conn.createStatement().executeQuery("SELECT ARRAY_REMOVE(nullVarChar,'34567') FROM " + tableName + " LIMIT 1");
    +        assertTrue(rs.next());
    +
    +        assertNull(rs.getArray(1));
    +        assertFalse(rs.next());
    +    }
    +    
    +    @Test
    +    public void testArrayRemoveFunctionVarchar() throws Exception {
    +        Connection conn = DriverManager.getConnection(getUrl());
    +        String tableName = initTables(conn);
    +
    +        ResultSet rs;
    +        rs = conn.createStatement().executeQuery("SELECT ARRAY_REMOVE(varchars,'23234') FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +        assertTrue(rs.next());
    +
    +        String[] strings = new String[]{"2345", "46345"};
    +
    +        Array array = conn.createArrayOf("VARCHAR", strings);
    +
    +        assertEquals(array, rs.getArray(1));
    +        assertFalse(rs.next());
    +    }
    +
    +    @Test
    +    public void testArrayRemoveFunctionInteger() throws Exception {
    +        Connection conn = DriverManager.getConnection(getUrl());
    +        String tableName = initTables(conn);
    +
    +        ResultSet rs;
    +        rs = conn.createStatement().executeQuery("SELECT ARRAY_REMOVE(integers,456) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +        assertTrue(rs.next());
    +
    +        Integer[] integers = new Integer[]{2345, 46345, 23234};
    +
    +        Array array = conn.createArrayOf("INTEGER", integers);
    +
    +        assertEquals(array, rs.getArray(1));
    +        assertFalse(rs.next());
    +    }
    +
    +    @Test
    +    public void testArrayRemoveFunctionDouble() throws Exception {
    +        Connection conn = DriverManager.getConnection(getUrl());
    +        String tableName = initTables(conn);
    +
    +        ResultSet rs;
    +        rs = conn.createStatement().executeQuery("SELECT ARRAY_REMOVE(doubles,double1) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +        assertTrue(rs.next());
    +
    +        Double[] doubles = new Double[]{46.345, 23.234, 45.6, 5.78};
    +
    +        Array array = conn.createArrayOf("DOUBLE", doubles);
    +
    +        assertEquals(array, rs.getArray(1));
    +        assertFalse(rs.next());
    +    }
    +    
    +    @Test
    +    public void testArrayRemoveFunctionDoubleWithInt() throws Exception {
    +        Connection conn = DriverManager.getConnection(getUrl());
    +        String tableName = initTables(conn);
    +
    +        ResultSet rs;
    +        rs = conn.createStatement().executeQuery("SELECT ARRAY_REMOVE(doubles,10) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +        assertTrue(rs.next());
    +
    +        Double[] doubles = new Double[]{23.45,46.345,23.234,45.6,5.78};
    +
    +        Array array = conn.createArrayOf("DOUBLE", doubles);
    +
    +        assertEquals(array, rs.getArray(1));
    +        assertFalse(rs.next());
    +    }
    +
    +    @Test
    +    public void testArrayRemoveFunctionBigint() throws Exception {
    +        Connection conn = DriverManager.getConnection(getUrl());
    +        String tableName = initTables(conn);
    +        ResultSet rs;
    +        rs = conn.createStatement().executeQuery("SELECT ARRAY_REMOVE(bigints,56) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    --- End diff --
    
    Same as above - make sure 56L is in array and is removed.


---

[GitHub] phoenix issue #294: PHOENIX-4643 Implement ARRAY_REMOVE built in function

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on the issue:

    https://github.com/apache/phoenix/pull/294
  
    Changes look good. I've committed PHOENIX-4644, so can you make sure everything works without specify a default value for the second argument for ARRAY_REMOVE?


---

[GitHub] phoenix pull request #294: PHOENIX-4643 Implement ARRAY_REMOVE built in func...

Posted by xjodoin <gi...@git.apache.org>.
Github user xjodoin commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/294#discussion_r172642442
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayRemoveFunction.java ---
    @@ -0,0 +1,102 @@
    +/*
    + * 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.expression.function;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.commons.lang.ArrayUtils;
    +import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
    +import org.apache.phoenix.expression.Expression;
    +import org.apache.phoenix.parse.FunctionParseNode;
    +import org.apache.phoenix.schema.SortOrder;
    +import org.apache.phoenix.schema.TypeMismatchException;
    +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;
    +import org.apache.phoenix.schema.types.PhoenixArray;
    +
    +@FunctionParseNode.BuiltInFunction(name = ArrayRemoveFunction.NAME, args = {
    +		@FunctionParseNode.Argument(allowedTypes = { PBinaryArray.class, PVarbinaryArray.class }),
    +		@FunctionParseNode.Argument(allowedTypes = { PVarbinary.class }, defaultValue = "null") })
    +public class ArrayRemoveFunction extends ArrayModifierFunction {
    +
    +	public static final String NAME = "ARRAY_REMOVE";
    +
    +	public ArrayRemoveFunction() {
    +	}
    +
    +	public ArrayRemoveFunction(List<Expression> children) throws TypeMismatchException {
    +		super(children);
    +	}
    +
    +	@Override
    +	protected boolean modifierFunction(ImmutableBytesWritable ptr, int length, int offset, byte[] arrayBytes,
    +			PDataType baseType, int arrayLength, Integer maxLength, Expression arrayExp) {
    +		SortOrder sortOrder = arrayExp.getSortOrder();
    +
    +		if (ptr.getLength() == 0 || arrayBytes.length == 0) {
    +			ptr.set(arrayBytes, offset, length);
    +			return true;
    +		}
    +
    +		PhoenixArray array = (PhoenixArray) getDataType().toObject(arrayBytes, offset, length, getDataType(), sortOrder,
    +				maxLength, null);
    --- End diff --
    
    Fixed replace with binary compareTo


---

[GitHub] phoenix pull request #294: PHOENIX-4643 Implement ARRAY_REMOVE built in func...

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/294#discussion_r175620626
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayRemoveFunctionIT.java ---
    @@ -0,0 +1,383 @@
    +/*
    + * 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.end2end;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.assertNull;
    +import static org.junit.Assert.assertTrue;
    +
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +
    +import org.apache.phoenix.schema.TypeMismatchException;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +public class ArrayRemoveFunctionIT extends ParallelStatsDisabledIT {
    +
    +	private Connection conn;
    +	private String tableName;
    +
    +	@Before
    +	public void setup() throws Exception {
    +		conn = DriverManager.getConnection(getUrl());
    +		tableName = initTables(conn);
    +	}
    +
    +	private String initTables(Connection conn) throws Exception {
    +		String tableName = generateUniqueName();
    +		String ddl = "CREATE TABLE " + tableName
    +				+ " (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[],integers INTEGER[],doubles DOUBLE[],bigints BIGINT[],"
    +				+ "chars CHAR(15)[],double1 DOUBLE,char1 CHAR(17),nullcheck INTEGER,chars2 CHAR(15)[], nullVarchar VARCHAR[], nullBigInt BIGINT[],double2 DOUBLE,integer1 INTEGER,oneItem VARCHAR[],char2 char(15),varchar1 VARCHAR)";
    +		conn.createStatement().execute(ddl);
    +		String dml = "UPSERT INTO " + tableName
    +				+ "(region_name,varchars,integers,doubles,bigints,chars,double1,char1,nullcheck,chars2,double2,integer1,oneItem,char2,varchar1) VALUES('SF Bay Area',"
    +				+ "ARRAY['2345','46345','23234']," + "ARRAY[2345,46345,23234,456],"
    +				+ "ARRAY[10.0,23.45,46.345,23.234,45.6,5.78]," + "ARRAY[12,34,56,78,910],"
    +				+ "ARRAY['a','bbbb','c','ddd','e','c']," + "23.45," + "'wert'," + "NULL,"
    +				+ "ARRAY['a','bbbb','c','ddd','e','foo']," + "12," + "12," + "ARRAY['alone'],'2345','bbbb')";
    +		PreparedStatement stmt = conn.prepareStatement(dml);
    +		stmt.execute();
    +		conn.commit();
    +		return tableName;
    +	}
    +
    +	@Test
    +	public void testEmptyArrayModification() throws Exception {
    +		ResultSet rs = conn.createStatement()
    +				.executeQuery("SELECT ARRAY_REMOVE(nullVarChar,'34567') FROM " + tableName + " LIMIT 1");
    +		assertTrue(rs.next());
    +
    +		assertNull(rs.getArray(1));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionVarchar() throws Exception {
    +		ResultSet rs = conn.createStatement().executeQuery(
    +				"SELECT ARRAY_REMOVE(varchars,'23234') FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("VARCHAR", new String[] { "2345", "46345" }), rs.getArray(1));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionInteger() throws Exception {
    +		ResultSet rs = conn.createStatement().executeQuery(
    +				"SELECT ARRAY_REMOVE(integers,456) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("INTEGER", new Integer[] { 2345, 46345, 23234 }), rs.getArray(1));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionDouble() throws Exception {
    +		ResultSet rs = conn.createStatement().executeQuery(
    +				"SELECT ARRAY_REMOVE(doubles,double1) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("DOUBLE", new Double[] { 10.0, 46.345, 23.234, 45.6, 5.78 }), rs.getArray(1));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionDoubleWithInt() throws Exception {
    +		ResultSet rs = conn.createStatement().executeQuery(
    +				"SELECT ARRAY_REMOVE(doubles,10),doubles FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("DOUBLE", new Double[] { 23.45, 46.345, 23.234, 45.6, 5.78 }), rs.getArray(1));
    +		assertEquals(conn.createArrayOf("DOUBLE", new Double[] { 10.0, 23.45, 46.345, 23.234, 45.6, 5.78 }),
    +				rs.getArray(2));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionBigint() throws Exception {
    +		ResultSet rs = conn.createStatement().executeQuery(
    +				"SELECT ARRAY_REMOVE(bigints,56),bigints FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("BIGINT", new Long[] { 12l, 34l, 78l, 910l }), rs.getArray(1));
    +		assertEquals(conn.createArrayOf("BIGINT", new Long[] { 12l, 34l, 56l, 78l, 910l }), rs.getArray(2));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionBigintWithInteger() throws Exception {
    +		ResultSet rs = conn.createStatement().executeQuery(
    +				"SELECT ARRAY_REMOVE(bigints,integer1) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("BIGINT", new Long[] { 34l, 56l, 78l, 910l }), rs.getArray(1));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test(expected = TypeMismatchException.class)
    +	public void testArrayRemoveFunctionBigintWithDouble() throws Exception {
    +		ResultSet rs = conn.createStatement().executeQuery(
    +				"SELECT ARRAY_REMOVE(bigints,double2) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("BIGINT", new Long[] { 34l, 56l, 78l, 910l }), rs.getArray(1));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionChar() throws Exception {
    +		ResultSet rs = conn.createStatement().executeQuery(
    +				"SELECT ARRAY_REMOVE(chars,'ddd') FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("CHAR", new String[] { "a", "bbbb", "c", "e", "c" }), rs.getArray(1));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test(expected = TypeMismatchException.class)
    +	public void testArrayRemoveFunctionIntToCharArray() throws Exception {
    +		conn.createStatement().executeQuery(
    +				"SELECT ARRAY_REMOVE(varchars,234) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +	}
    +
    +	@Test(expected = TypeMismatchException.class)
    +	public void testArrayRemoveFunctionVarcharToIntegerArray() throws Exception {
    +		conn.createStatement().executeQuery(
    +				"SELECT ARRAY_REMOVE(integers,'234') FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionWithNestedFunctions1() throws Exception {
    +		ResultSet rs = conn.createStatement().executeQuery("SELECT ARRAY_REMOVE(ARRAY[23,2345],integers[1]) FROM "
    +				+ tableName + " WHERE region_name = 'SF Bay Area'");
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("INTEGER", new Integer[] { 23 }), rs.getArray(1));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionWithNestedFunctions2() throws Exception {
    +		ResultSet rs = conn.createStatement()
    +				.executeQuery("SELECT ARRAY_REMOVE(integers,ARRAY_ELEM(ARRAY[2345,4],1)) FROM " + tableName
    +						+ " WHERE region_name = 'SF Bay Area'");
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("INTEGER", new Integer[] { 46345, 23234, 456 }), rs.getArray(1));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionWithUpsert1() throws Exception {
    +		String uniqueName = generateUniqueName();
    +		String ddl = "CREATE TABLE " + uniqueName + " (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[])";
    +		conn.createStatement().execute(ddl);
    +
    +		String dml = "UPSERT INTO " + uniqueName
    +				+ "(region_name,varchars) VALUES('SF Bay Area',ARRAY_REMOVE(ARRAY['hello','world'],'world'))";
    +		conn.createStatement().execute(dml);
    +		conn.commit();
    +
    +		ResultSet rs = conn.createStatement()
    +				.executeQuery("SELECT varchars FROM " + uniqueName + " WHERE region_name = 'SF Bay Area'");
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("VARCHAR", new String[] { "hello" }), rs.getArray(1));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionWithUpsert2() throws Exception {
    +		String tableName = generateUniqueName();
    +		String ddl = "CREATE TABLE " + tableName + " (region_name VARCHAR PRIMARY KEY,integers INTEGER[])";
    +		conn.createStatement().execute(ddl);
    +
    +		String dml = "UPSERT INTO " + tableName
    +				+ "(region_name,integers) VALUES('SF Bay Area',ARRAY_REMOVE(ARRAY[4,5],5))";
    +		conn.createStatement().execute(dml);
    +		conn.commit();
    +
    +		ResultSet rs = conn.createStatement()
    +				.executeQuery("SELECT integers FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("INTEGER", new Integer[] { 4 }), rs.getArray(1));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionWithUpsertSelect1() throws Exception {
    +		String sourceTableName = generateUniqueName();
    +		String targetTableName = generateUniqueName();
    +
    +		String ddl = "CREATE TABLE " + sourceTableName + " (region_name VARCHAR PRIMARY KEY,doubles DOUBLE[])";
    +		conn.createStatement().execute(ddl);
    +
    +		ddl = "CREATE TABLE " + targetTableName + " (region_name VARCHAR PRIMARY KEY,doubles DOUBLE[])";
    +		conn.createStatement().execute(ddl);
    +
    +		String dml = "UPSERT INTO " + sourceTableName
    +				+ "(region_name,doubles) VALUES('SF Bay Area',ARRAY_APPEND(ARRAY[5.67,7.87],9))";
    +		conn.createStatement().execute(dml);
    +
    +		dml = "UPSERT INTO " + sourceTableName
    +				+ "(region_name,doubles) VALUES('SF Bay Area2',ARRAY_APPEND(ARRAY[56.7,7.87],9))";
    +		conn.createStatement().execute(dml);
    +		conn.commit();
    +
    +		dml = "UPSERT INTO " + targetTableName
    +				+ "(region_name, doubles) SELECT region_name, ARRAY_REMOVE(doubles,9) FROM " + sourceTableName;
    +		conn.createStatement().execute(dml);
    +		conn.commit();
    +
    +		ResultSet rs = conn.createStatement().executeQuery("SELECT doubles FROM " + targetTableName);
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("DOUBLE", new Double[] { 5.67, 7.87 }), rs.getArray(1));
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("DOUBLE", new Double[] { 56.7, 7.87 }), rs.getArray(1));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionInWhere1() throws Exception {
    +		ResultSet rs = conn.createStatement().executeQuery(
    +				"SELECT region_name FROM " + tableName + " WHERE ARRAY[2345,46345,23234]=ARRAY_REMOVE(integers,456)");
    +		assertTrue(rs.next());
    +
    +		assertEquals("SF Bay Area", rs.getString(1));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionVarcharWithNull() throws Exception {
    +		ResultSet rs = conn.createStatement().executeQuery(
    +				"SELECT ARRAY_REMOVE(varchars,NULL) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("VARCHAR", new String[] { "2345", "46345", "23234" }), rs.getArray(1));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionDoublesWithNull() throws Exception {
    +		ResultSet rs = conn.createStatement().executeQuery(
    +				"SELECT ARRAY_REMOVE(doubles,NULL) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("DOUBLE", new Double[] { 10.0, 23.45, 46.345, 23.234, 45.6, 5.78 }),
    +				rs.getArray(1));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionCharsWithNull() throws Exception {
    +		ResultSet rs = conn.createStatement().executeQuery(
    +				"SELECT ARRAY_REMOVE(chars,NULL) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("CHAR", new String[] { "a", "bbbb", "c", "ddd", "e", "c" }), rs.getArray(1));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionWithNull() throws Exception {
    +		ResultSet rs = conn.createStatement().executeQuery(
    +				"SELECT ARRAY_REMOVE(integers,nullcheck) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("INTEGER", new Integer[] { 2345, 46345, 23234, 456 }), rs.getArray(1));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionFirstElement() throws Exception {
    +		ResultSet rs = conn.createStatement().executeQuery(
    +				"SELECT ARRAY_REMOVE(varchars,'2345') FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("VARCHAR", new String[] { "46345", "23234" }), rs.getArray(1));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionMiddleElement() throws Exception {
    +		ResultSet rs = conn.createStatement().executeQuery(
    +				"SELECT ARRAY_REMOVE(varchars,'46345') FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("VARCHAR", new String[] { "2345", "23234" }), rs.getArray(1));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionLastElement() throws Exception {
    +		ResultSet rs = conn.createStatement().executeQuery(
    +				"SELECT ARRAY_REMOVE(varchars,'23234') FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("VARCHAR", new String[] { "2345", "46345" }), rs.getArray(1));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionOneElement() throws Exception {
    +		ResultSet rs = conn.createStatement().executeQuery(
    +				"SELECT ARRAY_REMOVE(oneItem,'alone') FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("VARCHAR", new String[0]), rs.getArray(1));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionRepeatingElements() throws Exception {
    +		ResultSet rs = conn.createStatement().executeQuery(
    +				"SELECT ARRAY_REMOVE(chars,'c') FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("CHAR", new String[] { "a", "bbbb", "ddd", "e" }), rs.getArray(1));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionCharFromVarcharArray() throws Exception {
    +		ResultSet rs = conn.createStatement().executeQuery(
    +				"SELECT ARRAY_REMOVE(varchars,char2) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("VARCHAR", new String[] { "46345", "23234" }), rs.getArray(1));
    +		assertFalse(rs.next());
    +	}
    +
    +	@Test
    +	public void testArrayRemoveFunctionVarcharFromCharArray() throws Exception {
    +		ResultSet rs = conn.createStatement().executeQuery(
    +				"SELECT ARRAY_REMOVE(chars,varchar1) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
    +		assertTrue(rs.next());
    +
    +		assertEquals(conn.createArrayOf("CHAR", new String[] { "a", "c", "ddd", "e", "c" }), rs.getArray(1));
    --- End diff --
    
    Instead of removing "bbbb" which is the longest element so won't have trailing spaces, try removing "a" or "c" as it's a better test.


---

[GitHub] phoenix pull request #294: PHOENIX-4643 Implement ARRAY_REMOVE built in func...

Posted by xjodoin <gi...@git.apache.org>.
Github user xjodoin commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/294#discussion_r172642555
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayRemoveFunction.java ---
    @@ -0,0 +1,102 @@
    +/*
    + * 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.expression.function;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.commons.lang.ArrayUtils;
    +import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
    +import org.apache.phoenix.expression.Expression;
    +import org.apache.phoenix.parse.FunctionParseNode;
    +import org.apache.phoenix.schema.SortOrder;
    +import org.apache.phoenix.schema.TypeMismatchException;
    +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;
    +import org.apache.phoenix.schema.types.PhoenixArray;
    +
    +@FunctionParseNode.BuiltInFunction(name = ArrayRemoveFunction.NAME, args = {
    +		@FunctionParseNode.Argument(allowedTypes = { PBinaryArray.class, PVarbinaryArray.class }),
    +		@FunctionParseNode.Argument(allowedTypes = { PVarbinary.class }, defaultValue = "null") })
    +public class ArrayRemoveFunction extends ArrayModifierFunction {
    +
    +	public static final String NAME = "ARRAY_REMOVE";
    +
    +	public ArrayRemoveFunction() {
    +	}
    +
    +	public ArrayRemoveFunction(List<Expression> children) throws TypeMismatchException {
    +		super(children);
    +	}
    +
    +	@Override
    +	protected boolean modifierFunction(ImmutableBytesWritable ptr, int length, int offset, byte[] arrayBytes,
    +			PDataType baseType, int arrayLength, Integer maxLength, Expression arrayExp) {
    +		SortOrder sortOrder = arrayExp.getSortOrder();
    +
    +		if (ptr.getLength() == 0 || arrayBytes.length == 0) {
    +			ptr.set(arrayBytes, offset, length);
    +			return true;
    +		}
    +
    +		PhoenixArray array = (PhoenixArray) getDataType().toObject(arrayBytes, offset, length, getDataType(), sortOrder,
    +				maxLength, null);
    +
    +		
    +		Object toCompare = baseType.toObject(ptr, sortOrder);
    +		
    +		if(baseType.isFixedWidth()) {
    +			toCompare = baseType.pad(toCompare, array.getMaxLength());
    +		}
    +
    +		int dimensions = array.getDimensions();
    +		List<Object> values = new ArrayList<>();
    +		for (int i = 0; i < dimensions; i++) {
    +			Object element = array.getElement(i);
    +			if (element != null && element.equals(toCompare)
    +					|| (element.getClass().isArray() && ArrayUtils.isEquals(element, toCompare))) {
    +				if (getDataType().isFixedWidth()) {
    --- End diff --
    
    fix


---

[GitHub] phoenix pull request #294: PHOENIX-4643 Implement ARRAY_REMOVE built in func...

Posted by xjodoin <gi...@git.apache.org>.
Github user xjodoin commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/294#discussion_r173839154
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayRemoveFunction.java ---
    @@ -0,0 +1,79 @@
    +/*
    + * 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.expression.function;
    +
    +import java.util.List;
    +
    +import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
    +import org.apache.phoenix.expression.Expression;
    +import org.apache.phoenix.parse.FunctionParseNode;
    +import org.apache.phoenix.schema.SortOrder;
    +import org.apache.phoenix.schema.TypeMismatchException;
    +import org.apache.phoenix.schema.types.PArrayDataTypeDecoder;
    +import org.apache.phoenix.schema.types.PArrayDataTypeEncoder;
    +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 = ArrayRemoveFunction.NAME, args = {
    +		@FunctionParseNode.Argument(allowedTypes = { PBinaryArray.class, PVarbinaryArray.class }),
    +		@FunctionParseNode.Argument(allowedTypes = { PVarbinary.class }, defaultValue = "null") })
    +public class ArrayRemoveFunction extends ArrayModifierFunction {
    +
    +	public static final String NAME = "ARRAY_REMOVE";
    +
    +	public ArrayRemoveFunction() {
    +	}
    +
    +	public ArrayRemoveFunction(List<Expression> children) throws TypeMismatchException {
    +		super(children);
    +	}
    +
    +	@Override
    +	protected boolean modifierFunction(ImmutableBytesWritable ptr, int length, int offset, byte[] arrayBytes,
    +			PDataType baseType, int arrayLength, Integer maxLength, Expression arrayExp) {
    +		SortOrder sortOrder = arrayExp.getSortOrder();
    +
    +		if (ptr.getLength() == 0 || arrayBytes.length == 0) {
    +			ptr.set(arrayBytes, offset, length);
    +			return true;
    +		}
    +
    +		PArrayDataTypeEncoder arrayDataTypeEncoder = new PArrayDataTypeEncoder(baseType, sortOrder);
    +
    +		for (int arrayIndex = 0; arrayIndex < arrayLength; arrayIndex++) {
    +			ImmutableBytesWritable ptr2 = new ImmutableBytesWritable(arrayBytes, offset, length);
    +			PArrayDataTypeDecoder.positionAtArrayElement(ptr2, arrayIndex, baseType, maxLength);
    +			if (baseType.compareTo(ptr2, sortOrder, ptr, sortOrder, baseType) != 0) {
    --- End diff --
    
    it's works I added some tests to validate it


---

[GitHub] phoenix issue #294: PHOENIX-4643 Implement ARRAY_REMOVE built in function

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on the issue:

    https://github.com/apache/phoenix/pull/294
  
    Thanks for the pull request, @xjodoin. Looks very good overall - I made a few specific comments on the implementation of the built-in function.


---

[GitHub] phoenix pull request #294: PHOENIX-4643 Implement ARRAY_REMOVE built in func...

Posted by xjodoin <gi...@git.apache.org>.
Github user xjodoin commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/294#discussion_r175558050
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayRemoveFunctionIT.java ---
    @@ -0,0 +1,425 @@
    +/*
    + * 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.end2end;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.assertNull;
    +import static org.junit.Assert.assertTrue;
    +
    +import java.sql.Array;
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +
    +import org.apache.phoenix.schema.TypeMismatchException;
    +import org.junit.Test;
    +
    +public class ArrayRemoveFunctionIT extends ParallelStatsDisabledIT {
    +    private String initTables(Connection conn) throws Exception {
    +        String tableName = generateUniqueName();
    +        String ddl = "CREATE TABLE " + tableName
    +                + " (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[],integers INTEGER[],doubles DOUBLE[],bigints BIGINT[],"
    +                + "chars CHAR(15)[],double1 DOUBLE,char1 CHAR(17),nullcheck INTEGER,chars2 CHAR(15)[], nullVarchar VARCHAR[], nullBigInt BIGINT[],double2 DOUBLE,integer1 INTEGER)";
    +        conn.createStatement().execute(ddl);
    +        String dml = "UPSERT INTO " + tableName + "(region_name,varchars,integers,doubles,bigints,chars,double1,char1,nullcheck,chars2,double2,integer1) VALUES('SF Bay Area'," +
    +                "ARRAY['2345','46345','23234']," +
    +                "ARRAY[2345,46345,23234,456]," +
    +                "ARRAY[23.45,46.345,23.234,45.6,5.78]," +
    +                "ARRAY[12,34,56,78,910]," +
    +                "ARRAY['a','bbbb','c','ddd','e']," +
    +                "23.45," +
    +                "'wert'," +
    +                "NULL," +
    +                "ARRAY['a','bbbb','c','ddd','e','foo']," +
    +                "12,"+
    +                "12"+
    +                ")";
    +        PreparedStatement stmt = conn.prepareStatement(dml);
    +        stmt.execute();
    +        conn.commit();
    +        return tableName;
    +    }
    +
    --- End diff --
    
    I added missing tests, and I think it should remove all matching element from the array


---

[GitHub] phoenix pull request #294: PHOENIX-4643 Implement ARRAY_REMOVE built in func...

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/294#discussion_r175226865
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayRemoveFunctionIT.java ---
    @@ -0,0 +1,425 @@
    +/*
    + * 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.end2end;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.assertNull;
    +import static org.junit.Assert.assertTrue;
    +
    +import java.sql.Array;
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +
    +import org.apache.phoenix.schema.TypeMismatchException;
    +import org.junit.Test;
    +
    +public class ArrayRemoveFunctionIT extends ParallelStatsDisabledIT {
    +    private String initTables(Connection conn) throws Exception {
    +        String tableName = generateUniqueName();
    +        String ddl = "CREATE TABLE " + tableName
    +                + " (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[],integers INTEGER[],doubles DOUBLE[],bigints BIGINT[],"
    +                + "chars CHAR(15)[],double1 DOUBLE,char1 CHAR(17),nullcheck INTEGER,chars2 CHAR(15)[], nullVarchar VARCHAR[], nullBigInt BIGINT[],double2 DOUBLE,integer1 INTEGER)";
    +        conn.createStatement().execute(ddl);
    +        String dml = "UPSERT INTO " + tableName + "(region_name,varchars,integers,doubles,bigints,chars,double1,char1,nullcheck,chars2,double2,integer1) VALUES('SF Bay Area'," +
    +                "ARRAY['2345','46345','23234']," +
    +                "ARRAY[2345,46345,23234,456]," +
    +                "ARRAY[23.45,46.345,23.234,45.6,5.78]," +
    +                "ARRAY[12,34,56,78,910]," +
    +                "ARRAY['a','bbbb','c','ddd','e']," +
    +                "23.45," +
    +                "'wert'," +
    +                "NULL," +
    +                "ARRAY['a','bbbb','c','ddd','e','foo']," +
    +                "12,"+
    +                "12"+
    +                ")";
    +        PreparedStatement stmt = conn.prepareStatement(dml);
    +        stmt.execute();
    +        conn.commit();
    +        return tableName;
    +    }
    +
    --- End diff --
    
    Add tests for:
    * removing first array element 
    * removing last array element
    * removing middle array element 
    * removing only element in array 
    * removing element which repeats (what should happen?)
    * removing CHAR(10) from VARCHAR
    * removing VARCHAR from CHAR(10)


---

[GitHub] phoenix issue #294: PHOENIX-4643 Implement ARRAY_REMOVE built in function

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on the issue:

    https://github.com/apache/phoenix/pull/294
  
    Looks good - one minor nit and then I think we'll be good to go.


---

[GitHub] phoenix pull request #294: PHOENIX-4643 Implement ARRAY_REMOVE built in func...

Posted by xjodoin <gi...@git.apache.org>.
Github user xjodoin closed the pull request at:

    https://github.com/apache/phoenix/pull/294


---

[GitHub] phoenix pull request #294: PHOENIX-4643 Implement ARRAY_REMOVE built in func...

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/294#discussion_r172606335
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayRemoveFunction.java ---
    @@ -0,0 +1,102 @@
    +/*
    + * 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.expression.function;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.commons.lang.ArrayUtils;
    +import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
    +import org.apache.phoenix.expression.Expression;
    +import org.apache.phoenix.parse.FunctionParseNode;
    +import org.apache.phoenix.schema.SortOrder;
    +import org.apache.phoenix.schema.TypeMismatchException;
    +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;
    +import org.apache.phoenix.schema.types.PhoenixArray;
    +
    +@FunctionParseNode.BuiltInFunction(name = ArrayRemoveFunction.NAME, args = {
    +		@FunctionParseNode.Argument(allowedTypes = { PBinaryArray.class, PVarbinaryArray.class }),
    +		@FunctionParseNode.Argument(allowedTypes = { PVarbinary.class }, defaultValue = "null") })
    +public class ArrayRemoveFunction extends ArrayModifierFunction {
    +
    +	public static final String NAME = "ARRAY_REMOVE";
    +
    +	public ArrayRemoveFunction() {
    +	}
    +
    +	public ArrayRemoveFunction(List<Expression> children) throws TypeMismatchException {
    +		super(children);
    +	}
    +
    +	@Override
    +	protected boolean modifierFunction(ImmutableBytesWritable ptr, int length, int offset, byte[] arrayBytes,
    +			PDataType baseType, int arrayLength, Integer maxLength, Expression arrayExp) {
    +		SortOrder sortOrder = arrayExp.getSortOrder();
    +
    +		if (ptr.getLength() == 0 || arrayBytes.length == 0) {
    +			ptr.set(arrayBytes, offset, length);
    +			return true;
    +		}
    +
    +		PhoenixArray array = (PhoenixArray) getDataType().toObject(arrayBytes, offset, length, getDataType(), sortOrder,
    +				maxLength, null);
    +
    +		
    +		Object toCompare = baseType.toObject(ptr, sortOrder);
    +		
    +		if(baseType.isFixedWidth()) {
    +			toCompare = baseType.pad(toCompare, array.getMaxLength());
    +		}
    +
    +		int dimensions = array.getDimensions();
    +		List<Object> values = new ArrayList<>();
    +		for (int i = 0; i < dimensions; i++) {
    +			Object element = array.getElement(i);
    +			if (element != null && element.equals(toCompare)
    +					|| (element.getClass().isArray() && ArrayUtils.isEquals(element, toCompare))) {
    --- End diff --
    
    We don't support arrays as elements of arrays, so I don't think you need this check.


---

[GitHub] phoenix pull request #294: PHOENIX-4643 Implement ARRAY_REMOVE built in func...

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/294#discussion_r172604371
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayRemoveFunction.java ---
    @@ -0,0 +1,102 @@
    +/*
    + * 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.expression.function;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.commons.lang.ArrayUtils;
    +import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
    +import org.apache.phoenix.expression.Expression;
    +import org.apache.phoenix.parse.FunctionParseNode;
    +import org.apache.phoenix.schema.SortOrder;
    +import org.apache.phoenix.schema.TypeMismatchException;
    +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;
    +import org.apache.phoenix.schema.types.PhoenixArray;
    +
    +@FunctionParseNode.BuiltInFunction(name = ArrayRemoveFunction.NAME, args = {
    +		@FunctionParseNode.Argument(allowedTypes = { PBinaryArray.class, PVarbinaryArray.class }),
    +		@FunctionParseNode.Argument(allowedTypes = { PVarbinary.class }, defaultValue = "null") })
    +public class ArrayRemoveFunction extends ArrayModifierFunction {
    +
    +	public static final String NAME = "ARRAY_REMOVE";
    +
    +	public ArrayRemoveFunction() {
    +	}
    +
    +	public ArrayRemoveFunction(List<Expression> children) throws TypeMismatchException {
    +		super(children);
    +	}
    +
    +	@Override
    +	protected boolean modifierFunction(ImmutableBytesWritable ptr, int length, int offset, byte[] arrayBytes,
    +			PDataType baseType, int arrayLength, Integer maxLength, Expression arrayExp) {
    +		SortOrder sortOrder = arrayExp.getSortOrder();
    +
    +		if (ptr.getLength() == 0 || arrayBytes.length == 0) {
    +			ptr.set(arrayBytes, offset, length);
    +			return true;
    +		}
    +
    +		PhoenixArray array = (PhoenixArray) getDataType().toObject(arrayBytes, offset, length, getDataType(), sortOrder,
    +				maxLength, null);
    --- End diff --
    
    This will potentially use a lot of memory for large arrays. It'd also be hit with the deserialization cost for every element. An alternative would be to make one pass through the binary array data and serialize the values as you go, skipping an element if it's equal to the one being removed.


---

[GitHub] phoenix issue #294: PHOENIX-4643 Implement ARRAY_REMOVE built in function

Posted by xjodoin <gi...@git.apache.org>.
Github user xjodoin commented on the issue:

    https://github.com/apache/phoenix/pull/294
  
    I did the last change. I hope the feature is ready :)


---

[GitHub] phoenix pull request #294: PHOENIX-4643 Implement ARRAY_REMOVE built in func...

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/294#discussion_r173612840
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayRemoveFunction.java ---
    @@ -0,0 +1,79 @@
    +/*
    + * 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.expression.function;
    +
    +import java.util.List;
    +
    +import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
    +import org.apache.phoenix.expression.Expression;
    +import org.apache.phoenix.parse.FunctionParseNode;
    +import org.apache.phoenix.schema.SortOrder;
    +import org.apache.phoenix.schema.TypeMismatchException;
    +import org.apache.phoenix.schema.types.PArrayDataTypeDecoder;
    +import org.apache.phoenix.schema.types.PArrayDataTypeEncoder;
    +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 = ArrayRemoveFunction.NAME, args = {
    +		@FunctionParseNode.Argument(allowedTypes = { PBinaryArray.class, PVarbinaryArray.class }),
    +		@FunctionParseNode.Argument(allowedTypes = { PVarbinary.class }, defaultValue = "null") })
    +public class ArrayRemoveFunction extends ArrayModifierFunction {
    +
    +	public static final String NAME = "ARRAY_REMOVE";
    +
    +	public ArrayRemoveFunction() {
    +	}
    +
    +	public ArrayRemoveFunction(List<Expression> children) throws TypeMismatchException {
    +		super(children);
    +	}
    +
    +	@Override
    +	protected boolean modifierFunction(ImmutableBytesWritable ptr, int length, int offset, byte[] arrayBytes,
    +			PDataType baseType, int arrayLength, Integer maxLength, Expression arrayExp) {
    +		SortOrder sortOrder = arrayExp.getSortOrder();
    +
    +		if (ptr.getLength() == 0 || arrayBytes.length == 0) {
    +			ptr.set(arrayBytes, offset, length);
    +			return true;
    +		}
    +
    +		PArrayDataTypeEncoder arrayDataTypeEncoder = new PArrayDataTypeEncoder(baseType, sortOrder);
    +
    +		for (int arrayIndex = 0; arrayIndex < arrayLength; arrayIndex++) {
    +			ImmutableBytesWritable ptr2 = new ImmutableBytesWritable(arrayBytes, offset, length);
    +			PArrayDataTypeDecoder.positionAtArrayElement(ptr2, arrayIndex, baseType, maxLength);
    +			if (baseType.compareTo(ptr2, sortOrder, ptr, sortOrder, baseType) != 0) {
    --- End diff --
    
    This looks good. Can you make sure you have tests around removing an element from an array where the element type is slightly different than the array element type? For example, removing an int from a long array or removing a long from a decimal array?


---

[GitHub] phoenix pull request #294: PHOENIX-4643 Implement ARRAY_REMOVE built in func...

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/294#discussion_r172711644
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayRemoveFunction.java ---
    @@ -0,0 +1,79 @@
    +/*
    + * 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.expression.function;
    +
    +import java.util.List;
    +
    +import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
    +import org.apache.phoenix.expression.Expression;
    +import org.apache.phoenix.parse.FunctionParseNode;
    +import org.apache.phoenix.schema.SortOrder;
    +import org.apache.phoenix.schema.TypeMismatchException;
    +import org.apache.phoenix.schema.types.PArrayDataTypeDecoder;
    +import org.apache.phoenix.schema.types.PArrayDataTypeEncoder;
    +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 = ArrayRemoveFunction.NAME, args = {
    +		@FunctionParseNode.Argument(allowedTypes = { PBinaryArray.class, PVarbinaryArray.class }),
    +		@FunctionParseNode.Argument(allowedTypes = { PVarbinary.class }, defaultValue = "null") })
    --- End diff --
    
    I filed PHOENIX-4644 with a patch so that you can get rid of this defaultValue. You'll also need to add the nodeClassName attribute here as you see for ARRAY_APPEND.


---

[GitHub] phoenix issue #294: PHOENIX-4643 Implement ARRAY_REMOVE built in function

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on the issue:

    https://github.com/apache/phoenix/pull/294
  
    +1. Thanks for the contribution, @xjodoin. I'll commit this to the various branches.


---