You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ac...@apache.org on 2020/01/13 00:53:29 UTC

[phoenix] branch 4.x-HBase-1.4 updated: PHOENIX-5628 Phoenix Function to Return HBase Row Key of Column Cell

This is an automated email from the ASF dual-hosted git repository.

achouhan pushed a commit to branch 4.x-HBase-1.4
in repository https://gitbox.apache.org/repos/asf/phoenix.git


The following commit(s) were added to refs/heads/4.x-HBase-1.4 by this push:
     new a60cb5a  PHOENIX-5628 Phoenix Function to Return HBase Row Key of Column Cell
a60cb5a is described below

commit a60cb5a18c3d31ae767d0a3b7c44482f1707366a
Author: Abhishek Singh Chouhan <ac...@apache.org>
AuthorDate: Fri Jan 10 17:09:16 2020 -0800

    PHOENIX-5628 Phoenix Function to Return HBase Row Key of Column Cell
---
 .../end2end/RowKeyBytesStringFunctionIT.java       | 81 ++++++++++++++++++++++
 .../apache/phoenix/expression/ExpressionType.java  |  1 +
 .../function/RowKeyBytesStringFunction.java        | 73 +++++++++++++++++++
 3 files changed, 155 insertions(+)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowKeyBytesStringFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowKeyBytesStringFunctionIT.java
new file mode 100644
index 0000000..cde7aa2
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowKeyBytesStringFunctionIT.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.end2end;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.types.PInteger;
+import org.junit.Test;
+
+public class RowKeyBytesStringFunctionIT extends ParallelStatsDisabledIT {
+
+    @Test
+    public void getRowKeyBytesAndVerify() throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            int[] values = {3,7,9,158,5};
+            String tableName = generateUniqueName();
+            String ddl =
+                    "CREATE TABLE IF NOT EXISTS " + tableName + " "
+                            + "(id INTEGER NOT NULL, pkcol VARCHAR, page_id UNSIGNED_LONG,"
+                            + " \"DATE\" BIGINT, \"value\" INTEGER,"
+                            + " constraint pk primary key(id, pkcol)) COLUMN_ENCODED_BYTES = 0";
+            conn.createStatement().execute(ddl);
+
+            conn.createStatement().execute("UPSERT INTO " + tableName
+                    + " (id, pkcol, page_id, \"DATE\", \"value\") VALUES (1, 'a', 8, 1," + values[0] + ")");
+            conn.createStatement().execute("UPSERT INTO " + tableName
+                    + " (id, pkcol, page_id, \"DATE\", \"value\") VALUES (2, 'ab', 8, 2," + values[1] + ")");
+            conn.createStatement().execute("UPSERT INTO " + tableName
+                    + " (id, pkcol, page_id, \"DATE\", \"value\") VALUES (3, 'abc', 8, 3," + values[2] + ")");
+            conn.createStatement().execute("UPSERT INTO " + tableName
+                    + " (id, pkcol, page_id, \"DATE\", \"value\") VALUES (5, 'abcde', 8, 5," + values[4] + ")");
+            conn.createStatement().execute("UPSERT INTO " + tableName
+                    + " (id, pkcol, page_id, \"DATE\", \"value\") VALUES (4, 'abcd', 8, 4," + values[3] + ")");
+            conn.commit();
+
+            ResultSet rs =
+                    conn.createStatement().executeQuery("SELECT ROWKEY_BYTES_STRING() FROM " + tableName);
+            try (org.apache.hadoop.hbase.client.Connection hconn =
+                    ConnectionFactory.createConnection(config)) {
+                Table table = hconn.getTable(TableName.valueOf(tableName));
+                int i = 0;
+                while (rs.next()) {
+                    String s = rs.getString(1);
+                    Get get = new Get(Bytes.toBytesBinary(s));
+                    Result hbaseRes = table.get(get);
+                    assertFalse(hbaseRes.isEmpty());
+                    assertTrue(Bytes.equals(hbaseRes.getValue(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, Bytes.toBytes("value")), 
+                        PInteger.INSTANCE.toBytes(values[i])));
+                    i++;
+                }
+            }
+        }
+    }
+}
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 8f36e23..a9eb772 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
@@ -191,6 +191,7 @@ public enum ExpressionType {
     SinFunction(SinFunction.class),
     CosFunction(CosFunction.class),
     TanFunction(TanFunction.class),
+    RowKeyBytesStringFunction(RowKeyBytesStringFunction.class)
     ;
 
     ExpressionType(Class<? extends Expression> clazz) {
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RowKeyBytesStringFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RowKeyBytesStringFunction.java
new file mode 100644
index 0000000..5c85141
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RowKeyBytesStringFunction.java
@@ -0,0 +1,73 @@
+/*
+ * 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.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PVarchar;
+
+/**
+ * Function to return Rowkey(s) of result(s) in a printable/usable format. The returned result can
+ * be used for debugging(eg. using HBase shell), logging etc.
+ */
+@BuiltInFunction(name = RowKeyBytesStringFunction.NAME, args = {})
+public class RowKeyBytesStringFunction extends ScalarFunction {
+
+    public static final String NAME = "ROWKEY_BYTES_STRING";
+
+    public RowKeyBytesStringFunction() {
+    }
+
+    public RowKeyBytesStringFunction(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        tuple.getKey(ptr);
+        String rowkey = Bytes.toStringBinary(ptr.get(), ptr.getOffset(), ptr.getLength());
+        ptr.set(PVarchar.INSTANCE.toBytes(rowkey));
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PVarchar.INSTANCE;
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+
+    @Override
+    public OrderPreserving preservesOrder() {
+        return OrderPreserving.YES;
+    }
+
+    @Override
+    public boolean isStateless() {
+        return false;
+    }
+}