You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2014/05/23 06:45:53 UTC

git commit: PHOENIX-984 Add support for DECODE built-in function (tzolkincz)

Repository: incubator-phoenix
Updated Branches:
  refs/heads/3.0 e3193abb8 -> 7d81c4212


PHOENIX-984 Add support for DECODE built-in function (tzolkincz)


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

Branch: refs/heads/3.0
Commit: 7d81c42127d5026be836a221f4cbed4cf9862b12
Parents: e3193ab
Author: James Taylor <jt...@salesforce.com>
Authored: Thu May 22 21:46:50 2014 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu May 22 21:46:50 2014 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/DecodeFunctionIT.java       | 138 +++++++++++++++++++
 .../phoenix/expression/ExpressionType.java      |   4 +-
 .../expression/function/DecodeFunction.java     | 127 +++++++++++++++++
 .../expression/function/EncodeFormat.java       |  21 +++
 4 files changed, 289 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/7d81c421/phoenix-core/src/it/java/org/apache/phoenix/end2end/DecodeFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DecodeFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DecodeFunctionIT.java
new file mode 100644
index 0000000..19dad3e
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DecodeFunctionIT.java
@@ -0,0 +1,138 @@
+/*
+ * 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 java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.schema.IllegalDataException;
+import org.apache.phoenix.schema.PDataType;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.junit.Test;
+
+public class DecodeFunctionIT extends BaseHBaseManagedTimeIT {
+
+	@Test
+	public void shouldPass() throws Exception {
+		Connection conn = DriverManager.getConnection(getUrl());
+
+		String ddl = "CREATE TABLE test_table ( some_column BINARY(12) NOT NULL CONSTRAINT PK PRIMARY KEY (some_column))";
+
+		conn.createStatement().execute(ddl);
+		PreparedStatement ps = conn.prepareStatement("UPSERT INTO test_table (some_column) VALUES (?)");
+
+		byte[] kk = Bytes.add(PDataType.UNSIGNED_LONG.toBytes(2232594215l), PDataType.INTEGER.toBytes(-8));
+		ps.setBytes(1, kk);
+
+		ps.execute();
+		conn.commit();
+
+		ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM test_table WHERE some_column = DECODE('000000008512af277ffffff8', 'hex')");
+		assertTrue(rs.next());
+	}
+
+	@Test
+	public void upperCaseHexEncoding() throws Exception {
+		Connection conn = DriverManager.getConnection(getUrl());
+
+		String ddl = "CREATE TABLE test_table ( some_column BINARY(12) NOT NULL CONSTRAINT PK PRIMARY KEY (some_column))";
+
+		conn.createStatement().execute(ddl);
+		PreparedStatement ps = conn.prepareStatement("UPSERT INTO test_table (some_column) VALUES (?)");
+
+		byte[] kk = Bytes.add(PDataType.UNSIGNED_LONG.toBytes(2232594215l), PDataType.INTEGER.toBytes(-8));
+		ps.setBytes(1, kk);
+
+		ps.execute();
+		conn.commit();
+
+		ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM test_table WHERE some_column = DECODE('000000008512af277ffffff8', 'HEX')");
+		assertTrue(rs.next());
+	}
+
+	@Test
+	public void invalidCharacters() throws Exception {
+		Connection conn = DriverManager.getConnection(getUrl());
+		String ddl = "CREATE TABLE test_table ( some_column BINARY(12) NOT NULL CONSTRAINT PK PRIMARY KEY (some_column))";
+
+		conn.createStatement().execute(ddl);
+
+		try {
+			conn.createStatement().executeQuery("SELECT * FROM test_table WHERE some_column = DECODE('zzxxuuyyzzxxuuyy', 'hex')");
+		} catch (IllegalDataException e) {
+			assertTrue(true);
+			return;
+		}
+		fail();
+	}
+
+	@Test
+	public void invalidLength() throws Exception {
+		Connection conn = DriverManager.getConnection(getUrl());
+		String ddl = "CREATE TABLE test_table ( some_column BINARY(12) NOT NULL CONSTRAINT PK PRIMARY KEY (some_column))";
+
+		conn.createStatement().execute(ddl);
+
+		try {
+			conn.createStatement().executeQuery("SELECT * FROM test_table WHERE some_column = DECODE('8', 'hex')");
+		} catch (IllegalDataException e) {
+			assertTrue(true);
+			return;
+		}
+		fail();
+	}
+
+	@Test
+	public void nullEncoding() throws Exception {
+		Connection conn = DriverManager.getConnection(getUrl());
+		String ddl = "CREATE TABLE test_table ( some_column BINARY(12) NOT NULL CONSTRAINT PK PRIMARY KEY (some_column))";
+
+		conn.createStatement().execute(ddl);
+
+		try {
+			conn.createStatement().executeQuery("SELECT * FROM test_table WHERE some_column = DECODE('8', NULL)");
+		} catch (IllegalDataException e) {
+			assertTrue(true);
+			return;
+		}
+		fail();
+	}
+
+	@Test
+	public void invalidEncoding() throws Exception {
+		Connection conn = DriverManager.getConnection(getUrl());
+		String ddl = "CREATE TABLE test_table ( some_column BINARY(12) NOT NULL CONSTRAINT PK PRIMARY KEY (some_column))";
+
+		conn.createStatement().execute(ddl);
+
+		try {
+			conn.createStatement().executeQuery("SELECT * FROM test_table WHERE some_column = DECODE('8', 'someNonexistFormat')");
+		} catch (SQLException e) {
+			assertTrue(true);
+			return;
+		}
+		fail();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/7d81c421/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 892e698..e25dcb1 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
@@ -65,11 +65,12 @@ import org.apache.phoenix.expression.function.TrimFunction;
 import org.apache.phoenix.expression.function.TruncFunction;
 import org.apache.phoenix.expression.function.UpperFunction;
 import org.apache.phoenix.expression.function.TimezoneOffsetFunction;
+import org.apache.phoenix.expression.function.DecodeFunction;
 
 import com.google.common.collect.Maps;
 
 /**
- * 
+ *
  * Enumeration of all Expression types that may be evaluated on the server-side.
  * Used during serialization and deserialization to pass Expression between client
  * and server.
@@ -155,6 +156,7 @@ public enum ExpressionType {
     ArrayConstructorExpression(ArrayConstructorExpression.class),
     SQLViewTypeFunction(SQLViewTypeFunction.class),
     ExternalSqlTypeIdFunction(ExternalSqlTypeIdFunction.class),
+    DecodeFunction(DecodeFunction.class),
     TimezoneOffsetFunction(TimezoneOffsetFunction.class);
     ExpressionType(Class<? extends Expression> clazz) {
         this.clazz = clazz;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/7d81c421/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DecodeFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DecodeFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DecodeFunction.java
new file mode 100644
index 0000000..f191eb4
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DecodeFunction.java
@@ -0,0 +1,127 @@
+/*
+ * 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;
+import org.apache.phoenix.schema.IllegalDataException;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+/**
+ * Convert string to bytes
+ */
+@FunctionParseNode.BuiltInFunction(name = DecodeFunction.NAME, args = {
+	@FunctionParseNode.Argument(allowedTypes = {PDataType.VARCHAR}),
+	@FunctionParseNode.Argument(enumeration = "EncodeFormat")})
+public class DecodeFunction extends ScalarFunction {
+
+	public static final String NAME = "DECODE";
+
+	public DecodeFunction() {
+	}
+
+	public DecodeFunction(List<Expression> children) throws SQLException {
+		super(children);
+	}
+
+	@Override
+	public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+		Expression expression = getExpression();
+		if (!expression.evaluate(tuple, ptr)) {
+			return false;
+		}
+		if (ptr.getLength() == 0) {
+			return true; // expression was evaluated, but evaluated to null
+		}
+
+		PDataType type = expression.getDataType();
+		String stringToDecode = (String) type.toObject(ptr);
+
+		Expression encodingExpression = getEncodingExpression();
+		if (!encodingExpression.evaluate(tuple, ptr)) {
+			return false;
+		}
+
+		if (ptr.getLength() == 0) {
+			throw new IllegalDataException("Missing bytes encoding.");
+		}
+
+		type = encodingExpression.getDataType();
+		String encoding = ((String) type.toObject(ptr)).toUpperCase();
+
+		byte out[];
+
+		EncodeFormat format = EncodeFormat.valueOf(encoding);
+		switch (format) {
+			case HEX:
+				out = decodeHex(stringToDecode);
+				break;
+			default:
+				throw new IllegalDataException("Unsupported encoding \"" + encoding + "\"");
+		}
+		ptr.set(out);
+
+		return true;
+	}
+
+	private byte[] decodeHex(String hexStr) {
+		byte[] out = new byte[hexStr.length() / 2];
+		for (int i = 0; i < hexStr.length(); i = i + 2) {
+			try {
+				out[i / 2] = (byte) Integer.parseInt(hexStr.substring(i, i + 2), 16);
+			} catch (NumberFormatException ex) {
+				throw new IllegalDataException("Value " + hexStr.substring(i, i + 2) + " cannot be cast to hex number");
+			} catch (StringIndexOutOfBoundsException ex) {
+				throw new IllegalDataException("Invalid value length, cannot cast to hex number (" + hexStr + ")");
+			}
+		}
+		return out;
+	}
+
+	@Override
+	public PDataType getDataType() {
+		return PDataType.VARBINARY;
+	}
+
+	@Override
+	public boolean isNullable() {
+		return getExpression().isNullable();
+	}
+
+	private Expression getExpression() {
+		return children.get(0);
+	}
+
+	private Expression getEncodingExpression() {
+		return children.get(1);
+	}
+
+	@Override
+	public String getName() {
+		return NAME;
+	}
+
+	@Override
+	public Integer getMaxLength() {
+		return getExpression().getMaxLength();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/7d81c421/phoenix-core/src/main/java/org/apache/phoenix/expression/function/EncodeFormat.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/EncodeFormat.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/EncodeFormat.java
new file mode 100644
index 0000000..b02c6e4
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/EncodeFormat.java
@@ -0,0 +1,21 @@
+/*
+ * Copyright 2014 Apache Software Foundation.
+ *
+ * Licensed 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;
+
+public enum EncodeFormat {
+
+	HEX //format for encoding HEX value to bytes
+};