Return-Path: X-Original-To: apmail-phoenix-commits-archive@minotaur.apache.org Delivered-To: apmail-phoenix-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id C5A7A184DB for ; Wed, 5 Aug 2015 17:48:40 +0000 (UTC) Received: (qmail 76835 invoked by uid 500); 5 Aug 2015 17:48:40 -0000 Delivered-To: apmail-phoenix-commits-archive@phoenix.apache.org Received: (qmail 76796 invoked by uid 500); 5 Aug 2015 17:48:40 -0000 Mailing-List: contact commits-help@phoenix.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@phoenix.apache.org Delivered-To: mailing list commits@phoenix.apache.org Received: (qmail 76787 invoked by uid 99); 5 Aug 2015 17:48:40 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 05 Aug 2015 17:48:40 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 74EC7DFBD5; Wed, 5 Aug 2015 17:48:40 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: samarth@apache.org To: commits@phoenix.apache.org Message-Id: <7f27c6f3b9d045c48720fe9cf3dcd18f@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: phoenix git commit: PHOENIX-2140 Implement get_byte/set_byte/get_bit/set_bit Binary String build-in function (Shuxiong Ye) Date: Wed, 5 Aug 2015 17:48:40 +0000 (UTC) Repository: phoenix Updated Branches: refs/heads/4.x-HBase-0.98 0410dde88 -> b985a4a3f PHOENIX-2140 Implement get_byte/set_byte/get_bit/set_bit Binary String build-in function (Shuxiong Ye) Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/b985a4a3 Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/b985a4a3 Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/b985a4a3 Branch: refs/heads/4.x-HBase-0.98 Commit: b985a4a3fe1083f892517222035112ecdfde0f57 Parents: 0410dde Author: Samarth Authored: Wed Aug 5 10:48:24 2015 -0700 Committer: Samarth Committed: Wed Aug 5 10:48:24 2015 -0700 ---------------------------------------------------------------------- .../end2end/GetSetByteBitFunctionEnd2EndIT.java | 100 ++++++++++ .../phoenix/expression/ExpressionType.java | 10 +- .../expression/function/GetBitFunction.java | 96 ++++++++++ .../expression/function/GetByteFunction.java | 96 ++++++++++ .../expression/function/SetBitFunction.java | 81 ++++++++ .../expression/function/SetByteFunction.java | 81 ++++++++ .../apache/phoenix/schema/types/PBinary.java | 2 +- .../phoenix/schema/types/PBinaryBase.java | 86 +++++++++ .../apache/phoenix/schema/types/PVarbinary.java | 2 +- .../phoenix/compile/WhereOptimizerTest.java | 33 +++- .../expression/GetSetByteBitFunctionTest.java | 189 +++++++++++++++++++ .../java/org/apache/phoenix/query/BaseTest.java | 8 + .../java/org/apache/phoenix/util/TestUtil.java | 1 + 13 files changed, 780 insertions(+), 5 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/phoenix/blob/b985a4a3/phoenix-core/src/it/java/org/apache/phoenix/end2end/GetSetByteBitFunctionEnd2EndIT.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/GetSetByteBitFunctionEnd2EndIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/GetSetByteBitFunctionEnd2EndIT.java new file mode 100644 index 0000000..a7f7ca3 --- /dev/null +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/GetSetByteBitFunctionEnd2EndIT.java @@ -0,0 +1,100 @@ +/* + * 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.apache.phoenix.util.TestUtil.closeStmtAndConn; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +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.expression.function.GetBitFunction; +import org.apache.phoenix.expression.function.GetByteFunction; +import org.apache.phoenix.expression.function.SetBitFunction; +import org.apache.phoenix.expression.function.SetByteFunction; +import org.junit.Before; +import org.junit.Test; + +/** + * End to end tests for {@link GetByteFunction} {@link SetByteFunction} {@link GetBitFunction} + * {@link SetBitFunction} + */ +public class GetSetByteBitFunctionEnd2EndIT extends BaseHBaseManagedTimeIT { + + private static final String KEY = "key"; + + @Before + public void initTable() throws Exception { + Connection conn = null; + PreparedStatement stmt = null; + try { + conn = DriverManager.getConnection(getUrl()); + String ddl; + ddl = "CREATE TABLE ta (k VARCHAR NOT NULL PRIMARY KEY, b BINARY(4), vb VARBINARY)"; + conn.createStatement().execute(ddl); + conn.commit(); + } finally { + closeStmtAndConn(stmt, conn); + } + } + + @Test + public void test() throws Exception { + Connection conn = DriverManager.getConnection(getUrl()); + PreparedStatement stmt = conn.prepareStatement("UPSERT INTO ta VALUES (?, ?, ?)"); + stmt.setString(1, KEY); + stmt.setBytes(2, new byte[] { 1, 2, 3, 4 }); + stmt.setBytes(3, new byte[] { 1, 2, 3, 4 }); + stmt.executeUpdate(); + conn.commit(); + ResultSet rs = + conn.createStatement() + .executeQuery( + "SELECT GET_BYTE(vb, 1), GET_BYTE(b, 1) FROM ta WHERE GET_BYTE(vb, 1)=2 and GET_BYTE(b, 1)=2"); + assertTrue(rs.next()); + assertEquals(2, rs.getInt(1)); + assertEquals(2, rs.getInt(2)); + assertTrue(!rs.next()); + rs = + conn.createStatement() + .executeQuery( + "SELECT GET_BIT(b, 0),GET_BIT(b, 9) FROM ta WHERE GET_BIT(vb, 0)=1 and GET_BIT(vb, 9)=1"); + assertTrue(rs.next()); + assertEquals(1, rs.getInt(1)); + assertEquals(1, rs.getInt(2)); + assertTrue(!rs.next()); + rs = + conn.createStatement().executeQuery( + "SELECT SET_BYTE(vb, 1, 1), SET_BYTE(b, 1, 1) FROM ta"); + assertTrue(rs.next()); + assertArrayEquals(new byte[] { 1, 1, 3, 4 }, rs.getBytes(1)); + assertArrayEquals(new byte[] { 1, 1, 3, 4 }, rs.getBytes(2)); + assertTrue(!rs.next()); + rs = + conn.createStatement().executeQuery( + "SELECT SET_BIT(vb, 8, 1), SET_BIT(b, 8, 1) FROM ta"); + assertTrue(rs.next()); + assertArrayEquals(new byte[] { 1, 3, 3, 4 }, rs.getBytes(1)); + assertArrayEquals(new byte[] { 1, 3, 3, 4 }, rs.getBytes(2)); + assertTrue(!rs.next()); + } +} http://git-wip-us.apache.org/repos/asf/phoenix/blob/b985a4a3/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java index 53d77ef..2089b6c 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java @@ -52,6 +52,8 @@ import org.apache.phoenix.expression.function.FirstValueFunction; import org.apache.phoenix.expression.function.FloorDateExpression; import org.apache.phoenix.expression.function.FloorDecimalExpression; import org.apache.phoenix.expression.function.FloorFunction; +import org.apache.phoenix.expression.function.GetBitFunction; +import org.apache.phoenix.expression.function.GetByteFunction; import org.apache.phoenix.expression.function.HourFunction; import org.apache.phoenix.expression.function.IndexStateNameFunction; import org.apache.phoenix.expression.function.InstrFunction; @@ -88,6 +90,8 @@ import org.apache.phoenix.expression.function.SQLIndexTypeFunction; import org.apache.phoenix.expression.function.SQLTableTypeFunction; import org.apache.phoenix.expression.function.SQLViewTypeFunction; import org.apache.phoenix.expression.function.SecondFunction; +import org.apache.phoenix.expression.function.SetBitFunction; +import org.apache.phoenix.expression.function.SetByteFunction; import org.apache.phoenix.expression.function.SignFunction; import org.apache.phoenix.expression.function.SqlTypeNameFunction; import org.apache.phoenix.expression.function.SqrtFunction; @@ -252,7 +256,11 @@ public enum ExpressionType { ArrayConcatFunction(ArrayConcatFunction.class), ArrayFillFunction(ArrayFillFunction.class), ArrayToStringFunction(ArrayToStringFunction.class), - StringToArrayFunction(StringToArrayFunction.class) + StringToArrayFunction(StringToArrayFunction.class), + GetByteFunction(GetByteFunction.class), + SetByteFunction(SetByteFunction.class), + GetBitFunction(GetBitFunction.class), + SetBitFunction(SetBitFunction.class) ; ExpressionType(Class clazz) { this.clazz = clazz; http://git-wip-us.apache.org/repos/asf/phoenix/blob/b985a4a3/phoenix-core/src/main/java/org/apache/phoenix/expression/function/GetBitFunction.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/GetBitFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/GetBitFunction.java new file mode 100644 index 0000000..011841f --- /dev/null +++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/GetBitFunction.java @@ -0,0 +1,96 @@ +/* + * 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.sql.SQLException; +import java.util.List; + +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.phoenix.expression.Determinism; +import org.apache.phoenix.expression.Expression; +import org.apache.phoenix.parse.FunctionParseNode.Argument; +import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction; +import org.apache.phoenix.schema.tuple.Tuple; +import org.apache.phoenix.schema.types.PBinary; +import org.apache.phoenix.schema.types.PBinaryBase; +import org.apache.phoenix.schema.types.PDataType; +import org.apache.phoenix.schema.types.PInteger; +import org.apache.phoenix.schema.types.PVarbinary; + +@BuiltInFunction(name = GetBitFunction.NAME, args = { + @Argument(allowedTypes = { PBinary.class, PVarbinary.class }), + @Argument(allowedTypes = { PInteger.class }) }) +public class GetBitFunction extends PrefixFunction { + + public static final String NAME = "GET_BIT"; + + private Integer offsetPreCompute; + + public GetBitFunction() { + } + + public GetBitFunction(List children) throws SQLException { + super(children); + init(); + } + + @Override + public String getName() { + return NAME; + } + + @Override + public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) { + // get offset parameter + int offset; + if (offsetPreCompute == null) { + Expression offsetExpr = children.get(1); + if (!offsetExpr.evaluate(tuple, ptr)) return false; + offset = (Integer) PInteger.INSTANCE.toObject(ptr, offsetExpr.getSortOrder()); + } else offset = offsetPreCompute; + // get binary data parameter + Expression dataExpr = children.get(0); + if (!dataExpr.evaluate(tuple, ptr)) return false; + if (ptr.getLength() == 0) return true; + int len = ptr.getLength() * Byte.SIZE; + offset = (offset % len + len) % len; + // set result + ((PBinaryBase) dataExpr.getDataType()).getBit(ptr, dataExpr.getSortOrder(), offset, ptr); + return true; + } + + @Override + public PDataType getDataType() { + return PInteger.INSTANCE; + } + + private void init() { + Expression offsetExpr = children.get(1); + ImmutableBytesWritable ptr = new ImmutableBytesWritable(); + if (offsetExpr.isStateless() && offsetExpr.getDeterminism() == Determinism.ALWAYS + && offsetExpr.evaluate(null, ptr)) { + offsetPreCompute = (Integer) PInteger.INSTANCE.toObject(ptr, offsetExpr.getSortOrder()); + } else offsetPreCompute = null; + } + + @Override + public OrderPreserving preservesOrder() { + return (offsetPreCompute != null && offsetPreCompute == 0) ? OrderPreserving.YES_IF_LAST + : OrderPreserving.NO; + } +} http://git-wip-us.apache.org/repos/asf/phoenix/blob/b985a4a3/phoenix-core/src/main/java/org/apache/phoenix/expression/function/GetByteFunction.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/GetByteFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/GetByteFunction.java new file mode 100644 index 0000000..8e455ae --- /dev/null +++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/GetByteFunction.java @@ -0,0 +1,96 @@ +/* + * 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.sql.SQLException; +import java.util.List; + +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.phoenix.expression.Determinism; +import org.apache.phoenix.expression.Expression; +import org.apache.phoenix.parse.FunctionParseNode.Argument; +import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction; +import org.apache.phoenix.schema.tuple.Tuple; +import org.apache.phoenix.schema.types.PBinary; +import org.apache.phoenix.schema.types.PBinaryBase; +import org.apache.phoenix.schema.types.PDataType; +import org.apache.phoenix.schema.types.PInteger; +import org.apache.phoenix.schema.types.PVarbinary; + +@BuiltInFunction(name = GetByteFunction.NAME, args = { + @Argument(allowedTypes = { PBinary.class, PVarbinary.class }), + @Argument(allowedTypes = { PInteger.class }) }) +public class GetByteFunction extends PrefixFunction { + + public static final String NAME = "GET_BYTE"; + + private Integer offsetPreCompute; + + public GetByteFunction() { + } + + public GetByteFunction(List children) throws SQLException { + super(children); + init(); + } + + @Override + public String getName() { + return NAME; + } + + @Override + public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) { + // get offset parameter + int offset; + if (offsetPreCompute == null) { + Expression offsetExpr = children.get(1); + if (!offsetExpr.evaluate(tuple, ptr)) return false; + offset = (Integer) PInteger.INSTANCE.toObject(ptr, offsetExpr.getSortOrder()); + } else offset = offsetPreCompute; + // get binary data parameter + Expression dataExpr = children.get(0); + if (!dataExpr.evaluate(tuple, ptr)) return false; + if (ptr.getLength() == 0) return true; + int len = ptr.getLength(); + offset = (offset % len + len) % len; + // set result + ((PBinaryBase) dataExpr.getDataType()).getByte(ptr, dataExpr.getSortOrder(), offset, ptr); + return true; + } + + @Override + public PDataType getDataType() { + return PInteger.INSTANCE; + } + + private void init() { + Expression offsetExpr = children.get(1); + ImmutableBytesWritable ptr = new ImmutableBytesWritable(); + if (offsetExpr.isStateless() && offsetExpr.getDeterminism() == Determinism.ALWAYS + && offsetExpr.evaluate(null, ptr)) { + offsetPreCompute = (Integer) PInteger.INSTANCE.toObject(ptr, offsetExpr.getSortOrder()); + } else offsetPreCompute = null; + } + + @Override + public OrderPreserving preservesOrder() { + return (offsetPreCompute != null && offsetPreCompute == 0) ? OrderPreserving.YES_IF_LAST + : OrderPreserving.NO; + } +} http://git-wip-us.apache.org/repos/asf/phoenix/blob/b985a4a3/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SetBitFunction.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SetBitFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SetBitFunction.java new file mode 100644 index 0000000..a19ce80 --- /dev/null +++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SetBitFunction.java @@ -0,0 +1,81 @@ +/* + * 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.sql.SQLException; +import java.util.List; + +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.phoenix.expression.Expression; +import org.apache.phoenix.parse.FunctionParseNode.Argument; +import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction; +import org.apache.phoenix.schema.tuple.Tuple; +import org.apache.phoenix.schema.types.PBinary; +import org.apache.phoenix.schema.types.PBinaryBase; +import org.apache.phoenix.schema.types.PDataType; +import org.apache.phoenix.schema.types.PInteger; +import org.apache.phoenix.schema.types.PVarbinary; + +@BuiltInFunction(name = SetBitFunction.NAME, + args = { @Argument(allowedTypes = { PBinary.class, PVarbinary.class }), + @Argument(allowedTypes = { PInteger.class }), + @Argument(allowedTypes = { PInteger.class }) }) +public class SetBitFunction extends ScalarFunction { + + public static final String NAME = "SET_BIT"; + + public SetBitFunction() { + } + + public SetBitFunction(List children) throws SQLException { + super(children); + } + + @Override + public String getName() { + return NAME; + } + + @Override + public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) { + // get offset parameter + Expression offsetExpr = children.get(1); + if (!offsetExpr.evaluate(tuple, ptr)) return false; + int offset = (Integer) PInteger.INSTANCE.toObject(ptr, offsetExpr.getSortOrder()); + // get newValue parameter + Expression newValueExpr = children.get(2); + if (!newValueExpr.evaluate(tuple, ptr)) return false; + int newValue = (Integer) PInteger.INSTANCE.toObject(ptr, newValueExpr.getSortOrder()); + byte newByteValue = (byte) (newValue & 0x1); + // get binary data parameter + Expression dataExpr = children.get(0); + if (!dataExpr.evaluate(tuple, ptr)) return false; + if (ptr.getLength() == 0) return true; + int len = ptr.getLength() * Byte.SIZE; + offset = (offset % len + len) % len; + // set result + ((PBinaryBase) dataExpr.getDataType()).setBit(ptr, dataExpr.getSortOrder(), offset, + newByteValue, ptr); + return true; + } + + @Override + public PDataType getDataType() { + return children.get(0).getDataType(); + } +} http://git-wip-us.apache.org/repos/asf/phoenix/blob/b985a4a3/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SetByteFunction.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SetByteFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SetByteFunction.java new file mode 100644 index 0000000..e38ee18 --- /dev/null +++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SetByteFunction.java @@ -0,0 +1,81 @@ +/* + * 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.sql.SQLException; +import java.util.List; + +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.phoenix.expression.Expression; +import org.apache.phoenix.parse.FunctionParseNode.Argument; +import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction; +import org.apache.phoenix.schema.tuple.Tuple; +import org.apache.phoenix.schema.types.PBinary; +import org.apache.phoenix.schema.types.PBinaryBase; +import org.apache.phoenix.schema.types.PDataType; +import org.apache.phoenix.schema.types.PInteger; +import org.apache.phoenix.schema.types.PVarbinary; + +@BuiltInFunction(name = SetByteFunction.NAME, + args = { @Argument(allowedTypes = { PBinary.class, PVarbinary.class }), + @Argument(allowedTypes = { PInteger.class }), + @Argument(allowedTypes = { PInteger.class }) }) +public class SetByteFunction extends ScalarFunction { + + public static final String NAME = "SET_BYTE"; + + public SetByteFunction() { + } + + public SetByteFunction(List children) throws SQLException { + super(children); + } + + @Override + public String getName() { + return NAME; + } + + @Override + public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) { + // get offset parameter + Expression offsetExpr = children.get(1); + if (!offsetExpr.evaluate(tuple, ptr)) return false; + int offset = (Integer) PInteger.INSTANCE.toObject(ptr, offsetExpr.getSortOrder()); + // get newValue parameter + Expression newValueExpr = children.get(2); + if (!newValueExpr.evaluate(tuple, ptr)) return false; + int newValue = (Integer) PInteger.INSTANCE.toObject(ptr, newValueExpr.getSortOrder()); + byte newByteValue = (byte) (newValue & 0xff); + // get binary data parameter + Expression dataExpr = children.get(0); + if (!dataExpr.evaluate(tuple, ptr)) return false; + if (ptr.getLength() == 0) return true; + int len = ptr.getLength(); + offset = (offset % len + len) % len; + // set result + ((PBinaryBase) dataExpr.getDataType()).setByte(ptr, dataExpr.getSortOrder(), offset, + newByteValue, ptr); + return true; + } + + @Override + public PDataType getDataType() { + return children.get(0).getDataType(); + } +} http://git-wip-us.apache.org/repos/asf/phoenix/blob/b985a4a3/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java index 8e0c4b5..9aa3f42 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java @@ -28,7 +28,7 @@ import org.apache.phoenix.exception.DataExceedsCapacityException; import org.apache.phoenix.query.QueryConstants; import org.apache.phoenix.schema.SortOrder; -public class PBinary extends PDataType { +public class PBinary extends PBinaryBase { public static final PBinary INSTANCE = new PBinary(); http://git-wip-us.apache.org/repos/asf/phoenix/blob/b985a4a3/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinaryBase.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinaryBase.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinaryBase.java new file mode 100644 index 0000000..ec0793e --- /dev/null +++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinaryBase.java @@ -0,0 +1,86 @@ +/* + * 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.schema.types; + +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.phoenix.schema.SortOrder; + +public abstract class PBinaryBase extends PDataType { + + protected PBinaryBase(String sqlTypeName, int sqlType, Class clazz, + org.apache.phoenix.schema.types.PDataType.PDataCodec codec, int ordinal) { + super(sqlTypeName, sqlType, clazz, codec, ordinal); + } + + public void getByte(ImmutableBytesWritable ptr, SortOrder sortOrder, int offset, + ImmutableBytesWritable outPtr) { + getByte(ptr.get(), ptr.getOffset(), ptr.getLength(), sortOrder, offset, outPtr); + } + + public void getByte(byte[] bytes, int offset, int length, SortOrder sortOrder, int off, + ImmutableBytesWritable outPtr) { + byte ret = bytes[offset + off]; + if (sortOrder == SortOrder.DESC) ret = SortOrder.invert(ret); + outPtr.set(PInteger.INSTANCE.toBytes(Integer.valueOf(ret))); + } + + public void setByte(ImmutableBytesWritable ptr, SortOrder sortOrder, int offset, byte newValue, + ImmutableBytesWritable outPtr) { + setByte(ptr.get(), ptr.getOffset(), ptr.getLength(), sortOrder, offset, newValue, outPtr); + } + + public void setByte(byte[] bytes, int offset, int length, SortOrder sortOrder, int off, + byte newValue, ImmutableBytesWritable outPtr) { + byte[] ret; + if (sortOrder == SortOrder.ASC) { + ret = new byte[length]; + System.arraycopy(bytes, offset, ret, 0, length); + } else { + ret = SortOrder.invert(bytes, offset, length); + } + ret[off] = newValue; + outPtr.set(ret); + } + + public void getBit(ImmutableBytesWritable ptr, SortOrder sortOrder, int offset, + ImmutableBytesWritable outPtr) { + getBit(ptr.get(), ptr.getOffset(), ptr.getLength(), sortOrder, offset, outPtr); + } + + public void getBit(byte[] bytes, int offset, int length, SortOrder sortOrder, int off, + ImmutableBytesWritable outPtr) { + byte ret = bytes[offset + (off / Byte.SIZE)]; + if (sortOrder == SortOrder.DESC) ret = SortOrder.invert(ret); + ret &= 1 << (off % Byte.SIZE); + ret = (ret != 0) ? (byte) 1 : (byte) 0; + outPtr.set(PInteger.INSTANCE.toBytes(Integer.valueOf(ret))); + } + + public void setBit(ImmutableBytesWritable ptr, SortOrder sortOrder, int offset, byte newValue, + ImmutableBytesWritable outPtr) { + setBit(ptr.get(), ptr.getOffset(), ptr.getLength(), sortOrder, offset, newValue, outPtr); + } + + public void setBit(byte[] bytes, int offset, int length, SortOrder sortOrder, int off, + byte newValue, ImmutableBytesWritable outPtr) { + byte ret = bytes[offset + (off / Byte.SIZE)]; + if (sortOrder == SortOrder.DESC) ret = SortOrder.invert(ret); + ret = (byte) ((ret & (~(1 << (off % Byte.SIZE)))) | (newValue << (off % Byte.SIZE))); + setByte(bytes, offset, length, sortOrder, off / Byte.SIZE, ret, outPtr); + } +} http://git-wip-us.apache.org/repos/asf/phoenix/blob/b985a4a3/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java index bb1d4c6..aafa1c6 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java @@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.phoenix.schema.SortOrder; import org.apache.phoenix.util.ByteUtil; -public class PVarbinary extends PDataType { +public class PVarbinary extends PBinaryBase { public static final PVarbinary INSTANCE = new PVarbinary(); http://git-wip-us.apache.org/repos/asf/phoenix/blob/b985a4a3/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java index c1787ca..227cbd7 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java @@ -17,6 +17,7 @@ */ package org.apache.phoenix.compile; +import static org.apache.phoenix.util.TestUtil.BINARY_NAME; import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES; import static org.apache.phoenix.util.TestUtil.assertDegenerate; import static org.apache.phoenix.util.TestUtil.assertEmptyScanKey; @@ -68,8 +69,6 @@ import org.apache.phoenix.util.StringUtil; import org.apache.phoenix.util.TestUtil; import org.junit.Test; - - public class WhereOptimizerTest extends BaseConnectionlessQueryTest { private static StatementContext compileStatement(String query) throws SQLException { @@ -106,6 +105,36 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest { } @Test + public void testGetByteBitExpression() throws SQLException { + ensureTableCreated(getUrl(), TestUtil.BINARY_NAME); + int result = 1; + String query = "select * from " + BINARY_NAME + " where GET_BYTE(a_binary, 0)=" + result; + Scan scan = compileStatement(query).getScan(); + + byte[] tmpBytes, tmpBytes2, tmpBytes3; + tmpBytes = PInteger.INSTANCE.toBytes(result); + tmpBytes2 = new byte[16]; + System.arraycopy(tmpBytes, 0, tmpBytes2, 0, tmpBytes.length); + tmpBytes = ByteUtil.nextKey(tmpBytes); + tmpBytes3 = new byte[16]; + System.arraycopy(tmpBytes, 0, tmpBytes3, 0, tmpBytes.length); + assertArrayEquals(tmpBytes2, scan.getStartRow()); + assertArrayEquals(tmpBytes3, scan.getStopRow()); + + query = "select * from " + BINARY_NAME + " where GET_BIT(a_binary, 0)=" + result; + scan = compileStatement(query).getScan(); + + tmpBytes = PInteger.INSTANCE.toBytes(result); + tmpBytes2 = new byte[16]; + System.arraycopy(tmpBytes, 0, tmpBytes2, 0, tmpBytes.length); + tmpBytes = ByteUtil.nextKey(tmpBytes); + tmpBytes3 = new byte[16]; + System.arraycopy(tmpBytes, 0, tmpBytes3, 0, tmpBytes.length); + assertArrayEquals(tmpBytes2, scan.getStartRow()); + assertArrayEquals(tmpBytes3, scan.getStopRow()); + } + + @Test public void testSingleCharPaddedKeyExpression() throws SQLException { String tenantId = "1"; String query = "select * from atable where organization_id='" + tenantId + "'"; http://git-wip-us.apache.org/repos/asf/phoenix/blob/b985a4a3/phoenix-core/src/test/java/org/apache/phoenix/expression/GetSetByteBitFunctionTest.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/test/java/org/apache/phoenix/expression/GetSetByteBitFunctionTest.java b/phoenix-core/src/test/java/org/apache/phoenix/expression/GetSetByteBitFunctionTest.java new file mode 100644 index 0000000..f01ce3b --- /dev/null +++ b/phoenix-core/src/test/java/org/apache/phoenix/expression/GetSetByteBitFunctionTest.java @@ -0,0 +1,189 @@ +/* + * 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; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +import java.sql.SQLException; +import java.util.List; + +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.phoenix.expression.function.GetBitFunction; +import org.apache.phoenix.expression.function.GetByteFunction; +import org.apache.phoenix.expression.function.SetBitFunction; +import org.apache.phoenix.expression.function.SetByteFunction; +import org.apache.phoenix.schema.SortOrder; +import org.apache.phoenix.schema.types.PBinary; +import org.apache.phoenix.schema.types.PBinaryBase; +import org.apache.phoenix.schema.types.PInteger; +import org.apache.phoenix.schema.types.PVarbinary; +import org.junit.Test; + +import com.google.common.collect.Lists; + +/** + * Unit tests for {@link GetByteFunction} {@link SetByteFunction} {@link GetBitFunction} + * {@link SetBitFunction} + */ +public class GetSetByteBitFunctionTest { + private void testGetByteExpression(Expression data, Expression offset, int expected) + throws SQLException { + List expressions = Lists.newArrayList(data, offset); + Expression getByteFunction = new GetByteFunction(expressions); + ImmutableBytesWritable ptr = new ImmutableBytesWritable(); + getByteFunction.evaluate(null, ptr); + Integer result = + (Integer) getByteFunction.getDataType().toObject(ptr, + getByteFunction.getSortOrder()); + assertEquals(expected, result.intValue()); + } + + private void testSetByteExpression(Expression data, Expression offset, Expression newValue, + byte[] expected) throws SQLException { + List expressions = Lists.newArrayList(data, offset, newValue); + Expression setByteFunction = new SetByteFunction(expressions); + ImmutableBytesWritable ptr = new ImmutableBytesWritable(); + setByteFunction.evaluate(null, ptr); + byte[] result = + (byte[]) setByteFunction.getDataType() + .toObject(ptr, setByteFunction.getSortOrder()); + assertArrayEquals(expected, result); + } + + private void testGetByte(byte[] bytes, int offset, PBinaryBase dataType, int expected) + throws SQLException { + LiteralExpression dataExpr, offsetExpr; + dataExpr = LiteralExpression.newConstant(bytes, dataType, SortOrder.ASC); + offsetExpr = LiteralExpression.newConstant(offset, PInteger.INSTANCE, SortOrder.ASC); + testGetByteExpression(dataExpr, offsetExpr, expected); + dataExpr = LiteralExpression.newConstant(bytes, dataType, SortOrder.DESC); + offsetExpr = LiteralExpression.newConstant(offset, PInteger.INSTANCE, SortOrder.DESC); + testGetByteExpression(dataExpr, offsetExpr, expected); + } + + private void testSetByte(byte[] bytes, int offset, int newValue, PBinaryBase dataType, + byte[] expected) throws SQLException { + LiteralExpression dataExpr, offsetExpr, newValueExpr; + dataExpr = LiteralExpression.newConstant(bytes, dataType, SortOrder.ASC); + offsetExpr = LiteralExpression.newConstant(offset, PInteger.INSTANCE, SortOrder.ASC); + newValueExpr = LiteralExpression.newConstant(newValue, PInteger.INSTANCE, SortOrder.ASC); + testSetByteExpression(dataExpr, offsetExpr, newValueExpr, expected); + dataExpr = LiteralExpression.newConstant(bytes, dataType, SortOrder.DESC); + offsetExpr = LiteralExpression.newConstant(offset, PInteger.INSTANCE, SortOrder.DESC); + newValueExpr = LiteralExpression.newConstant(newValue, PInteger.INSTANCE, SortOrder.DESC); + testSetByteExpression(dataExpr, offsetExpr, newValueExpr, expected); + } + + @Test + public void testByteBatch() throws SQLException { + byte[] bytes = new byte[256]; + int sum = 0; + for (int i = 0; i < 256; ++i) { + bytes[i] = (byte) (i & 0xff); + sum += bytes[i]; + } + assertEquals(-128, sum); + for (int offset = 0; offset < 256; ++offset) { + testGetByte(bytes, offset, PBinary.INSTANCE, bytes[offset]); + testGetByte(bytes, offset, PVarbinary.INSTANCE, bytes[offset]); + } + for (int offset = 0; offset < 256; ++offset) + for (int tmp = Byte.MIN_VALUE; tmp <= Byte.MAX_VALUE; ++tmp) { + byte[] expected = new byte[bytes.length]; + System.arraycopy(bytes, 0, expected, 0, bytes.length); + expected[offset] = (byte) (tmp & 0xff); + testSetByte(bytes, offset, tmp, PBinary.INSTANCE, expected); + } + } + + private void testGetBitExpression(Expression data, Expression offset, int expected) + throws SQLException { + List expressions = Lists.newArrayList(data, offset); + Expression getBitFunction = new GetBitFunction(expressions); + ImmutableBytesWritable ptr = new ImmutableBytesWritable(); + getBitFunction.evaluate(null, ptr); + Integer result = + (Integer) getBitFunction.getDataType().toObject(ptr, getBitFunction.getSortOrder()); + assertEquals(expected, result.intValue()); + } + + private void testSetBitExpression(Expression data, Expression offset, Expression newValue, + byte[] expected) throws SQLException { + List expressions = Lists.newArrayList(data, offset, newValue); + Expression setBitFunction = new SetBitFunction(expressions); + ImmutableBytesWritable ptr = new ImmutableBytesWritable(); + setBitFunction.evaluate(null, ptr); + byte[] result = + (byte[]) setBitFunction.getDataType().toObject(ptr, setBitFunction.getSortOrder()); + assertArrayEquals(expected, result); + } + + private void testGetBit(byte[] bytes, int offset, PBinaryBase dataType, int expected) + throws SQLException { + LiteralExpression dataExpr, offsetExpr; + dataExpr = LiteralExpression.newConstant(bytes, dataType, SortOrder.ASC); + offsetExpr = LiteralExpression.newConstant(offset, PInteger.INSTANCE, SortOrder.ASC); + testGetBitExpression(dataExpr, offsetExpr, expected); + dataExpr = LiteralExpression.newConstant(bytes, dataType, SortOrder.DESC); + offsetExpr = LiteralExpression.newConstant(offset, PInteger.INSTANCE, SortOrder.DESC); + testGetBitExpression(dataExpr, offsetExpr, expected); + } + + private void testSetBit(byte[] bytes, int offset, int newValue, PBinaryBase dataType, + byte[] expected) throws SQLException { + LiteralExpression dataExpr, offsetExpr, newValueExpr; + dataExpr = LiteralExpression.newConstant(bytes, dataType, SortOrder.ASC); + offsetExpr = LiteralExpression.newConstant(offset, PInteger.INSTANCE, SortOrder.ASC); + newValueExpr = LiteralExpression.newConstant(newValue, PInteger.INSTANCE, SortOrder.ASC); + testSetBitExpression(dataExpr, offsetExpr, newValueExpr, expected); + dataExpr = LiteralExpression.newConstant(bytes, dataType, SortOrder.DESC); + offsetExpr = LiteralExpression.newConstant(offset, PInteger.INSTANCE, SortOrder.DESC); + newValueExpr = LiteralExpression.newConstant(newValue, PInteger.INSTANCE, SortOrder.DESC); + testSetBitExpression(dataExpr, offsetExpr, newValueExpr, expected); + } + + @Test + public void testGetBitBatch() throws SQLException { + byte[] bytes = new byte[256]; + int sum = 0; + for (int i = 0; i < 256; ++i) { + bytes[i] = (byte) (i & 0xff); + sum += bytes[i]; + } + assertEquals(-128, sum); + for (int offset = 0; offset < 256 * Byte.SIZE; ++offset) { + byte expected = + (bytes[offset / Byte.SIZE] & (1 << (offset % Byte.SIZE))) != 0 ? (byte) 1 + : (byte) 0; + testGetBit(bytes, offset, PBinary.INSTANCE, expected); + testGetBit(bytes, offset, PVarbinary.INSTANCE, expected); + } + for (int offset = 0; offset < 256 * Byte.SIZE; ++offset) + for (int tmp = 0; tmp <= 1; ++tmp) { + byte[] expected = new byte[bytes.length]; + System.arraycopy(bytes, 0, expected, 0, bytes.length); + if (tmp != 0) { + expected[offset / Byte.SIZE] |= (byte) (1 << (offset % Byte.SIZE)); + } else { + expected[offset / Byte.SIZE] &= (byte) (~(1 << (offset % Byte.SIZE))); + } + testSetBit(bytes, offset, tmp, PBinary.INSTANCE, expected); + } + } +} http://git-wip-us.apache.org/repos/asf/phoenix/blob/b985a4a3/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java index 356e3fb..420bb1b 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java @@ -23,6 +23,7 @@ import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR; import static org.apache.phoenix.util.PhoenixRuntime.PHOENIX_TEST_DRIVER_URL_PARAM; import static org.apache.phoenix.util.TestUtil.ATABLE_NAME; import static org.apache.phoenix.util.TestUtil.A_VALUE; +import static org.apache.phoenix.util.TestUtil.BINARY_NAME; import static org.apache.phoenix.util.TestUtil.BTABLE_NAME; import static org.apache.phoenix.util.TestUtil.B_VALUE; import static org.apache.phoenix.util.TestUtil.CUSTOM_ENTITY_DATA_FULL_NAME; @@ -458,6 +459,13 @@ public abstract class BaseTest { " co_item_name varchar " + " CONSTRAINT pk PRIMARY KEY (item_id, item_name)) " + " SALT_BUCKETS=4"); + builder.put(BINARY_NAME,"create table " + BINARY_NAME + + " (a_binary BINARY(16) not null, \n" + + " b_binary BINARY(16), \n" + + " a_varbinary VARBINARY, \n" + + " b_varbinary VARBINARY, \n" + + " CONSTRAINT pk PRIMARY KEY (a_binary)\n" + + ") "); tableDDLMap = builder.build(); } http://git-wip-us.apache.org/repos/asf/phoenix/blob/b985a4a3/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java index 15e3583..1d56dba 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java @@ -207,6 +207,7 @@ public class TestUtil { public static final String JOIN_ITEM_TABLE_DISPLAY_NAME = JOIN_SCHEMA + "." + JOIN_ITEM_TABLE; public static final String JOIN_SUPPLIER_TABLE_DISPLAY_NAME = JOIN_SCHEMA + "." + JOIN_SUPPLIER_TABLE; public static final String JOIN_COITEM_TABLE_DISPLAY_NAME = JOIN_SCHEMA + "." + JOIN_COITEM_TABLE; + public static final String BINARY_NAME = "BinaryTable"; /** * Read-only properties used by all tests