You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@phoenix.apache.org by GitBox <gi...@apache.org> on 2020/03/05 03:25:53 UTC

[GitHub] [phoenix] jpisaac opened a new pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

jpisaac opened a new pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724
 
 
   Implements a Function **PHOENIX_ROW_TIMESTAMP** that returns the phoenix_row_timestamp - which as defined as the row_timestamp of the empty column.
   
   - Clients can then issue queries using the newly minted function **PHOENIX_ROW_TIMESTAMP**.
   for e.g To get the phoenix_row_timestamp on the client side, one would include it in the SELECT clause as such, **SELECT PHOENIX_ROW_TIMESTAMP(), ...other columns FROM table WHERE where-clause**
   
   - Can also be used as a predicate to filter the rows that need to be fetched from the table. for e.g **SELECT columns ... FROM table where PHOENIX_ROW_TIMESTAMP() < time**.
   
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r406614653
 
 

 ##########
 File path: phoenix-core/src/test/java/org/apache/phoenix/parse/PhoenixRowTimestampFunctionTest.java
 ##########
 @@ -0,0 +1,45 @@
+package org.apache.phoenix.parse;
+
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class PhoenixRowTimestampFunctionTest {
+
+    @Test
+    public void testExpressionWithPhoenixRowTimestamp() throws Exception {
+        ParseNode parseNode = SQLParser.parseCondition("(PHOENIX_ROW_TIMESTAMP() = PK2)");
+        boolean hasPhoenixRowTimestampParseNode = false;
+        for (ParseNode childNode : parseNode.getChildren()) {
+            if (childNode.getClass().isAssignableFrom(PhoenixRowTimestampParseNode.class)) {
+                assertEquals("", childNode.getChildren().size(), 0);
 
 Review comment:
   nit: Remove empty string print in assertion and also fix if condition formatting

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r406608036
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/expression/function/PhoenixRowTimestampFunction.java
 ##########
 @@ -0,0 +1,125 @@
+/*
+ * 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 org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.phoenix.expression.Determinism;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.parse.PhoenixRowTimestampParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Date;
+import java.util.List;
+
+/**
+ * Function to return the timestamp of the empty column which functions as the row timestamp. The
+ * result returned can be used for debugging(eg. using HBase shell), logging etc.
+ * Can also be used in sql predicates.
+ */
+@BuiltInFunction(name = PhoenixRowTimestampFunction.NAME,
+        nodeClass= PhoenixRowTimestampParseNode.class,
+        args = {})
+public class PhoenixRowTimestampFunction extends ScalarFunction {
+    public static final String NAME = "PHOENIX_ROW_TIMESTAMP";
+    private byte[] emptyCF;
+    private byte[] emptyCQ;
+
+    public PhoenixRowTimestampFunction() {
+    }
+
+    public PhoenixRowTimestampFunction(List<Expression> children, byte[] emptyCF, byte[] emptyCQ) {
+        super(children);
+        this.emptyCF = emptyCF;
+        this.emptyCQ = emptyCQ;
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+
+        if (tuple == null || tuple.size() == 0) return false;
 
 Review comment:
   nit: Use braces and move return to next line

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] jpisaac commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
jpisaac commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r402493827
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/parse/PhoenixRowTimestampParseNode.java
 ##########
 @@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.phoenix.parse;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.KeyValueColumnExpression;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.expression.function.PhoenixRowTimestampFunction;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PDatum;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.util.IndexUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+
+public class PhoenixRowTimestampParseNode extends FunctionParseNode {
+
+    PhoenixRowTimestampParseNode(String name, List<ParseNode> children,
+                                 BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+
+        // PhoenixRowTimestampFunction does not take any parameters.
+        assert children.size() == 0;
 
 Review comment:
   Found that when a param is passed into PHOENIX_ROW_TIMESTAMP it gets parsed as UDFParseNode. But added the check and exception anyways.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] jpisaac commented on issue #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
jpisaac commented on issue #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#issuecomment-614677659
 
 
   Thanks @ChinmaySKulkarni @yanxinyi @swaroopak @tkhurana for the reviews. Closing it for now.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r392445610
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRowTimestampFunctionIT.java
 ##########
 @@ -191,4 +192,158 @@ public void testRowTimestampColumn() throws Exception {
         }
     }
 
+    @Test
+    public void testRowTimestampColumnAndPredicates() throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR"
+                    + " CONSTRAINT PK PRIMARY KEY(PK1, PK2))" + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
+
+            String dml = "UPSERT INTO " + tableName + " (PK1, PK2, KV1, KV2) VALUES (?, ?, ?, ?)";
+
+            long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis();
+            Date rowTimestampDate1 = new Date(rowTimestamp - 120000);
+            PreparedStatement stmt = conn.prepareStatement(dml);
+
+            int count = 5;
+            for (int id = 0; id < count; ++id) {
+                int idValue = id;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate1);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            // case: No rows should be selected
+            // Since we used a past date for column PK2
+            String dql1 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV1 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() = PK2";
+
+            ResultSet rs1 = conn.createStatement().executeQuery(dql1);
+            assertTrue(!rs1.next());
 
 Review comment:
   Use assertFalse instead

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r406611949
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/expression/function/PhoenixRowTimestampFunction.java
 ##########
 @@ -0,0 +1,125 @@
+/*
+ * 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 org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.phoenix.expression.Determinism;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.parse.PhoenixRowTimestampParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Date;
+import java.util.List;
+
+/**
+ * Function to return the timestamp of the empty column which functions as the row timestamp. The
+ * result returned can be used for debugging(eg. using HBase shell), logging etc.
+ * Can also be used in sql predicates.
+ */
+@BuiltInFunction(name = PhoenixRowTimestampFunction.NAME,
+        nodeClass= PhoenixRowTimestampParseNode.class,
+        args = {})
 
 Review comment:
   What is the behavior if we were to call `PHOENIX_ROW_TIMESTAMP(<arg(s)>)`? Does specifying `args = {}` cut it, or do we need to throw a specific exception to indicate improper usage?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r406617491
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/parse/PhoenixRowTimestampParseNode.java
 ##########
 @@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.phoenix.parse;
+
+import org.apache.phoenix.compile.FromCompiler;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.KeyValueColumnExpression;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.expression.function.PhoenixRowTimestampFunction;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PDatum;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.util.IndexUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+
+public class PhoenixRowTimestampParseNode extends FunctionParseNode {
+
+    PhoenixRowTimestampParseNode(String name, List<ParseNode> children,
+                                 BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+
+        // PhoenixRowTimestampFunction does not take any parameters.
+        if (children.size() != 0) {
+            throw new IllegalArgumentException("PhoenixRowTimestampFunction does not take any parameters");
+        }
+
+        // Get the empty column family and qualifier for the context.
+        PTable table = context.getCurrentTable().getTable();
+        byte[] emptyColumnFamilyName = SchemaUtil.getEmptyColumnFamily(table);
+        byte[] emptyColumnName = table.getEncodingScheme() == PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS ?
+                QueryConstants.EMPTY_COLUMN_BYTES :
+                table.getEncodingScheme().encode(QueryConstants.ENCODED_EMPTY_COLUMN_NAME);
+
+        // Create an empty column key value expression.
+        // This will cause the empty column key value to evaluated during scan filter processing.
+        List<Expression> emptyColumnExpression = Arrays.asList(new Expression[] {new KeyValueColumnExpression(new PDatum() {
 
 Review comment:
   Why do we need this `emptyColumnExpression`? This gets passed as a `List<Expression> children` and passed to the parent class (`ScalarFunction`) constructor, but is never used since we technically don't have any arguments for this function. Is it possible to pass in null instead of creating this expression? Please let me know if I'm missing something.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] jpisaac commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
jpisaac commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r401292504
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRowTimestampFunctionIT.java
 ##########
 @@ -191,4 +192,158 @@ public void testRowTimestampColumn() throws Exception {
         }
     }
 
+    @Test
+    public void testRowTimestampColumnAndPredicates() throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR"
+                    + " CONSTRAINT PK PRIMARY KEY(PK1, PK2))" + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
+
+            String dml = "UPSERT INTO " + tableName + " (PK1, PK2, KV1, KV2) VALUES (?, ?, ?, ?)";
+
+            long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis();
+            Date rowTimestampDate1 = new Date(rowTimestamp - 120000);
+            PreparedStatement stmt = conn.prepareStatement(dml);
+
+            int count = 5;
+            for (int id = 0; id < count; ++id) {
+                int idValue = id;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate1);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            // case: No rows should be selected
+            // Since we used a past date for column PK2
+            String dql1 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV1 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() = PK2";
+
+            ResultSet rs1 = conn.createStatement().executeQuery(dql1);
+            assertTrue(!rs1.next());
+            rs1.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() > date column
+            // Since we used a past date for column PK2
+            String dql2 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV1 FROM " + tableName +
+                    " WHERE PK2 < PHOENIX_ROW_TIMESTAMP() ";
+
+            ResultSet rs2 = conn.createStatement().executeQuery(dql2);
+            int actualCount2 = 0;
+            while(rs2.next()) {
+                assertTrue(rs2.getDate(1).after(rowTimestampDate1));
+                actualCount2++;
+            }
+            assertEquals(count, actualCount2);
+            rs2.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() < date column
+            // So using a future date.
+            Date rowTimestampDate2 = new Date(rowTimestamp + 120000);
+            for (int id = 0; id < count; ++id) {
+                int idValue = id + 100;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate2);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            String dql3 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV2 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() < PK2";
+
+            ResultSet rs3 = conn.createStatement().executeQuery(dql3);
+            int actualCount3 = 0;
+            while(rs3.next()) {
+                assertTrue(rs3.getDate(1).before(rowTimestampDate2));
+                actualCount3++;
+            }
+            assertEquals(count, actualCount3);
+            rs3.close();
+        }
+    }
+
+    @Ignore("Fails when encoding=NON_ENCODED_QUALIFIERS and PHOENIX_ROW_TIMESTAMP "
+            + "is the only select column"
+            + "Expected rows do match expected:<5> but was:<0>")
+    public void testRowTimestampColumnOnlyAndPredicates() throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR"
+                    + " CONSTRAINT PK PRIMARY KEY(PK1, PK2))" + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
+
+            String dml = "UPSERT INTO " + tableName + " (PK1, PK2, KV1, KV2) VALUES (?, ?, ?, ?)";
+
+            long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis();
+            Date rowTimestampDate1 = new Date(rowTimestamp - 120000);
+            PreparedStatement stmt = conn.prepareStatement(dml);
+
+            int count = 5;
+            for (int id = 0; id < count; ++id) {
+                int idValue = id;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate1);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            // case: No rows should be selected
+            // Since we used a past date for column PK2
+            String dql1 = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() = PK2";
+
+            ResultSet rs1 = conn.createStatement().executeQuery(dql1);
+            assertTrue(!rs1.next());
+            rs1.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() > date column
+            // Since we used a past date for column PK2
+            String dql2 = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName +
+                    " WHERE PK2 < PHOENIX_ROW_TIMESTAMP() ";
+
+            ResultSet rs2 = conn.createStatement().executeQuery(dql2);
+            int actualCount2 = 0;
+            while(rs2.next()) {
+                assertTrue(rs2.getDate(1).after(rowTimestampDate1));
+                actualCount2++;
+            }
+            assertEquals(count, actualCount2);
+            rs2.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() < date column
+            // So using a future date.
+            Date rowTimestampDate2 = new Date(rowTimestamp + 120000);
+            for (int id = 0; id < count; ++id) {
+                int idValue = id + 100;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate2);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            String dql3 = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() < PK2";
+
+            ResultSet rs3 = conn.createStatement().executeQuery(dql3);
+            int actualCount3 = 0;
+            while(rs3.next()) {
+                assertTrue(rs3.getDate(1).before(rowTimestampDate2));
+                actualCount3++;
+            }
+            assertEquals(count, actualCount3);
+            rs3.close();
+        }
 
 Review comment:
   Had kept the test just as a reminder for me - that I need to fix this case. Will update the IT with the proper tests.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] jpisaac commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
jpisaac commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r401293884
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
 ##########
 @@ -507,6 +507,9 @@ public static RowProjector compile(StatementContext context, SelectStatement sta
                 } else {
                     for (byte[] cq : entry.getValue()) {
                             PColumn column = family.getPColumnForColumnQualifier(cq);
+                            // Continue: If an EMPTY_COLUMN is in the projection list, since
+                            // no new KeyValue is returned.
+                            if (column == null) continue;
 
 Review comment:
   This check is for the case when we add empty column in the projection list and since empty column is not part of the table column list, we need to bypass further processing.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r406899931
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/expression/function/PhoenixRowTimestampFunction.java
 ##########
 @@ -0,0 +1,125 @@
+/*
+ * 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 org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.phoenix.expression.Determinism;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.parse.PhoenixRowTimestampParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Date;
+import java.util.List;
+
+/**
+ * Function to return the timestamp of the empty column which functions as the row timestamp. The
+ * result returned can be used for debugging(eg. using HBase shell), logging etc.
+ * Can also be used in sql predicates.
+ */
+@BuiltInFunction(name = PhoenixRowTimestampFunction.NAME,
+        nodeClass= PhoenixRowTimestampParseNode.class,
+        args = {})
+public class PhoenixRowTimestampFunction extends ScalarFunction {
+    public static final String NAME = "PHOENIX_ROW_TIMESTAMP";
+    private byte[] emptyCF;
+    private byte[] emptyCQ;
+
+    public PhoenixRowTimestampFunction() {
+    }
+
+    public PhoenixRowTimestampFunction(List<Expression> children, byte[] emptyCF, byte[] emptyCQ) {
+        super(children);
+        this.emptyCF = emptyCF;
+        this.emptyCQ = emptyCQ;
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+
+        if (tuple == null || tuple.size() == 0) return false;
+
+        long ts = tuple.getValue(0).getTimestamp();
 
 Review comment:
   Discussed offline with @jpisaac about the reasoning behind this. Let's add a comment here to explain these nuances.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r406611279
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRowTimestampFunctionIT.java
 ##########
 @@ -0,0 +1,398 @@
+/*
+ * 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 com.google.common.collect.Lists;
+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.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+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.PTable.ImmutableStorageScheme;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
+import org.apache.phoenix.util.EncodedColumnsUtil;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.Collection;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PhoenixRowTimestampFunctionIT extends ParallelStatsDisabledIT {
+    private final boolean encoded;
+    private final String tableDDLOptions;
+    private static final int NUM_ROWS = 5;
+    private static final long TS_OFFSET = 120000;
+
+    public PhoenixRowTimestampFunctionIT(QualifierEncodingScheme encoding,
+            ImmutableStorageScheme storage) {
+        StringBuilder optionBuilder = new StringBuilder();
+        optionBuilder.append(" COLUMN_ENCODED_BYTES = " + encoding.ordinal());
+        optionBuilder.append(",IMMUTABLE_STORAGE_SCHEME = "+ storage.toString());
+        this.tableDDLOptions = optionBuilder.toString();
+        this.encoded = (encoding != QualifierEncodingScheme.NON_ENCODED_QUALIFIERS) ? true : false;
+    }
+
+    @Parameterized.Parameters(name = "encoding={0},storage={1}")
+    public static synchronized Collection<Object[]> data() {
+        List<Object[]> list = Lists.newArrayList();
+        for (QualifierEncodingScheme encoding : QualifierEncodingScheme.values()) {
+            for (ImmutableStorageScheme storage : ImmutableStorageScheme.values()) {
+                list.add(new Object[]{encoding, storage});
+            }
+        }
+        return list;
+    }
+
+    private void verifyHbaseAllRowsTimestamp(String tableName, ResultSet rs, int expectedRowCount)
+            throws Exception {
+
+        Scan scan = new Scan();
+        byte[] emptyKVQualifier = EncodedColumnsUtil.getEmptyKeyValueInfo(this.encoded).getFirst();
+        try (org.apache.hadoop.hbase.client.Connection hconn =
+                ConnectionFactory.createConnection(config)) {
+            Table table = hconn.getTable(TableName.valueOf(tableName));
+            ResultScanner resultScanner = table.getScanner(scan);
+            int rowCount = 0;
+            while (rs.next()) {
+                Result result = resultScanner.next();
+                long timeStamp = result.getColumnLatestCell(
+                        QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES,
+                        emptyKVQualifier).getTimestamp();
+                assertEquals(rs.getDate(1).getTime(), timeStamp);
+                ++rowCount;
+            }
+            assertEquals(expectedRowCount, rowCount);
+        }
+    }
+
+    private void verifyHbaseRowTimestamp(String tableName, String rowKey, Date expectedTimestamp)
+            throws Exception {
+
+        byte[] emptyKVQualifier = EncodedColumnsUtil.getEmptyKeyValueInfo(this.encoded).getFirst();
+        try (org.apache.hadoop.hbase.client.Connection hconn =
+                ConnectionFactory.createConnection(config)) {
+            Table table = hconn.getTable(TableName.valueOf(tableName));
+            Get get = new Get(Bytes.toBytesBinary(rowKey));
+            Result result = table.get(get);
+            assertFalse(result.isEmpty());
+            long timeStamp = result.getColumnLatestCell(
+                    QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, emptyKVQualifier).getTimestamp();
+            assertEquals(expectedTimestamp.getTime(), timeStamp);
+        }
+    }
+
+    private String createTestData(long rowTimestamp, int numRows) throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            // Create a test table.
+            try (Statement stmt = conn.createStatement()) {
+                String ddl = "CREATE TABLE IF NOT EXISTS " + tableName +
+                        " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR" +
+                        " CONSTRAINT PK PRIMARY KEY(PK1, PK2))" + this.tableDDLOptions;
+                stmt.execute(ddl);
+            }
+
+            // Upsert data into the test table.
+            String dml = "UPSERT INTO " + tableName + " (PK1, PK2, KV1, KV2) VALUES (?, ?, ?, ?)";
+            try (PreparedStatement stmt = conn.prepareStatement(dml)) {
+                Date rowTimestampDate = new Date(rowTimestamp);
+                int count = numRows;
+                for (int id = 0; id < count; ++id) {
+                    int idValue = id;
+                    stmt.setInt(1, idValue);
+                    stmt.setDate(2, rowTimestampDate);
+                    stmt.setString(3, "KV1_" + idValue);
+                    stmt.setString(4, "KV2_" + idValue);
+                    stmt.executeUpdate();
+                }
+            }
+            conn.commit();
+        }
+        return tableName;
+    }
+
+    @Test
+    public void testRowTimestampDefault() throws Exception {
+
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK INTEGER NOT NULL PRIMARY KEY, KV1 VARCHAR, KV2 VARCHAR)"
+                    + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
+
+            String dml = "UPSERT INTO " + tableName + " (PK, KV1, KV2) VALUES (?, ?, ?)";
+            PreparedStatement stmt = conn.prepareStatement(dml);
+
+            int count = 5;
+            for (int id = 0; id < count; ++id) {
+                stmt.setInt(1, id);
+                stmt.setString(2, "KV1_" + id);
+                stmt.setString(3, "KV2_" + id);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            String dql = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName;
+
+            ResultSet rs = conn.createStatement().executeQuery(dql);
+            // verify row timestamp returned by the query matches the empty column cell timestamp
+            verifyHbaseAllRowsTimestamp(tableName, rs, count);
+
+            // update one row
+            conn.createStatement().execute("UPSERT INTO " + tableName
+                    + " (PK, KV1) VALUES (2, 'KV1_foo')");
+            conn.commit();
+
+            rs = conn.createStatement().executeQuery(dql);
+            // verify again after update
+            verifyHbaseAllRowsTimestamp(tableName, rs, count);
+
+            dql = "SELECT ROWKEY_BYTES_STRING(), PHOENIX_ROW_TIMESTAMP() FROM " + tableName
+                    + " WHERE PK >= 1 AND PK <=3 ";
+            rs = conn.createStatement().executeQuery(dql);
+
+            while (rs.next()) {
+                verifyHbaseRowTimestamp(tableName, rs.getString(1), rs.getDate(2));
+            }
+        }
+    }
+
+    @Test
+    public void testRowTimestampColumn() throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR"
+                    + " CONSTRAINT PK PRIMARY KEY(PK1, PK2 ROW_TIMESTAMP))" + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
+
+            String dml = "UPSERT INTO " + tableName + " (PK1, PK2, KV1, KV2) VALUES (?, ?, ?, ?)";
+
+            long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis();
+            Date rowTimestampDate = new Date(rowTimestamp);
+            PreparedStatement stmt = conn.prepareStatement(dml);
+
+            int count = 5;
+            for (int id = 0; id < count; ++id) {
+                stmt.setInt(1, id);
+                stmt.setDate(2, rowTimestampDate);
+                stmt.setString(3, "KV1_" + id);
+                stmt.setString(4, "KV2_" + id);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            String dql = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName;
+
+            ResultSet rs = conn.createStatement().executeQuery(dql);
+            while(rs.next()) {
+                assertEquals(rs.getDate(1), rowTimestampDate);
+            }
+        }
+    }
+
+
+    @Test
+    // case: No rows should have the phoenix_row_timestamp() = date column
+    // Since we used a future date for column PK2
+    public void testRowTimestampFunctionAndEqualPredicate() throws Exception {
+        long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis() + TS_OFFSET;
+        String tableName = createTestData(rowTimestamp, NUM_ROWS);
+        // With phoenix_row_timestamp function only in projection
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String sql = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() = PK2 ";
+            try (Statement stmt = conn.createStatement()) {
+                ResultSet rs = stmt.executeQuery(sql);
+                assertFalse(rs.next());
+                rs.close();
+            }
+        }
+
+        // With phoenix_row_timestamp function and additional columns in projection
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String sql = "SELECT PHOENIX_ROW_TIMESTAMP(), KV1 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() = PK2 ";
+            try (Statement stmt = conn.createStatement()) {
+                ResultSet rs = stmt.executeQuery(sql);
+                assertFalse(rs.next());
+                rs.close();
+            }
+        }
+    }
+
+
+    @Test
+    // case: All rows selected should have the phoenix_row_timestamp() < date column
+    // Since we used a future date for column PK2
+    public void testRowTimestampFunctionOnlyWithLessThanPredicate() throws Exception {
+        long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis() + TS_OFFSET;
+        String tableName = createTestData(rowTimestamp, NUM_ROWS);
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String sql = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() < PK2 ";
+            try (Statement stmt = conn.createStatement()) {
+                ResultSet rs = stmt.executeQuery(sql);
+                int actualCount = 0;
+                while(rs.next()) {
+                    assertTrue(rs.getDate(1).before(new Date(rowTimestamp)));
+                    actualCount++;
+                }
+                assertEquals(NUM_ROWS, actualCount);
+                rs.close();
+            }
+        }
+    }
+
+    @Test
+    // case: All rows selected should have the phoenix_row_timestamp() < date column
+    // Since we used a future date for column PK2
+    // Additional columns should return non null values.
+    public void testRowTimestampFunctionAndAdditionalColsWithLessThanPredicate() throws Exception {
+        long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis() + TS_OFFSET;
+        String tableName = createTestData(rowTimestamp, NUM_ROWS);
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String sql = "SELECT PHOENIX_ROW_TIMESTAMP(), KV2 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() < PK2 ";
+            try (Statement stmt = conn.createStatement()) {
+                ResultSet rs = stmt.executeQuery(sql);
+                int actualCount = 0;
+                while(rs.next()) {
+                    assertTrue(rs.getDate(1).before(new Date(rowTimestamp)));
+                    rs.getString(2);
+                    assertFalse(rs.wasNull());
+                    actualCount++;
+                }
+                assertEquals(NUM_ROWS, actualCount);
+                rs.close();
+            }
+        }
+
+    }
+
+    @Test
+    // case: All rows selected should have the phoenix_row_timestamp() > date column
+    // Since we used a past date for column PK2
+    public void testRowTimestampFunctionOnlyWithGreaterThanPredicate() throws Exception {
+        long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis() - TS_OFFSET;
+        String tableName = createTestData(rowTimestamp, NUM_ROWS);
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String sql = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() > PK2 ";
+            try (Statement stmt = conn.createStatement()) {
+                ResultSet rs = stmt.executeQuery(sql);
+                int actualCount = 0;
+                while(rs.next()) {
+                    assertTrue(rs.getDate(1).after(new Date(rowTimestamp)));
+                    actualCount++;
+                }
+                assertEquals(NUM_ROWS, actualCount);
+                rs.close();
+            }
+        }
+    }
+
+    @Test
+    // case: All rows selected should have the phoenix_row_timestamp() > date column
+    // Since we used a past date for column PK2
+    // Additional columns should return non null values.
+    public void testRowTimestampFunctionAndAdditionalColsWithGreaterThanPredicate() throws Exception {
+        long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis() - TS_OFFSET;
+        String tableName = createTestData(rowTimestamp, NUM_ROWS);
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String sql = "SELECT PHOENIX_ROW_TIMESTAMP(), KV1 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() > PK2 ";
+            try (Statement stmt = conn.createStatement()) {
+                ResultSet rs = stmt.executeQuery(sql);
+                int actualCount = 0;
+                while(rs.next()) {
+                    assertTrue(rs.getDate(1).after(new Date(rowTimestamp)));
+                    rs.getString(2);
+                    assertFalse(rs.wasNull());
+                    actualCount++;
+                }
+                assertEquals(NUM_ROWS, actualCount);
+                rs.close();
+            }
+        }
+    }
+
+    @Test
+    // case: All rows selected should have the phoenix_row_timestamp() > date column
+    // Since we used a past date for column PK2
+    // Projected columns should return non null and expected values.
+    public void testSimpleSelectColsWithPhoenixRowTimestampPredicate() throws Exception {
+        long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis() - TS_OFFSET;
+        String tableName = createTestData(rowTimestamp, NUM_ROWS);
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String sql = "SELECT KV1 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() > PK2 ";
+            try (Statement stmt = conn.createStatement()) {
+                ResultSet rs = stmt.executeQuery(sql);
+                int actualCount = 0;
+                while(rs.next()) {
+                    String kv1Value = rs.getString(1);
+                    assertFalse(rs.wasNull());
+                    assertTrue(kv1Value.substring(0, "KV2_".length()).compareToIgnoreCase("KV1_") == 0);
+                    actualCount++;
+                }
+                assertEquals(NUM_ROWS, actualCount);
+                rs.close();
+            }
+        }
+    }
+
+    @Test
+    // case: Aggregate SQLs work with PhoenixRowTimestamp predicate.
+    public void testSelectCountWithPhoenixRowTimestampPredicate() throws Exception {
+        long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis() - TS_OFFSET;
+        String tableName = createTestData(rowTimestamp, NUM_ROWS);
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String sql = "SELECT COUNT(*) FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() > PK2 ";
+            try (Statement stmt = conn.createStatement()) {
+                ResultSet rs = stmt.executeQuery(sql);
+                while(rs.next()) {
+                    int rowCount = rs.getInt(1);
+                    assertFalse(rs.wasNull());
+                    assertTrue(rowCount == NUM_ROWS);
+                }
+                rs.close();
+            }
+        }
+    }
+
 
 Review comment:
   Can we add a test with a `WHERE PHOENIX_ROW_TIMESTAMP() < TO_DATE("date string')` or comparison with `TO_TIMESTAMP('timestamp string')`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] jpisaac commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
jpisaac commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r402490854
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRowTimestampFunctionIT.java
 ##########
 @@ -191,4 +192,158 @@ public void testRowTimestampColumn() throws Exception {
         }
     }
 
+    @Test
+    public void testRowTimestampColumnAndPredicates() throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR"
+                    + " CONSTRAINT PK PRIMARY KEY(PK1, PK2))" + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
+
+            String dml = "UPSERT INTO " + tableName + " (PK1, PK2, KV1, KV2) VALUES (?, ?, ?, ?)";
+
+            long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis();
+            Date rowTimestampDate1 = new Date(rowTimestamp - 120000);
+            PreparedStatement stmt = conn.prepareStatement(dml);
+
+            int count = 5;
+            for (int id = 0; id < count; ++id) {
+                int idValue = id;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate1);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            // case: No rows should be selected
+            // Since we used a past date for column PK2
+            String dql1 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV1 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() = PK2";
+
+            ResultSet rs1 = conn.createStatement().executeQuery(dql1);
+            assertTrue(!rs1.next());
+            rs1.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() > date column
+            // Since we used a past date for column PK2
+            String dql2 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV1 FROM " + tableName +
+                    " WHERE PK2 < PHOENIX_ROW_TIMESTAMP() ";
+
+            ResultSet rs2 = conn.createStatement().executeQuery(dql2);
+            int actualCount2 = 0;
+            while(rs2.next()) {
+                assertTrue(rs2.getDate(1).after(rowTimestampDate1));
+                actualCount2++;
+            }
+            assertEquals(count, actualCount2);
+            rs2.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() < date column
+            // So using a future date.
+            Date rowTimestampDate2 = new Date(rowTimestamp + 120000);
+            for (int id = 0; id < count; ++id) {
+                int idValue = id + 100;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate2);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            String dql3 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV2 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() < PK2";
+
+            ResultSet rs3 = conn.createStatement().executeQuery(dql3);
+            int actualCount3 = 0;
+            while(rs3.next()) {
+                assertTrue(rs3.getDate(1).before(rowTimestampDate2));
+                actualCount3++;
+            }
+            assertEquals(count, actualCount3);
+            rs3.close();
+        }
+    }
+
+    @Ignore("Fails when encoding=NON_ENCODED_QUALIFIERS and PHOENIX_ROW_TIMESTAMP "
+            + "is the only select column"
 
 Review comment:
   Fixed it in the new commit. A FirstKeyOnlyFilter filter was getting added which was causing the rows to get filtered out.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] yanxinyi commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
yanxinyi commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r405826813
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRowTimestampFunctionIT.java
 ##########
 @@ -191,4 +225,130 @@ public void testRowTimestampColumn() throws Exception {
         }
     }
 
+
+    @Test
+    // case: No rows should have the phoenix_row_timestamp() = date column
+    // Since we used a future date for column PK2
+    public void testRowTimestampFunctionAndEqualPredicate() throws Exception {
+        long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis() + TS_OFFSET;
+        String tableName = createTestData(rowTimestamp, NUM_ROWS);
+        // With phoenix_row_timestamp function only in projection
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String sql = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() = PK2 ";
+            try (Statement stmt = conn.createStatement()) {
+                ResultSet rs = stmt.executeQuery(sql);
+                assertFalse(rs.next());
+                rs.close();
+            }
+        }
+
+        // With phoenix_row_timestamp function and additional columns in projection
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String sql = "SELECT PHOENIX_ROW_TIMESTAMP(), KV1 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() = PK2 ";
+            try (Statement stmt = conn.createStatement()) {
+                ResultSet rs = stmt.executeQuery(sql);
+                assertFalse(rs.next());
+                rs.close();
+            }
+        }
+    }
+
+
+    @Test
+    // case: All rows selected should have the phoenix_row_timestamp() < date column
+    // Since we used a future date for column PK2
+    public void testRowTimestampFunctionOnlyWithLessThanPredicate() throws Exception {
+        long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis() + TS_OFFSET;
+        String tableName = createTestData(rowTimestamp, NUM_ROWS);
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String sql = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() < PK2 ";
+            try (Statement stmt = conn.createStatement()) {
+                ResultSet rs = stmt.executeQuery(sql);
+                int actualCount = 0;
+                while(rs.next()) {
+                    assertTrue(rs.getDate(1).before(new Date(rowTimestamp)));
+                    actualCount++;
+                }
+                assertEquals(NUM_ROWS, actualCount);
+                rs.close();
+            }
+        }
+    }
+
+    @Test
+    // case: All rows selected should have the phoenix_row_timestamp() < date column
+    // Since we used a future date for column PK2
+    // Additional columns should return non null values.
+    public void testRowTimestampFunctionAndAdditionalColsWithLessThanPredicate() throws Exception {
+        long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis() + TS_OFFSET;
+        String tableName = createTestData(rowTimestamp, NUM_ROWS);
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String sql = "SELECT PHOENIX_ROW_TIMESTAMP(), KV2 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() < PK2 ";
+            try (Statement stmt = conn.createStatement()) {
+                ResultSet rs = stmt.executeQuery(sql);
+                int actualCount = 0;
+                while(rs.next()) {
+                    assertTrue(rs.getDate(1).before(new Date(rowTimestamp)));
+                    rs.getString(2);
+                    assertFalse(rs.wasNull());
+                    actualCount++;
+                }
+                assertEquals(NUM_ROWS, actualCount);
+                rs.close();
+            }
+        }
+
+    }
+
+    @Test
+    // case: All rows selected should have the phoenix_row_timestamp() > date column
+    // Since we used a past date for column PK2
+    public void testRowTimestampFunctionOnlyWithGreaterThanPredicate() throws Exception {
+        long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis() - TS_OFFSET;
+        String tableName = createTestData(rowTimestamp, NUM_ROWS);
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String sql = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() > PK2 ";
+            try (Statement stmt = conn.createStatement()) {
+                ResultSet rs = stmt.executeQuery(sql);
+                int actualCount = 0;
+                while(rs.next()) {
+                    assertTrue(rs.getDate(1).after(new Date(rowTimestamp)));
+                    actualCount++;
+                }
+                assertEquals(NUM_ROWS, actualCount);
+                rs.close();
+            }
+        }
+    }
+
+    @Test
+    // case: All rows selected should have the phoenix_row_timestamp() > date column
+    // Since we used a past date for column PK2
+    // Additional columns should return non null values.
+    public void testRowTimestampFunctionAndAdditionalColsWithGreaterThanPredicate() throws Exception {
+        long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis() - TS_OFFSET;
+        String tableName = createTestData(rowTimestamp, NUM_ROWS);
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String sql = "SELECT PHOENIX_ROW_TIMESTAMP(), KV1 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() > PK2 ";
+            try (Statement stmt = conn.createStatement()) {
+                ResultSet rs = stmt.executeQuery(sql);
+                int actualCount = 0;
+                while(rs.next()) {
+                    assertTrue(rs.getDate(1).after(new Date(rowTimestamp)));
+                    rs.getString(2);
+                    assertFalse(rs.wasNull());
+                    actualCount++;
+                }
+                assertEquals(NUM_ROWS, actualCount);
+                rs.close();
+            }
+        }
+    }
+
 
 Review comment:
   better to support test for `SELECT COLUMN_ONLY WHERE PHOENIX_ROW_TIMESTAMP()` predicate. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r392447033
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRowTimestampFunctionIT.java
 ##########
 @@ -191,4 +192,158 @@ public void testRowTimestampColumn() throws Exception {
         }
     }
 
+    @Test
+    public void testRowTimestampColumnAndPredicates() throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR"
+                    + " CONSTRAINT PK PRIMARY KEY(PK1, PK2))" + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
+
+            String dml = "UPSERT INTO " + tableName + " (PK1, PK2, KV1, KV2) VALUES (?, ?, ?, ?)";
+
+            long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis();
+            Date rowTimestampDate1 = new Date(rowTimestamp - 120000);
+            PreparedStatement stmt = conn.prepareStatement(dml);
+
+            int count = 5;
+            for (int id = 0; id < count; ++id) {
+                int idValue = id;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate1);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            // case: No rows should be selected
+            // Since we used a past date for column PK2
+            String dql1 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV1 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() = PK2";
+
+            ResultSet rs1 = conn.createStatement().executeQuery(dql1);
+            assertTrue(!rs1.next());
+            rs1.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() > date column
+            // Since we used a past date for column PK2
+            String dql2 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV1 FROM " + tableName +
+                    " WHERE PK2 < PHOENIX_ROW_TIMESTAMP() ";
+
+            ResultSet rs2 = conn.createStatement().executeQuery(dql2);
+            int actualCount2 = 0;
+            while(rs2.next()) {
+                assertTrue(rs2.getDate(1).after(rowTimestampDate1));
+                actualCount2++;
+            }
+            assertEquals(count, actualCount2);
+            rs2.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() < date column
+            // So using a future date.
+            Date rowTimestampDate2 = new Date(rowTimestamp + 120000);
+            for (int id = 0; id < count; ++id) {
+                int idValue = id + 100;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate2);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            String dql3 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV2 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() < PK2";
+
+            ResultSet rs3 = conn.createStatement().executeQuery(dql3);
+            int actualCount3 = 0;
+            while(rs3.next()) {
+                assertTrue(rs3.getDate(1).before(rowTimestampDate2));
+                actualCount3++;
+            }
+            assertEquals(count, actualCount3);
+            rs3.close();
+        }
+    }
+
+    @Ignore("Fails when encoding=NON_ENCODED_QUALIFIERS and PHOENIX_ROW_TIMESTAMP "
+            + "is the only select column"
+            + "Expected rows do match expected:<5> but was:<0>")
+    public void testRowTimestampColumnOnlyAndPredicates() throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR"
+                    + " CONSTRAINT PK PRIMARY KEY(PK1, PK2))" + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
 
 Review comment:
   nit: Add statement creation to try-with-resources

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r392448417
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRowTimestampFunctionIT.java
 ##########
 @@ -191,4 +192,158 @@ public void testRowTimestampColumn() throws Exception {
         }
     }
 
+    @Test
+    public void testRowTimestampColumnAndPredicates() throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR"
+                    + " CONSTRAINT PK PRIMARY KEY(PK1, PK2))" + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
+
+            String dml = "UPSERT INTO " + tableName + " (PK1, PK2, KV1, KV2) VALUES (?, ?, ?, ?)";
+
+            long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis();
+            Date rowTimestampDate1 = new Date(rowTimestamp - 120000);
+            PreparedStatement stmt = conn.prepareStatement(dml);
+
+            int count = 5;
+            for (int id = 0; id < count; ++id) {
+                int idValue = id;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate1);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            // case: No rows should be selected
+            // Since we used a past date for column PK2
+            String dql1 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV1 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() = PK2";
+
+            ResultSet rs1 = conn.createStatement().executeQuery(dql1);
+            assertTrue(!rs1.next());
+            rs1.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() > date column
+            // Since we used a past date for column PK2
+            String dql2 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV1 FROM " + tableName +
+                    " WHERE PK2 < PHOENIX_ROW_TIMESTAMP() ";
+
+            ResultSet rs2 = conn.createStatement().executeQuery(dql2);
+            int actualCount2 = 0;
+            while(rs2.next()) {
+                assertTrue(rs2.getDate(1).after(rowTimestampDate1));
+                actualCount2++;
+            }
+            assertEquals(count, actualCount2);
+            rs2.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() < date column
+            // So using a future date.
+            Date rowTimestampDate2 = new Date(rowTimestamp + 120000);
+            for (int id = 0; id < count; ++id) {
+                int idValue = id + 100;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate2);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            String dql3 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV2 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() < PK2";
+
+            ResultSet rs3 = conn.createStatement().executeQuery(dql3);
+            int actualCount3 = 0;
+            while(rs3.next()) {
+                assertTrue(rs3.getDate(1).before(rowTimestampDate2));
+                actualCount3++;
+            }
+            assertEquals(count, actualCount3);
+            rs3.close();
+        }
+    }
+
+    @Ignore("Fails when encoding=NON_ENCODED_QUALIFIERS and PHOENIX_ROW_TIMESTAMP "
+            + "is the only select column"
+            + "Expected rows do match expected:<5> but was:<0>")
+    public void testRowTimestampColumnOnlyAndPredicates() throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR"
+                    + " CONSTRAINT PK PRIMARY KEY(PK1, PK2))" + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
+
+            String dml = "UPSERT INTO " + tableName + " (PK1, PK2, KV1, KV2) VALUES (?, ?, ?, ?)";
+
+            long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis();
+            Date rowTimestampDate1 = new Date(rowTimestamp - 120000);
+            PreparedStatement stmt = conn.prepareStatement(dml);
+
+            int count = 5;
+            for (int id = 0; id < count; ++id) {
+                int idValue = id;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate1);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            // case: No rows should be selected
+            // Since we used a past date for column PK2
+            String dql1 = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() = PK2";
+
+            ResultSet rs1 = conn.createStatement().executeQuery(dql1);
+            assertTrue(!rs1.next());
+            rs1.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() > date column
+            // Since we used a past date for column PK2
+            String dql2 = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName +
+                    " WHERE PK2 < PHOENIX_ROW_TIMESTAMP() ";
+
+            ResultSet rs2 = conn.createStatement().executeQuery(dql2);
+            int actualCount2 = 0;
+            while(rs2.next()) {
+                assertTrue(rs2.getDate(1).after(rowTimestampDate1));
+                actualCount2++;
+            }
+            assertEquals(count, actualCount2);
+            rs2.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() < date column
+            // So using a future date.
+            Date rowTimestampDate2 = new Date(rowTimestamp + 120000);
+            for (int id = 0; id < count; ++id) {
+                int idValue = id + 100;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate2);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            String dql3 = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() < PK2";
+
+            ResultSet rs3 = conn.createStatement().executeQuery(dql3);
+            int actualCount3 = 0;
+            while(rs3.next()) {
+                assertTrue(rs3.getDate(1).before(rowTimestampDate2));
+                actualCount3++;
+            }
+            assertEquals(count, actualCount3);
+            rs3.close();
+        }
+    }
+
 }
 
 Review comment:
   @jpisaac  Since we rely on the empty column timestamp to be reflected as the `PHOENIX_ROW_TIMESTAMP()`, should we add tests to confirm this? Also perhaps add tests that confirm that this is modified whenever a row is updated

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r392449168
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
 ##########
 @@ -507,6 +507,9 @@ public static RowProjector compile(StatementContext context, SelectStatement sta
                 } else {
                     for (byte[] cq : entry.getValue()) {
                             PColumn column = family.getPColumnForColumnQualifier(cq);
+                            // Continue: If an EMPTY_COLUMN is in the projection list, since
+                            // no new KeyValue is returned.
+                            if (column == null) continue;
 
 Review comment:
   Don't we omit the empty column when projecting? Under which circumstances will this condition be true?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r406613824
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/expression/function/PhoenixRowTimestampFunction.java
 ##########
 @@ -0,0 +1,125 @@
+/*
+ * 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 org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.phoenix.expression.Determinism;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.parse.PhoenixRowTimestampParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Date;
+import java.util.List;
+
+/**
+ * Function to return the timestamp of the empty column which functions as the row timestamp. The
+ * result returned can be used for debugging(eg. using HBase shell), logging etc.
+ * Can also be used in sql predicates.
+ */
+@BuiltInFunction(name = PhoenixRowTimestampFunction.NAME,
+        nodeClass= PhoenixRowTimestampParseNode.class,
+        args = {})
+public class PhoenixRowTimestampFunction extends ScalarFunction {
+    public static final String NAME = "PHOENIX_ROW_TIMESTAMP";
+    private byte[] emptyCF;
+    private byte[] emptyCQ;
+
+    public PhoenixRowTimestampFunction() {
+    }
+
+    public PhoenixRowTimestampFunction(List<Expression> children, byte[] emptyCF, byte[] emptyCQ) {
+        super(children);
+        this.emptyCF = emptyCF;
+        this.emptyCQ = emptyCQ;
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+
+        if (tuple == null || tuple.size() == 0) return false;
+
+        long ts = tuple.getValue(0).getTimestamp();
 
 Review comment:
   Why do we do this and also later resolve the timestamp from `emptyColumnKV`? Shouldn't the below `if` condition always succeed and `ts` be evaluated that way?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] jpisaac commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
jpisaac commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r401293054
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRowTimestampFunctionIT.java
 ##########
 @@ -191,4 +192,158 @@ public void testRowTimestampColumn() throws Exception {
         }
     }
 
+    @Test
+    public void testRowTimestampColumnAndPredicates() throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR"
+                    + " CONSTRAINT PK PRIMARY KEY(PK1, PK2))" + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
+
+            String dml = "UPSERT INTO " + tableName + " (PK1, PK2, KV1, KV2) VALUES (?, ?, ?, ?)";
+
+            long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis();
+            Date rowTimestampDate1 = new Date(rowTimestamp - 120000);
+            PreparedStatement stmt = conn.prepareStatement(dml);
+
+            int count = 5;
+            for (int id = 0; id < count; ++id) {
+                int idValue = id;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate1);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            // case: No rows should be selected
+            // Since we used a past date for column PK2
+            String dql1 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV1 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() = PK2";
+
+            ResultSet rs1 = conn.createStatement().executeQuery(dql1);
+            assertTrue(!rs1.next());
+            rs1.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() > date column
+            // Since we used a past date for column PK2
+            String dql2 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV1 FROM " + tableName +
+                    " WHERE PK2 < PHOENIX_ROW_TIMESTAMP() ";
+
+            ResultSet rs2 = conn.createStatement().executeQuery(dql2);
+            int actualCount2 = 0;
+            while(rs2.next()) {
+                assertTrue(rs2.getDate(1).after(rowTimestampDate1));
+                actualCount2++;
+            }
+            assertEquals(count, actualCount2);
+            rs2.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() < date column
+            // So using a future date.
+            Date rowTimestampDate2 = new Date(rowTimestamp + 120000);
+            for (int id = 0; id < count; ++id) {
+                int idValue = id + 100;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate2);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            String dql3 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV2 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() < PK2";
+
+            ResultSet rs3 = conn.createStatement().executeQuery(dql3);
+            int actualCount3 = 0;
+            while(rs3.next()) {
+                assertTrue(rs3.getDate(1).before(rowTimestampDate2));
+                actualCount3++;
+            }
+            assertEquals(count, actualCount3);
+            rs3.close();
+        }
+    }
+
+    @Ignore("Fails when encoding=NON_ENCODED_QUALIFIERS and PHOENIX_ROW_TIMESTAMP "
+            + "is the only select column"
+            + "Expected rows do match expected:<5> but was:<0>")
+    public void testRowTimestampColumnOnlyAndPredicates() throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR"
+                    + " CONSTRAINT PK PRIMARY KEY(PK1, PK2))" + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
+
+            String dml = "UPSERT INTO " + tableName + " (PK1, PK2, KV1, KV2) VALUES (?, ?, ?, ?)";
+
+            long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis();
+            Date rowTimestampDate1 = new Date(rowTimestamp - 120000);
+            PreparedStatement stmt = conn.prepareStatement(dml);
+
+            int count = 5;
+            for (int id = 0; id < count; ++id) {
+                int idValue = id;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate1);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            // case: No rows should be selected
+            // Since we used a past date for column PK2
+            String dql1 = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() = PK2";
+
+            ResultSet rs1 = conn.createStatement().executeQuery(dql1);
+            assertTrue(!rs1.next());
+            rs1.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() > date column
+            // Since we used a past date for column PK2
+            String dql2 = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName +
+                    " WHERE PK2 < PHOENIX_ROW_TIMESTAMP() ";
+
+            ResultSet rs2 = conn.createStatement().executeQuery(dql2);
+            int actualCount2 = 0;
+            while(rs2.next()) {
+                assertTrue(rs2.getDate(1).after(rowTimestampDate1));
+                actualCount2++;
+            }
+            assertEquals(count, actualCount2);
+            rs2.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() < date column
+            // So using a future date.
+            Date rowTimestampDate2 = new Date(rowTimestamp + 120000);
+            for (int id = 0; id < count; ++id) {
+                int idValue = id + 100;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate2);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            String dql3 = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() < PK2";
+
+            ResultSet rs3 = conn.createStatement().executeQuery(dql3);
+            int actualCount3 = 0;
+            while(rs3.next()) {
+                assertTrue(rs3.getDate(1).before(rowTimestampDate2));
+                actualCount3++;
+            }
+            assertEquals(count, actualCount3);
+            rs3.close();
+        }
+    }
+
 }
 
 Review comment:
   The EmptyColumnIT tests do an exhaustive verification around that condition. Let me know if there is anything missing?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r406617850
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/expression/function/PhoenixRowTimestampFunction.java
 ##########
 @@ -0,0 +1,125 @@
+/*
+ * 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 org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.phoenix.expression.Determinism;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.parse.PhoenixRowTimestampParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Date;
+import java.util.List;
+
+/**
+ * Function to return the timestamp of the empty column which functions as the row timestamp. The
+ * result returned can be used for debugging(eg. using HBase shell), logging etc.
+ * Can also be used in sql predicates.
+ */
+@BuiltInFunction(name = PhoenixRowTimestampFunction.NAME,
+        nodeClass= PhoenixRowTimestampParseNode.class,
+        args = {})
+public class PhoenixRowTimestampFunction extends ScalarFunction {
+    public static final String NAME = "PHOENIX_ROW_TIMESTAMP";
+    private byte[] emptyCF;
+    private byte[] emptyCQ;
+
+    public PhoenixRowTimestampFunction() {
+    }
+
+    public PhoenixRowTimestampFunction(List<Expression> children, byte[] emptyCF, byte[] emptyCQ) {
+        super(children);
 
 Review comment:
   Where are we using `children`? We don't expect any args for this UDF and so this wouldn't really be used anywhere right?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r406605780
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRowTimestampFunctionIT.java
 ##########
 @@ -0,0 +1,398 @@
+/*
+ * 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 com.google.common.collect.Lists;
+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.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+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.PTable.ImmutableStorageScheme;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
+import org.apache.phoenix.util.EncodedColumnsUtil;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.Collection;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PhoenixRowTimestampFunctionIT extends ParallelStatsDisabledIT {
+    private final boolean encoded;
+    private final String tableDDLOptions;
+    private static final int NUM_ROWS = 5;
+    private static final long TS_OFFSET = 120000;
+
+    public PhoenixRowTimestampFunctionIT(QualifierEncodingScheme encoding,
+            ImmutableStorageScheme storage) {
+        StringBuilder optionBuilder = new StringBuilder();
+        optionBuilder.append(" COLUMN_ENCODED_BYTES = " + encoding.ordinal());
+        optionBuilder.append(",IMMUTABLE_STORAGE_SCHEME = "+ storage.toString());
+        this.tableDDLOptions = optionBuilder.toString();
+        this.encoded = (encoding != QualifierEncodingScheme.NON_ENCODED_QUALIFIERS) ? true : false;
+    }
+
+    @Parameterized.Parameters(name = "encoding={0},storage={1}")
+    public static synchronized Collection<Object[]> data() {
+        List<Object[]> list = Lists.newArrayList();
+        for (QualifierEncodingScheme encoding : QualifierEncodingScheme.values()) {
+            for (ImmutableStorageScheme storage : ImmutableStorageScheme.values()) {
+                list.add(new Object[]{encoding, storage});
+            }
+        }
+        return list;
+    }
+
+    private void verifyHbaseAllRowsTimestamp(String tableName, ResultSet rs, int expectedRowCount)
+            throws Exception {
+
+        Scan scan = new Scan();
+        byte[] emptyKVQualifier = EncodedColumnsUtil.getEmptyKeyValueInfo(this.encoded).getFirst();
+        try (org.apache.hadoop.hbase.client.Connection hconn =
+                ConnectionFactory.createConnection(config)) {
+            Table table = hconn.getTable(TableName.valueOf(tableName));
+            ResultScanner resultScanner = table.getScanner(scan);
+            int rowCount = 0;
+            while (rs.next()) {
+                Result result = resultScanner.next();
+                long timeStamp = result.getColumnLatestCell(
+                        QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES,
+                        emptyKVQualifier).getTimestamp();
+                assertEquals(rs.getDate(1).getTime(), timeStamp);
+                ++rowCount;
+            }
+            assertEquals(expectedRowCount, rowCount);
+        }
+    }
+
+    private void verifyHbaseRowTimestamp(String tableName, String rowKey, Date expectedTimestamp)
+            throws Exception {
+
+        byte[] emptyKVQualifier = EncodedColumnsUtil.getEmptyKeyValueInfo(this.encoded).getFirst();
+        try (org.apache.hadoop.hbase.client.Connection hconn =
+                ConnectionFactory.createConnection(config)) {
+            Table table = hconn.getTable(TableName.valueOf(tableName));
+            Get get = new Get(Bytes.toBytesBinary(rowKey));
+            Result result = table.get(get);
+            assertFalse(result.isEmpty());
+            long timeStamp = result.getColumnLatestCell(
+                    QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, emptyKVQualifier).getTimestamp();
+            assertEquals(expectedTimestamp.getTime(), timeStamp);
+        }
+    }
+
+    private String createTestData(long rowTimestamp, int numRows) throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            // Create a test table.
+            try (Statement stmt = conn.createStatement()) {
+                String ddl = "CREATE TABLE IF NOT EXISTS " + tableName +
+                        " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR" +
+                        " CONSTRAINT PK PRIMARY KEY(PK1, PK2))" + this.tableDDLOptions;
+                stmt.execute(ddl);
+            }
+
+            // Upsert data into the test table.
+            String dml = "UPSERT INTO " + tableName + " (PK1, PK2, KV1, KV2) VALUES (?, ?, ?, ?)";
+            try (PreparedStatement stmt = conn.prepareStatement(dml)) {
+                Date rowTimestampDate = new Date(rowTimestamp);
+                int count = numRows;
+                for (int id = 0; id < count; ++id) {
+                    int idValue = id;
+                    stmt.setInt(1, idValue);
+                    stmt.setDate(2, rowTimestampDate);
+                    stmt.setString(3, "KV1_" + idValue);
+                    stmt.setString(4, "KV2_" + idValue);
+                    stmt.executeUpdate();
+                }
+            }
+            conn.commit();
+        }
+        return tableName;
+    }
+
+    @Test
+    public void testRowTimestampDefault() throws Exception {
+
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK INTEGER NOT NULL PRIMARY KEY, KV1 VARCHAR, KV2 VARCHAR)"
+                    + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
+
+            String dml = "UPSERT INTO " + tableName + " (PK, KV1, KV2) VALUES (?, ?, ?)";
+            PreparedStatement stmt = conn.prepareStatement(dml);
+
+            int count = 5;
+            for (int id = 0; id < count; ++id) {
+                stmt.setInt(1, id);
+                stmt.setString(2, "KV1_" + id);
+                stmt.setString(3, "KV2_" + id);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            String dql = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName;
+
+            ResultSet rs = conn.createStatement().executeQuery(dql);
+            // verify row timestamp returned by the query matches the empty column cell timestamp
+            verifyHbaseAllRowsTimestamp(tableName, rs, count);
+
+            // update one row
+            conn.createStatement().execute("UPSERT INTO " + tableName
 
 Review comment:
   Same with this statement

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] swaroopak commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
swaroopak commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r392524283
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRowTimestampFunctionIT.java
 ##########
 @@ -191,4 +192,158 @@ public void testRowTimestampColumn() throws Exception {
         }
     }
 
+    @Test
+    public void testRowTimestampColumnAndPredicates() throws Exception {
 
 Review comment:
   all these cases deserve different test functions, please split them. extracting common code will make the test suite more maintainable. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] jpisaac closed pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
jpisaac closed pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r406607709
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
 ##########
 @@ -507,6 +507,10 @@ public static RowProjector compile(StatementContext context, SelectStatement sta
                 } else {
                     for (byte[] cq : entry.getValue()) {
                             PColumn column = family.getPColumnForColumnQualifier(cq);
+                            // Continue: If an EMPTY_COLUMN is in the projection list,
+                            // since the table column list does not contain the EMPTY_COLUMN
+                            // no value is returned.
+                            if (column == null) continue;
 
 Review comment:
   nit: Can we use braces and put the `continue` on a separate line to be consistent with formatting?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r392447781
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRowTimestampFunctionIT.java
 ##########
 @@ -191,4 +192,158 @@ public void testRowTimestampColumn() throws Exception {
         }
     }
 
+    @Test
+    public void testRowTimestampColumnAndPredicates() throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR"
+                    + " CONSTRAINT PK PRIMARY KEY(PK1, PK2))" + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
+
+            String dml = "UPSERT INTO " + tableName + " (PK1, PK2, KV1, KV2) VALUES (?, ?, ?, ?)";
+
+            long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis();
+            Date rowTimestampDate1 = new Date(rowTimestamp - 120000);
+            PreparedStatement stmt = conn.prepareStatement(dml);
+
+            int count = 5;
+            for (int id = 0; id < count; ++id) {
+                int idValue = id;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate1);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            // case: No rows should be selected
+            // Since we used a past date for column PK2
+            String dql1 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV1 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() = PK2";
+
+            ResultSet rs1 = conn.createStatement().executeQuery(dql1);
+            assertTrue(!rs1.next());
+            rs1.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() > date column
+            // Since we used a past date for column PK2
+            String dql2 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV1 FROM " + tableName +
+                    " WHERE PK2 < PHOENIX_ROW_TIMESTAMP() ";
+
+            ResultSet rs2 = conn.createStatement().executeQuery(dql2);
+            int actualCount2 = 0;
+            while(rs2.next()) {
+                assertTrue(rs2.getDate(1).after(rowTimestampDate1));
+                actualCount2++;
+            }
+            assertEquals(count, actualCount2);
+            rs2.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() < date column
+            // So using a future date.
+            Date rowTimestampDate2 = new Date(rowTimestamp + 120000);
+            for (int id = 0; id < count; ++id) {
+                int idValue = id + 100;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate2);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            String dql3 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV2 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() < PK2";
+
+            ResultSet rs3 = conn.createStatement().executeQuery(dql3);
+            int actualCount3 = 0;
+            while(rs3.next()) {
+                assertTrue(rs3.getDate(1).before(rowTimestampDate2));
+                actualCount3++;
+            }
+            assertEquals(count, actualCount3);
+            rs3.close();
+        }
+    }
+
+    @Ignore("Fails when encoding=NON_ENCODED_QUALIFIERS and PHOENIX_ROW_TIMESTAMP "
+            + "is the only select column"
+            + "Expected rows do match expected:<5> but was:<0>")
+    public void testRowTimestampColumnOnlyAndPredicates() throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR"
+                    + " CONSTRAINT PK PRIMARY KEY(PK1, PK2))" + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
+
+            String dml = "UPSERT INTO " + tableName + " (PK1, PK2, KV1, KV2) VALUES (?, ?, ?, ?)";
+
+            long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis();
+            Date rowTimestampDate1 = new Date(rowTimestamp - 120000);
+            PreparedStatement stmt = conn.prepareStatement(dml);
+
+            int count = 5;
+            for (int id = 0; id < count; ++id) {
+                int idValue = id;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate1);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            // case: No rows should be selected
+            // Since we used a past date for column PK2
+            String dql1 = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() = PK2";
+
+            ResultSet rs1 = conn.createStatement().executeQuery(dql1);
+            assertTrue(!rs1.next());
+            rs1.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() > date column
+            // Since we used a past date for column PK2
+            String dql2 = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName +
+                    " WHERE PK2 < PHOENIX_ROW_TIMESTAMP() ";
+
+            ResultSet rs2 = conn.createStatement().executeQuery(dql2);
+            int actualCount2 = 0;
+            while(rs2.next()) {
+                assertTrue(rs2.getDate(1).after(rowTimestampDate1));
+                actualCount2++;
+            }
+            assertEquals(count, actualCount2);
+            rs2.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() < date column
+            // So using a future date.
+            Date rowTimestampDate2 = new Date(rowTimestamp + 120000);
+            for (int id = 0; id < count; ++id) {
+                int idValue = id + 100;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate2);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            String dql3 = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() < PK2";
+
+            ResultSet rs3 = conn.createStatement().executeQuery(dql3);
+            int actualCount3 = 0;
+            while(rs3.next()) {
+                assertTrue(rs3.getDate(1).before(rowTimestampDate2));
+                actualCount3++;
+            }
+            assertEquals(count, actualCount3);
+            rs3.close();
+        }
 
 Review comment:
   This test case seems to have 90% of the code common as the first one. Can we refactor to use a common helper method for both to reduce repetition?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] jpisaac commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
jpisaac commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r402494157
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/parse/PhoenixRowTimestampParseNode.java
 ##########
 @@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.phoenix.parse;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.KeyValueColumnExpression;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.expression.function.PhoenixRowTimestampFunction;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PDatum;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.util.IndexUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+
+public class PhoenixRowTimestampParseNode extends FunctionParseNode {
+
+    PhoenixRowTimestampParseNode(String name, List<ParseNode> children,
+                                 BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+
+        // PhoenixRowTimestampFunction does not take any parameters.
+        assert children.size() == 0;
+
+        // Get the empty column family and qualifier for the context.
+        PTable table = context.getCurrentTable().getTable();
+        byte[] emptyColumnFamilyName = SchemaUtil.getEmptyColumnFamily(table);
+        byte[] emptyColumnName = table.getEncodingScheme() == PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS ?
+                QueryConstants.EMPTY_COLUMN_BYTES :
+                table.getEncodingScheme().encode(QueryConstants.ENCODED_EMPTY_COLUMN_NAME);
+
+        // Create an empty column key value expression.
+        // This will cause the empty column key value to evaluated during scan filter processing.
+        List<Expression> emptyColumnExpression = Arrays.asList(new Expression[] {new KeyValueColumnExpression(new PDatum() {
+            @Override
+            public boolean isNullable() {
+                return true;
+            }
+            @Override
+            public PDataType getDataType() {
+                return PDate.INSTANCE;
+            }
+            @Override
+            public Integer getMaxLength() {
+                return null;
+            }
+            @Override
+            public Integer getScale() {
+                return null;
+            }
+            @Override
+            public SortOrder getSortOrder() {
+                return SortOrder.getDefault();
+            }
+        }, emptyColumnFamilyName, emptyColumnName)});
+
+        // Add the empty column to the projection list.
+        // According to PHOENIX-4179 this will then return the timestamp of the empty column cell.
+        IndexUtil.addEmptyColumnToScan(context.getScan(), emptyColumnFamilyName, emptyColumnName);
+        return new PhoenixRowTimestampFunction(emptyColumnExpression, emptyColumnFamilyName, emptyColumnName);
+    }
+}
 
 Review comment:
   Added it the unit tests

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r392447449
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRowTimestampFunctionIT.java
 ##########
 @@ -191,4 +192,158 @@ public void testRowTimestampColumn() throws Exception {
         }
     }
 
+    @Test
+    public void testRowTimestampColumnAndPredicates() throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR"
+                    + " CONSTRAINT PK PRIMARY KEY(PK1, PK2))" + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
+
+            String dml = "UPSERT INTO " + tableName + " (PK1, PK2, KV1, KV2) VALUES (?, ?, ?, ?)";
+
+            long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis();
+            Date rowTimestampDate1 = new Date(rowTimestamp - 120000);
+            PreparedStatement stmt = conn.prepareStatement(dml);
+
+            int count = 5;
+            for (int id = 0; id < count; ++id) {
+                int idValue = id;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate1);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            // case: No rows should be selected
+            // Since we used a past date for column PK2
+            String dql1 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV1 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() = PK2";
+
+            ResultSet rs1 = conn.createStatement().executeQuery(dql1);
+            assertTrue(!rs1.next());
+            rs1.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() > date column
+            // Since we used a past date for column PK2
+            String dql2 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV1 FROM " + tableName +
+                    " WHERE PK2 < PHOENIX_ROW_TIMESTAMP() ";
+
+            ResultSet rs2 = conn.createStatement().executeQuery(dql2);
+            int actualCount2 = 0;
+            while(rs2.next()) {
+                assertTrue(rs2.getDate(1).after(rowTimestampDate1));
+                actualCount2++;
+            }
+            assertEquals(count, actualCount2);
+            rs2.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() < date column
+            // So using a future date.
+            Date rowTimestampDate2 = new Date(rowTimestamp + 120000);
+            for (int id = 0; id < count; ++id) {
+                int idValue = id + 100;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate2);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            String dql3 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV2 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() < PK2";
+
+            ResultSet rs3 = conn.createStatement().executeQuery(dql3);
+            int actualCount3 = 0;
+            while(rs3.next()) {
+                assertTrue(rs3.getDate(1).before(rowTimestampDate2));
+                actualCount3++;
+            }
+            assertEquals(count, actualCount3);
+            rs3.close();
+        }
+    }
+
+    @Ignore("Fails when encoding=NON_ENCODED_QUALIFIERS and PHOENIX_ROW_TIMESTAMP "
+            + "is the only select column"
+            + "Expected rows do match expected:<5> but was:<0>")
+    public void testRowTimestampColumnOnlyAndPredicates() throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR"
+                    + " CONSTRAINT PK PRIMARY KEY(PK1, PK2))" + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
+
+            String dml = "UPSERT INTO " + tableName + " (PK1, PK2, KV1, KV2) VALUES (?, ?, ?, ?)";
+
+            long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis();
+            Date rowTimestampDate1 = new Date(rowTimestamp - 120000);
+            PreparedStatement stmt = conn.prepareStatement(dml);
+
+            int count = 5;
+            for (int id = 0; id < count; ++id) {
+                int idValue = id;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate1);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            // case: No rows should be selected
+            // Since we used a past date for column PK2
+            String dql1 = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() = PK2";
+
+            ResultSet rs1 = conn.createStatement().executeQuery(dql1);
+            assertTrue(!rs1.next());
 
 Review comment:
   Use assertFalse instead

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r392443993
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRowTimestampFunctionIT.java
 ##########
 @@ -191,4 +192,158 @@ public void testRowTimestampColumn() throws Exception {
         }
     }
 
+    @Test
+    public void testRowTimestampColumnAndPredicates() throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR"
+                    + " CONSTRAINT PK PRIMARY KEY(PK1, PK2))" + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
 
 Review comment:
   nit: Add statement creation to try-with-resources as well.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] yanxinyi commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
yanxinyi commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r405807785
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRowTimestampFunctionIT.java
 ##########
 @@ -31,6 +31,7 @@
 import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.EnvironmentEdgeManager;
+import org.junit.Ignore;
 
 Review comment:
   ignore import is not needed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r406605714
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRowTimestampFunctionIT.java
 ##########
 @@ -0,0 +1,398 @@
+/*
+ * 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 com.google.common.collect.Lists;
+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.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+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.PTable.ImmutableStorageScheme;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
+import org.apache.phoenix.util.EncodedColumnsUtil;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.Collection;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PhoenixRowTimestampFunctionIT extends ParallelStatsDisabledIT {
+    private final boolean encoded;
+    private final String tableDDLOptions;
+    private static final int NUM_ROWS = 5;
+    private static final long TS_OFFSET = 120000;
+
+    public PhoenixRowTimestampFunctionIT(QualifierEncodingScheme encoding,
+            ImmutableStorageScheme storage) {
+        StringBuilder optionBuilder = new StringBuilder();
+        optionBuilder.append(" COLUMN_ENCODED_BYTES = " + encoding.ordinal());
+        optionBuilder.append(",IMMUTABLE_STORAGE_SCHEME = "+ storage.toString());
+        this.tableDDLOptions = optionBuilder.toString();
+        this.encoded = (encoding != QualifierEncodingScheme.NON_ENCODED_QUALIFIERS) ? true : false;
+    }
+
+    @Parameterized.Parameters(name = "encoding={0},storage={1}")
+    public static synchronized Collection<Object[]> data() {
+        List<Object[]> list = Lists.newArrayList();
+        for (QualifierEncodingScheme encoding : QualifierEncodingScheme.values()) {
+            for (ImmutableStorageScheme storage : ImmutableStorageScheme.values()) {
+                list.add(new Object[]{encoding, storage});
+            }
+        }
+        return list;
+    }
+
+    private void verifyHbaseAllRowsTimestamp(String tableName, ResultSet rs, int expectedRowCount)
+            throws Exception {
+
+        Scan scan = new Scan();
+        byte[] emptyKVQualifier = EncodedColumnsUtil.getEmptyKeyValueInfo(this.encoded).getFirst();
+        try (org.apache.hadoop.hbase.client.Connection hconn =
+                ConnectionFactory.createConnection(config)) {
+            Table table = hconn.getTable(TableName.valueOf(tableName));
+            ResultScanner resultScanner = table.getScanner(scan);
+            int rowCount = 0;
+            while (rs.next()) {
+                Result result = resultScanner.next();
+                long timeStamp = result.getColumnLatestCell(
+                        QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES,
+                        emptyKVQualifier).getTimestamp();
+                assertEquals(rs.getDate(1).getTime(), timeStamp);
+                ++rowCount;
+            }
+            assertEquals(expectedRowCount, rowCount);
+        }
+    }
+
+    private void verifyHbaseRowTimestamp(String tableName, String rowKey, Date expectedTimestamp)
+            throws Exception {
+
+        byte[] emptyKVQualifier = EncodedColumnsUtil.getEmptyKeyValueInfo(this.encoded).getFirst();
+        try (org.apache.hadoop.hbase.client.Connection hconn =
+                ConnectionFactory.createConnection(config)) {
+            Table table = hconn.getTable(TableName.valueOf(tableName));
+            Get get = new Get(Bytes.toBytesBinary(rowKey));
+            Result result = table.get(get);
+            assertFalse(result.isEmpty());
+            long timeStamp = result.getColumnLatestCell(
+                    QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, emptyKVQualifier).getTimestamp();
+            assertEquals(expectedTimestamp.getTime(), timeStamp);
+        }
+    }
+
+    private String createTestData(long rowTimestamp, int numRows) throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            // Create a test table.
+            try (Statement stmt = conn.createStatement()) {
+                String ddl = "CREATE TABLE IF NOT EXISTS " + tableName +
+                        " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR" +
+                        " CONSTRAINT PK PRIMARY KEY(PK1, PK2))" + this.tableDDLOptions;
+                stmt.execute(ddl);
+            }
+
+            // Upsert data into the test table.
+            String dml = "UPSERT INTO " + tableName + " (PK1, PK2, KV1, KV2) VALUES (?, ?, ?, ?)";
+            try (PreparedStatement stmt = conn.prepareStatement(dml)) {
+                Date rowTimestampDate = new Date(rowTimestamp);
+                int count = numRows;
+                for (int id = 0; id < count; ++id) {
+                    int idValue = id;
+                    stmt.setInt(1, idValue);
+                    stmt.setDate(2, rowTimestampDate);
+                    stmt.setString(3, "KV1_" + idValue);
+                    stmt.setString(4, "KV2_" + idValue);
+                    stmt.executeUpdate();
+                }
+            }
+            conn.commit();
+        }
+        return tableName;
+    }
+
+    @Test
+    public void testRowTimestampDefault() throws Exception {
+
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK INTEGER NOT NULL PRIMARY KEY, KV1 VARCHAR, KV2 VARCHAR)"
+                    + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
+
+            String dml = "UPSERT INTO " + tableName + " (PK, KV1, KV2) VALUES (?, ?, ?)";
+            PreparedStatement stmt = conn.prepareStatement(dml);
 
 Review comment:
   nit: Use try-with-resources to close this statement?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] yanxinyi commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
yanxinyi commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r389224355
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRowTimestampFunctionIT.java
 ##########
 @@ -191,4 +192,158 @@ public void testRowTimestampColumn() throws Exception {
         }
     }
 
+    @Test
+    public void testRowTimestampColumnAndPredicates() throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR"
+                    + " CONSTRAINT PK PRIMARY KEY(PK1, PK2))" + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
+
+            String dml = "UPSERT INTO " + tableName + " (PK1, PK2, KV1, KV2) VALUES (?, ?, ?, ?)";
+
+            long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis();
+            Date rowTimestampDate1 = new Date(rowTimestamp - 120000);
+            PreparedStatement stmt = conn.prepareStatement(dml);
+
+            int count = 5;
+            for (int id = 0; id < count; ++id) {
+                int idValue = id;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate1);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            // case: No rows should be selected
+            // Since we used a past date for column PK2
+            String dql1 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV1 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() = PK2";
+
+            ResultSet rs1 = conn.createStatement().executeQuery(dql1);
+            assertTrue(!rs1.next());
+            rs1.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() > date column
+            // Since we used a past date for column PK2
+            String dql2 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV1 FROM " + tableName +
+                    " WHERE PK2 < PHOENIX_ROW_TIMESTAMP() ";
+
+            ResultSet rs2 = conn.createStatement().executeQuery(dql2);
+            int actualCount2 = 0;
+            while(rs2.next()) {
+                assertTrue(rs2.getDate(1).after(rowTimestampDate1));
+                actualCount2++;
+            }
+            assertEquals(count, actualCount2);
+            rs2.close();
+
+            // case: All rows selected should have the phoenix_row_timestamp() < date column
+            // So using a future date.
+            Date rowTimestampDate2 = new Date(rowTimestamp + 120000);
+            for (int id = 0; id < count; ++id) {
+                int idValue = id + 100;
+                stmt.setInt(1, idValue);
+                stmt.setDate(2, rowTimestampDate2);
+                stmt.setString(3, "KV1_" + idValue);
+                stmt.setString(4, "KV2_" + idValue);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            String dql3 = "SELECT PHOENIX_ROW_TIMESTAMP(), KV2 FROM " + tableName +
+                    " WHERE PHOENIX_ROW_TIMESTAMP() < PK2";
+
+            ResultSet rs3 = conn.createStatement().executeQuery(dql3);
+            int actualCount3 = 0;
+            while(rs3.next()) {
+                assertTrue(rs3.getDate(1).before(rowTimestampDate2));
+                actualCount3++;
+            }
+            assertEquals(count, actualCount3);
+            rs3.close();
+        }
+    }
+
+    @Ignore("Fails when encoding=NON_ENCODED_QUALIFIERS and PHOENIX_ROW_TIMESTAMP "
+            + "is the only select column"
 
 Review comment:
   why we can't get the right behavior of `SELECT PHOENIX_ROW_TIMESTAMP()` only here?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] jpisaac commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
jpisaac commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r406952826
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/parse/PhoenixRowTimestampParseNode.java
 ##########
 @@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.phoenix.parse;
+
+import org.apache.phoenix.compile.FromCompiler;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.KeyValueColumnExpression;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.expression.function.PhoenixRowTimestampFunction;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PDatum;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.util.IndexUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+
+public class PhoenixRowTimestampParseNode extends FunctionParseNode {
+
+    PhoenixRowTimestampParseNode(String name, List<ParseNode> children,
+                                 BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+
+        // PhoenixRowTimestampFunction does not take any parameters.
+        if (children.size() != 0) {
+            throw new IllegalArgumentException("PhoenixRowTimestampFunction does not take any parameters");
+        }
+
+        // Get the empty column family and qualifier for the context.
+        PTable table = context.getCurrentTable().getTable();
+        byte[] emptyColumnFamilyName = SchemaUtil.getEmptyColumnFamily(table);
+        byte[] emptyColumnName = table.getEncodingScheme() == PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS ?
+                QueryConstants.EMPTY_COLUMN_BYTES :
+                table.getEncodingScheme().encode(QueryConstants.ENCODED_EMPTY_COLUMN_NAME);
+
+        // Create an empty column key value expression.
+        // This will cause the empty column key value to evaluated during scan filter processing.
+        List<Expression> emptyColumnExpression = Arrays.asList(new Expression[] {new KeyValueColumnExpression(new PDatum() {
 
 Review comment:
   As discussed offline, adding more comments.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] yanxinyi commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
yanxinyi commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r389222892
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRowTimestampFunctionIT.java
 ##########
 @@ -191,4 +192,158 @@ public void testRowTimestampColumn() throws Exception {
         }
     }
 
+    @Test
+    public void testRowTimestampColumnAndPredicates() throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR"
+                    + " CONSTRAINT PK PRIMARY KEY(PK1, PK2))" + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
+
+            String dml = "UPSERT INTO " + tableName + " (PK1, PK2, KV1, KV2) VALUES (?, ?, ?, ?)";
+
+            long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis();
+            Date rowTimestampDate1 = new Date(rowTimestamp - 120000);
 
 Review comment:
   why 120000? some random number? can you make it as constant?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r392453053
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/parse/PhoenixRowTimestampParseNode.java
 ##########
 @@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.phoenix.parse;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.KeyValueColumnExpression;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.expression.function.PhoenixRowTimestampFunction;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PDatum;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.util.IndexUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+
+public class PhoenixRowTimestampParseNode extends FunctionParseNode {
+
+    PhoenixRowTimestampParseNode(String name, List<ParseNode> children,
+                                 BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+
+        // PhoenixRowTimestampFunction does not take any parameters.
+        assert children.size() == 0;
+
+        // Get the empty column family and qualifier for the context.
+        PTable table = context.getCurrentTable().getTable();
+        byte[] emptyColumnFamilyName = SchemaUtil.getEmptyColumnFamily(table);
+        byte[] emptyColumnName = table.getEncodingScheme() == PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS ?
+                QueryConstants.EMPTY_COLUMN_BYTES :
+                table.getEncodingScheme().encode(QueryConstants.ENCODED_EMPTY_COLUMN_NAME);
+
+        // Create an empty column key value expression.
+        // This will cause the empty column key value to evaluated during scan filter processing.
+        List<Expression> emptyColumnExpression = Arrays.asList(new Expression[] {new KeyValueColumnExpression(new PDatum() {
+            @Override
+            public boolean isNullable() {
+                return true;
+            }
+            @Override
+            public PDataType getDataType() {
+                return PDate.INSTANCE;
+            }
+            @Override
+            public Integer getMaxLength() {
+                return null;
+            }
+            @Override
+            public Integer getScale() {
+                return null;
+            }
+            @Override
+            public SortOrder getSortOrder() {
+                return SortOrder.getDefault();
+            }
+        }, emptyColumnFamilyName, emptyColumnName)});
+
+        // Add the empty column to the projection list.
+        // According to PHOENIX-4179 this will then return the timestamp of the empty column cell.
+        IndexUtil.addEmptyColumnToScan(context.getScan(), emptyColumnFamilyName, emptyColumnName);
+        return new PhoenixRowTimestampFunction(emptyColumnExpression, emptyColumnFamilyName, emptyColumnName);
+    }
+}
 
 Review comment:
   @jpisaac let's add unit tests for the newly added classes, however small the test classes may be. Good for code coverage.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] jpisaac commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
jpisaac commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r406953489
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/expression/function/PhoenixRowTimestampFunction.java
 ##########
 @@ -0,0 +1,125 @@
+/*
+ * 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 org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.phoenix.expression.Determinism;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.parse.PhoenixRowTimestampParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Date;
+import java.util.List;
+
+/**
+ * Function to return the timestamp of the empty column which functions as the row timestamp. The
+ * result returned can be used for debugging(eg. using HBase shell), logging etc.
+ * Can also be used in sql predicates.
+ */
+@BuiltInFunction(name = PhoenixRowTimestampFunction.NAME,
+        nodeClass= PhoenixRowTimestampParseNode.class,
+        args = {})
+public class PhoenixRowTimestampFunction extends ScalarFunction {
+    public static final String NAME = "PHOENIX_ROW_TIMESTAMP";
+    private byte[] emptyCF;
+    private byte[] emptyCQ;
+
+    public PhoenixRowTimestampFunction() {
+    }
+
+    public PhoenixRowTimestampFunction(List<Expression> children, byte[] emptyCF, byte[] emptyCQ) {
+        super(children);
 
 Review comment:
   Will add more comments as discussed offline.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r406611949
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/expression/function/PhoenixRowTimestampFunction.java
 ##########
 @@ -0,0 +1,125 @@
+/*
+ * 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 org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.phoenix.expression.Determinism;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.parse.PhoenixRowTimestampParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Date;
+import java.util.List;
+
+/**
+ * Function to return the timestamp of the empty column which functions as the row timestamp. The
+ * result returned can be used for debugging(eg. using HBase shell), logging etc.
+ * Can also be used in sql predicates.
+ */
+@BuiltInFunction(name = PhoenixRowTimestampFunction.NAME,
+        nodeClass= PhoenixRowTimestampParseNode.class,
+        args = {})
 
 Review comment:
   What is the behavior if we were to call `PHOENIX_ROW_TIMESTAMP(<arg(s)>)`? Does specifying `args = {}` cut it, or do we need to throw a specific exception to indicate improper usage?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] tkhurana commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
tkhurana commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r389212096
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/expression/function/PhoenixRowTimestampFunction.java
 ##########
 @@ -0,0 +1,136 @@
+/*
+ * 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 org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.phoenix.expression.Determinism;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.parse.PhoenixRowTimestampParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Date;
+import java.util.List;
+
+/**
+ * Function to return the timestamp of the empty column which functions as the row timestamp. The
+ * result returned can be used for debugging(eg. using HBase shell), logging etc.
+ * Can also be used in sql predicates.
+ */
+@BuiltInFunction(name = PhoenixRowTimestampFunction.NAME,
+        nodeClass= PhoenixRowTimestampParseNode.class,
+        args = {})
+public class PhoenixRowTimestampFunction extends ScalarFunction {
+    private static final Logger LOGGER = LoggerFactory.getLogger(PhoenixRowTimestampFunction.class);
+
+    public static final String NAME = "PHOENIX_ROW_TIMESTAMP";
+    private byte[] emptyCF;
+    private byte[] emptyCQ;
+
+    public PhoenixRowTimestampFunction() {
+    }
+
+    public PhoenixRowTimestampFunction(List<Expression> children, byte[] emptyCF, byte[] emptyCQ) {
+        super(children);
+        this.emptyCF = emptyCF;
+        this.emptyCQ = emptyCQ;
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+
+        if (tuple == null || tuple.size() == 0) return false;
+
+        long ts = tuple.getValue(0).getTimestamp();
+        Cell emptyColumnKV = tuple.getValue(emptyCF, emptyCQ);
+        if ((emptyColumnKV != null) && isEmptyColumn(emptyColumnKV)) {
+            ts = emptyColumnKV.getTimestamp();
+        }
+        Date rowTimestamp = new Date(ts);
+        ptr.set(PDate.INSTANCE.toBytes(rowTimestamp));
+        return true;
+    }
+
+    private boolean isEmptyColumn(Cell cell) {
+        return Bytes.compareTo(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
+                emptyCF, 0, emptyCF.length) == 0 &&
+                Bytes.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
+                        emptyCQ, 0, emptyCQ.length) == 0;
 
 Review comment:
   We could use CellUtil.matchingColumn(cell, emptyCF, emptyCQ)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] jpisaac commented on issue #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
jpisaac commented on issue #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#issuecomment-611813507
 
 
   @swaroopak @ChinmaySKulkarni @yanxinyi Please review the updates.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r406605821
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRowTimestampFunctionIT.java
 ##########
 @@ -0,0 +1,398 @@
+/*
+ * 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 com.google.common.collect.Lists;
+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.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+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.PTable.ImmutableStorageScheme;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
+import org.apache.phoenix.util.EncodedColumnsUtil;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.Collection;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PhoenixRowTimestampFunctionIT extends ParallelStatsDisabledIT {
+    private final boolean encoded;
+    private final String tableDDLOptions;
+    private static final int NUM_ROWS = 5;
+    private static final long TS_OFFSET = 120000;
+
+    public PhoenixRowTimestampFunctionIT(QualifierEncodingScheme encoding,
+            ImmutableStorageScheme storage) {
+        StringBuilder optionBuilder = new StringBuilder();
+        optionBuilder.append(" COLUMN_ENCODED_BYTES = " + encoding.ordinal());
+        optionBuilder.append(",IMMUTABLE_STORAGE_SCHEME = "+ storage.toString());
+        this.tableDDLOptions = optionBuilder.toString();
+        this.encoded = (encoding != QualifierEncodingScheme.NON_ENCODED_QUALIFIERS) ? true : false;
+    }
+
+    @Parameterized.Parameters(name = "encoding={0},storage={1}")
+    public static synchronized Collection<Object[]> data() {
+        List<Object[]> list = Lists.newArrayList();
+        for (QualifierEncodingScheme encoding : QualifierEncodingScheme.values()) {
+            for (ImmutableStorageScheme storage : ImmutableStorageScheme.values()) {
+                list.add(new Object[]{encoding, storage});
+            }
+        }
+        return list;
+    }
+
+    private void verifyHbaseAllRowsTimestamp(String tableName, ResultSet rs, int expectedRowCount)
+            throws Exception {
+
+        Scan scan = new Scan();
+        byte[] emptyKVQualifier = EncodedColumnsUtil.getEmptyKeyValueInfo(this.encoded).getFirst();
+        try (org.apache.hadoop.hbase.client.Connection hconn =
+                ConnectionFactory.createConnection(config)) {
+            Table table = hconn.getTable(TableName.valueOf(tableName));
+            ResultScanner resultScanner = table.getScanner(scan);
+            int rowCount = 0;
+            while (rs.next()) {
+                Result result = resultScanner.next();
+                long timeStamp = result.getColumnLatestCell(
+                        QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES,
+                        emptyKVQualifier).getTimestamp();
+                assertEquals(rs.getDate(1).getTime(), timeStamp);
+                ++rowCount;
+            }
+            assertEquals(expectedRowCount, rowCount);
+        }
+    }
+
+    private void verifyHbaseRowTimestamp(String tableName, String rowKey, Date expectedTimestamp)
+            throws Exception {
+
+        byte[] emptyKVQualifier = EncodedColumnsUtil.getEmptyKeyValueInfo(this.encoded).getFirst();
+        try (org.apache.hadoop.hbase.client.Connection hconn =
+                ConnectionFactory.createConnection(config)) {
+            Table table = hconn.getTable(TableName.valueOf(tableName));
+            Get get = new Get(Bytes.toBytesBinary(rowKey));
+            Result result = table.get(get);
+            assertFalse(result.isEmpty());
+            long timeStamp = result.getColumnLatestCell(
+                    QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, emptyKVQualifier).getTimestamp();
+            assertEquals(expectedTimestamp.getTime(), timeStamp);
+        }
+    }
+
+    private String createTestData(long rowTimestamp, int numRows) throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            // Create a test table.
+            try (Statement stmt = conn.createStatement()) {
+                String ddl = "CREATE TABLE IF NOT EXISTS " + tableName +
+                        " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR" +
+                        " CONSTRAINT PK PRIMARY KEY(PK1, PK2))" + this.tableDDLOptions;
+                stmt.execute(ddl);
+            }
+
+            // Upsert data into the test table.
+            String dml = "UPSERT INTO " + tableName + " (PK1, PK2, KV1, KV2) VALUES (?, ?, ?, ?)";
+            try (PreparedStatement stmt = conn.prepareStatement(dml)) {
+                Date rowTimestampDate = new Date(rowTimestamp);
+                int count = numRows;
+                for (int id = 0; id < count; ++id) {
+                    int idValue = id;
+                    stmt.setInt(1, idValue);
+                    stmt.setDate(2, rowTimestampDate);
+                    stmt.setString(3, "KV1_" + idValue);
+                    stmt.setString(4, "KV2_" + idValue);
+                    stmt.executeUpdate();
+                }
+            }
+            conn.commit();
+        }
+        return tableName;
+    }
+
+    @Test
+    public void testRowTimestampDefault() throws Exception {
+
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK INTEGER NOT NULL PRIMARY KEY, KV1 VARCHAR, KV2 VARCHAR)"
+                    + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
+
+            String dml = "UPSERT INTO " + tableName + " (PK, KV1, KV2) VALUES (?, ?, ?)";
+            PreparedStatement stmt = conn.prepareStatement(dml);
+
+            int count = 5;
+            for (int id = 0; id < count; ++id) {
+                stmt.setInt(1, id);
+                stmt.setString(2, "KV1_" + id);
+                stmt.setString(3, "KV2_" + id);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            String dql = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName;
+
+            ResultSet rs = conn.createStatement().executeQuery(dql);
+            // verify row timestamp returned by the query matches the empty column cell timestamp
+            verifyHbaseAllRowsTimestamp(tableName, rs, count);
+
+            // update one row
+            conn.createStatement().execute("UPSERT INTO " + tableName
+                    + " (PK, KV1) VALUES (2, 'KV1_foo')");
+            conn.commit();
+
+            rs = conn.createStatement().executeQuery(dql);
 
 Review comment:
   ditto

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] jpisaac commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
jpisaac commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r402494157
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/parse/PhoenixRowTimestampParseNode.java
 ##########
 @@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.phoenix.parse;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.KeyValueColumnExpression;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.expression.function.PhoenixRowTimestampFunction;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PDatum;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.util.IndexUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+
+public class PhoenixRowTimestampParseNode extends FunctionParseNode {
+
+    PhoenixRowTimestampParseNode(String name, List<ParseNode> children,
+                                 BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+
+        // PhoenixRowTimestampFunction does not take any parameters.
+        assert children.size() == 0;
+
+        // Get the empty column family and qualifier for the context.
+        PTable table = context.getCurrentTable().getTable();
+        byte[] emptyColumnFamilyName = SchemaUtil.getEmptyColumnFamily(table);
+        byte[] emptyColumnName = table.getEncodingScheme() == PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS ?
+                QueryConstants.EMPTY_COLUMN_BYTES :
+                table.getEncodingScheme().encode(QueryConstants.ENCODED_EMPTY_COLUMN_NAME);
+
+        // Create an empty column key value expression.
+        // This will cause the empty column key value to evaluated during scan filter processing.
+        List<Expression> emptyColumnExpression = Arrays.asList(new Expression[] {new KeyValueColumnExpression(new PDatum() {
+            @Override
+            public boolean isNullable() {
+                return true;
+            }
+            @Override
+            public PDataType getDataType() {
+                return PDate.INSTANCE;
+            }
+            @Override
+            public Integer getMaxLength() {
+                return null;
+            }
+            @Override
+            public Integer getScale() {
+                return null;
+            }
+            @Override
+            public SortOrder getSortOrder() {
+                return SortOrder.getDefault();
+            }
+        }, emptyColumnFamilyName, emptyColumnName)});
+
+        // Add the empty column to the projection list.
+        // According to PHOENIX-4179 this will then return the timestamp of the empty column cell.
+        IndexUtil.addEmptyColumnToScan(context.getScan(), emptyColumnFamilyName, emptyColumnName);
+        return new PhoenixRowTimestampFunction(emptyColumnExpression, emptyColumnFamilyName, emptyColumnName);
+    }
+}
 
 Review comment:
   Added the unit tests

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r406618862
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/expression/function/PhoenixRowTimestampFunction.java
 ##########
 @@ -0,0 +1,125 @@
+/*
+ * 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 org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.phoenix.expression.Determinism;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.parse.PhoenixRowTimestampParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Date;
+import java.util.List;
+
+/**
+ * Function to return the timestamp of the empty column which functions as the row timestamp. The
+ * result returned can be used for debugging(eg. using HBase shell), logging etc.
+ * Can also be used in sql predicates.
+ */
+@BuiltInFunction(name = PhoenixRowTimestampFunction.NAME,
+        nodeClass= PhoenixRowTimestampParseNode.class,
+        args = {})
+public class PhoenixRowTimestampFunction extends ScalarFunction {
+    public static final String NAME = "PHOENIX_ROW_TIMESTAMP";
+    private byte[] emptyCF;
+    private byte[] emptyCQ;
+
+    public PhoenixRowTimestampFunction() {
+    }
+
+    public PhoenixRowTimestampFunction(List<Expression> children, byte[] emptyCF, byte[] emptyCQ) {
+        super(children);
+        this.emptyCF = emptyCF;
+        this.emptyCQ = emptyCQ;
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+
+        if (tuple == null || tuple.size() == 0) return false;
+
+        long ts = tuple.getValue(0).getTimestamp();
 
 Review comment:
   If so, this initialization of `ts` would always be overwritten right? Or is there some case wherein the below `if` condition is false and so we use this first initialization?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r392450994
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/parse/PhoenixRowTimestampParseNode.java
 ##########
 @@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.phoenix.parse;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.KeyValueColumnExpression;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.expression.function.PhoenixRowTimestampFunction;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PDatum;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.util.IndexUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+
+public class PhoenixRowTimestampParseNode extends FunctionParseNode {
+
+    PhoenixRowTimestampParseNode(String name, List<ParseNode> children,
+                                 BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+
+        // PhoenixRowTimestampFunction does not take any parameters.
+        assert children.size() == 0;
 
 Review comment:
   This is a user error, so it will be better to throw an exception so the user understands the correct usage of PHOENIX_ROW_TIMESTAMP. Can we do that instead?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r406605958
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRowTimestampFunctionIT.java
 ##########
 @@ -0,0 +1,398 @@
+/*
+ * 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 com.google.common.collect.Lists;
+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.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+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.PTable.ImmutableStorageScheme;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
+import org.apache.phoenix.util.EncodedColumnsUtil;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.Collection;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PhoenixRowTimestampFunctionIT extends ParallelStatsDisabledIT {
+    private final boolean encoded;
+    private final String tableDDLOptions;
+    private static final int NUM_ROWS = 5;
+    private static final long TS_OFFSET = 120000;
+
+    public PhoenixRowTimestampFunctionIT(QualifierEncodingScheme encoding,
+            ImmutableStorageScheme storage) {
+        StringBuilder optionBuilder = new StringBuilder();
+        optionBuilder.append(" COLUMN_ENCODED_BYTES = " + encoding.ordinal());
+        optionBuilder.append(",IMMUTABLE_STORAGE_SCHEME = "+ storage.toString());
+        this.tableDDLOptions = optionBuilder.toString();
+        this.encoded = (encoding != QualifierEncodingScheme.NON_ENCODED_QUALIFIERS) ? true : false;
+    }
+
+    @Parameterized.Parameters(name = "encoding={0},storage={1}")
+    public static synchronized Collection<Object[]> data() {
+        List<Object[]> list = Lists.newArrayList();
+        for (QualifierEncodingScheme encoding : QualifierEncodingScheme.values()) {
+            for (ImmutableStorageScheme storage : ImmutableStorageScheme.values()) {
+                list.add(new Object[]{encoding, storage});
+            }
+        }
+        return list;
+    }
+
+    private void verifyHbaseAllRowsTimestamp(String tableName, ResultSet rs, int expectedRowCount)
+            throws Exception {
+
+        Scan scan = new Scan();
+        byte[] emptyKVQualifier = EncodedColumnsUtil.getEmptyKeyValueInfo(this.encoded).getFirst();
+        try (org.apache.hadoop.hbase.client.Connection hconn =
+                ConnectionFactory.createConnection(config)) {
+            Table table = hconn.getTable(TableName.valueOf(tableName));
+            ResultScanner resultScanner = table.getScanner(scan);
+            int rowCount = 0;
+            while (rs.next()) {
+                Result result = resultScanner.next();
+                long timeStamp = result.getColumnLatestCell(
+                        QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES,
+                        emptyKVQualifier).getTimestamp();
+                assertEquals(rs.getDate(1).getTime(), timeStamp);
+                ++rowCount;
+            }
+            assertEquals(expectedRowCount, rowCount);
+        }
+    }
+
+    private void verifyHbaseRowTimestamp(String tableName, String rowKey, Date expectedTimestamp)
+            throws Exception {
+
+        byte[] emptyKVQualifier = EncodedColumnsUtil.getEmptyKeyValueInfo(this.encoded).getFirst();
+        try (org.apache.hadoop.hbase.client.Connection hconn =
+                ConnectionFactory.createConnection(config)) {
+            Table table = hconn.getTable(TableName.valueOf(tableName));
+            Get get = new Get(Bytes.toBytesBinary(rowKey));
+            Result result = table.get(get);
+            assertFalse(result.isEmpty());
+            long timeStamp = result.getColumnLatestCell(
+                    QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, emptyKVQualifier).getTimestamp();
+            assertEquals(expectedTimestamp.getTime(), timeStamp);
+        }
+    }
+
+    private String createTestData(long rowTimestamp, int numRows) throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            // Create a test table.
+            try (Statement stmt = conn.createStatement()) {
+                String ddl = "CREATE TABLE IF NOT EXISTS " + tableName +
+                        " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR" +
+                        " CONSTRAINT PK PRIMARY KEY(PK1, PK2))" + this.tableDDLOptions;
+                stmt.execute(ddl);
+            }
+
+            // Upsert data into the test table.
+            String dml = "UPSERT INTO " + tableName + " (PK1, PK2, KV1, KV2) VALUES (?, ?, ?, ?)";
+            try (PreparedStatement stmt = conn.prepareStatement(dml)) {
+                Date rowTimestampDate = new Date(rowTimestamp);
+                int count = numRows;
+                for (int id = 0; id < count; ++id) {
+                    int idValue = id;
+                    stmt.setInt(1, idValue);
+                    stmt.setDate(2, rowTimestampDate);
+                    stmt.setString(3, "KV1_" + idValue);
+                    stmt.setString(4, "KV2_" + idValue);
+                    stmt.executeUpdate();
+                }
+            }
+            conn.commit();
+        }
+        return tableName;
+    }
+
+    @Test
+    public void testRowTimestampDefault() throws Exception {
+
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK INTEGER NOT NULL PRIMARY KEY, KV1 VARCHAR, KV2 VARCHAR)"
+                    + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
+
+            String dml = "UPSERT INTO " + tableName + " (PK, KV1, KV2) VALUES (?, ?, ?)";
+            PreparedStatement stmt = conn.prepareStatement(dml);
+
+            int count = 5;
+            for (int id = 0; id < count; ++id) {
+                stmt.setInt(1, id);
+                stmt.setString(2, "KV1_" + id);
+                stmt.setString(3, "KV2_" + id);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            String dql = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName;
+
+            ResultSet rs = conn.createStatement().executeQuery(dql);
+            // verify row timestamp returned by the query matches the empty column cell timestamp
+            verifyHbaseAllRowsTimestamp(tableName, rs, count);
+
+            // update one row
+            conn.createStatement().execute("UPSERT INTO " + tableName
+                    + " (PK, KV1) VALUES (2, 'KV1_foo')");
+            conn.commit();
+
+            rs = conn.createStatement().executeQuery(dql);
+            // verify again after update
+            verifyHbaseAllRowsTimestamp(tableName, rs, count);
+
+            dql = "SELECT ROWKEY_BYTES_STRING(), PHOENIX_ROW_TIMESTAMP() FROM " + tableName
+                    + " WHERE PK >= 1 AND PK <=3 ";
+            rs = conn.createStatement().executeQuery(dql);
+
+            while (rs.next()) {
+                verifyHbaseRowTimestamp(tableName, rs.getString(1), rs.getDate(2));
+            }
+        }
+    }
+
+    @Test
+    public void testRowTimestampColumn() throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR"
+                    + " CONSTRAINT PK PRIMARY KEY(PK1, PK2 ROW_TIMESTAMP))" + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
 
 Review comment:
   Same here. Same comment throughout for all other cases where we don't close the opened statement

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] jpisaac commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
jpisaac commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r401292042
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/parse/PhoenixRowTimestampParseNode.java
 ##########
 @@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.phoenix.parse;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.KeyValueColumnExpression;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.expression.function.PhoenixRowTimestampFunction;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PDatum;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.util.IndexUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+
+public class PhoenixRowTimestampParseNode extends FunctionParseNode {
+
+    PhoenixRowTimestampParseNode(String name, List<ParseNode> children,
+                                 BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+
+        // PhoenixRowTimestampFunction does not take any parameters.
+        assert children.size() == 0;
+
+        // Get the empty column family and qualifier for the context.
+        PTable table = context.getCurrentTable().getTable();
+        byte[] emptyColumnFamilyName = SchemaUtil.getEmptyColumnFamily(table);
+        byte[] emptyColumnName = table.getEncodingScheme() == PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS ?
+                QueryConstants.EMPTY_COLUMN_BYTES :
+                table.getEncodingScheme().encode(QueryConstants.ENCODED_EMPTY_COLUMN_NAME);
+
+        // Create an empty column key value expression.
+        // This will cause the empty column key value to evaluated during scan filter processing.
+        List<Expression> emptyColumnExpression = Arrays.asList(new Expression[] {new KeyValueColumnExpression(new PDatum() {
+            @Override
+            public boolean isNullable() {
+                return true;
+            }
+            @Override
+            public PDataType getDataType() {
+                return PDate.INSTANCE;
+            }
+            @Override
+            public Integer getMaxLength() {
+                return null;
+            }
+            @Override
+            public Integer getScale() {
+                return null;
+            }
+            @Override
+            public SortOrder getSortOrder() {
+                return SortOrder.getDefault();
+            }
+        }, emptyColumnFamilyName, emptyColumnName)});
+
+        // Add the empty column to the projection list.
+        // According to PHOENIX-4179 this will then return the timestamp of the empty column cell.
+        IndexUtil.addEmptyColumnToScan(context.getScan(), emptyColumnFamilyName, emptyColumnName);
 
 Review comment:
   addEmptyColumnToScan should handle all cases, But this will probably need refactoring to fix the test failure that we are seeing in the IT tests.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] jpisaac commented on issue #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
jpisaac commented on issue #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#issuecomment-595009300
 
 
   @ChinmaySKulkarni  @abhishek-chouhan  @gjacoby126  @tkhurana  Please review.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] yanxinyi commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
yanxinyi commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r389224762
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/parse/PhoenixRowTimestampParseNode.java
 ##########
 @@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.phoenix.parse;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.KeyValueColumnExpression;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.expression.function.PhoenixRowTimestampFunction;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PDatum;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.util.IndexUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+
+public class PhoenixRowTimestampParseNode extends FunctionParseNode {
+
+    PhoenixRowTimestampParseNode(String name, List<ParseNode> children,
+                                 BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+
+        // PhoenixRowTimestampFunction does not take any parameters.
+        assert children.size() == 0;
+
+        // Get the empty column family and qualifier for the context.
+        PTable table = context.getCurrentTable().getTable();
+        byte[] emptyColumnFamilyName = SchemaUtil.getEmptyColumnFamily(table);
+        byte[] emptyColumnName = table.getEncodingScheme() == PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS ?
+                QueryConstants.EMPTY_COLUMN_BYTES :
+                table.getEncodingScheme().encode(QueryConstants.ENCODED_EMPTY_COLUMN_NAME);
+
+        // Create an empty column key value expression.
+        // This will cause the empty column key value to evaluated during scan filter processing.
+        List<Expression> emptyColumnExpression = Arrays.asList(new Expression[] {new KeyValueColumnExpression(new PDatum() {
+            @Override
+            public boolean isNullable() {
+                return true;
+            }
+            @Override
+            public PDataType getDataType() {
+                return PDate.INSTANCE;
+            }
+            @Override
+            public Integer getMaxLength() {
+                return null;
+            }
+            @Override
+            public Integer getScale() {
+                return null;
+            }
+            @Override
+            public SortOrder getSortOrder() {
+                return SortOrder.getDefault();
+            }
+        }, emptyColumnFamilyName, emptyColumnName)});
+
+        // Add the empty column to the projection list.
+        // According to PHOENIX-4179 this will then return the timestamp of the empty column cell.
+        IndexUtil.addEmptyColumnToScan(context.getScan(), emptyColumnFamilyName, emptyColumnName);
 
 Review comment:
   does IndexUtil handle the non-index table here?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r406605845
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRowTimestampFunctionIT.java
 ##########
 @@ -0,0 +1,398 @@
+/*
+ * 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 com.google.common.collect.Lists;
+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.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+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.PTable.ImmutableStorageScheme;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
+import org.apache.phoenix.util.EncodedColumnsUtil;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.Collection;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PhoenixRowTimestampFunctionIT extends ParallelStatsDisabledIT {
+    private final boolean encoded;
+    private final String tableDDLOptions;
+    private static final int NUM_ROWS = 5;
+    private static final long TS_OFFSET = 120000;
+
+    public PhoenixRowTimestampFunctionIT(QualifierEncodingScheme encoding,
+            ImmutableStorageScheme storage) {
+        StringBuilder optionBuilder = new StringBuilder();
+        optionBuilder.append(" COLUMN_ENCODED_BYTES = " + encoding.ordinal());
+        optionBuilder.append(",IMMUTABLE_STORAGE_SCHEME = "+ storage.toString());
+        this.tableDDLOptions = optionBuilder.toString();
+        this.encoded = (encoding != QualifierEncodingScheme.NON_ENCODED_QUALIFIERS) ? true : false;
+    }
+
+    @Parameterized.Parameters(name = "encoding={0},storage={1}")
+    public static synchronized Collection<Object[]> data() {
+        List<Object[]> list = Lists.newArrayList();
+        for (QualifierEncodingScheme encoding : QualifierEncodingScheme.values()) {
+            for (ImmutableStorageScheme storage : ImmutableStorageScheme.values()) {
+                list.add(new Object[]{encoding, storage});
+            }
+        }
+        return list;
+    }
+
+    private void verifyHbaseAllRowsTimestamp(String tableName, ResultSet rs, int expectedRowCount)
+            throws Exception {
+
+        Scan scan = new Scan();
+        byte[] emptyKVQualifier = EncodedColumnsUtil.getEmptyKeyValueInfo(this.encoded).getFirst();
+        try (org.apache.hadoop.hbase.client.Connection hconn =
+                ConnectionFactory.createConnection(config)) {
+            Table table = hconn.getTable(TableName.valueOf(tableName));
+            ResultScanner resultScanner = table.getScanner(scan);
+            int rowCount = 0;
+            while (rs.next()) {
+                Result result = resultScanner.next();
+                long timeStamp = result.getColumnLatestCell(
+                        QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES,
+                        emptyKVQualifier).getTimestamp();
+                assertEquals(rs.getDate(1).getTime(), timeStamp);
+                ++rowCount;
+            }
+            assertEquals(expectedRowCount, rowCount);
+        }
+    }
+
+    private void verifyHbaseRowTimestamp(String tableName, String rowKey, Date expectedTimestamp)
+            throws Exception {
+
+        byte[] emptyKVQualifier = EncodedColumnsUtil.getEmptyKeyValueInfo(this.encoded).getFirst();
+        try (org.apache.hadoop.hbase.client.Connection hconn =
+                ConnectionFactory.createConnection(config)) {
+            Table table = hconn.getTable(TableName.valueOf(tableName));
+            Get get = new Get(Bytes.toBytesBinary(rowKey));
+            Result result = table.get(get);
+            assertFalse(result.isEmpty());
+            long timeStamp = result.getColumnLatestCell(
+                    QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, emptyKVQualifier).getTimestamp();
+            assertEquals(expectedTimestamp.getTime(), timeStamp);
+        }
+    }
+
+    private String createTestData(long rowTimestamp, int numRows) throws Exception {
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            // Create a test table.
+            try (Statement stmt = conn.createStatement()) {
+                String ddl = "CREATE TABLE IF NOT EXISTS " + tableName +
+                        " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR" +
+                        " CONSTRAINT PK PRIMARY KEY(PK1, PK2))" + this.tableDDLOptions;
+                stmt.execute(ddl);
+            }
+
+            // Upsert data into the test table.
+            String dml = "UPSERT INTO " + tableName + " (PK1, PK2, KV1, KV2) VALUES (?, ?, ?, ?)";
+            try (PreparedStatement stmt = conn.prepareStatement(dml)) {
+                Date rowTimestampDate = new Date(rowTimestamp);
+                int count = numRows;
+                for (int id = 0; id < count; ++id) {
+                    int idValue = id;
+                    stmt.setInt(1, idValue);
+                    stmt.setDate(2, rowTimestampDate);
+                    stmt.setString(3, "KV1_" + idValue);
+                    stmt.setString(4, "KV2_" + idValue);
+                    stmt.executeUpdate();
+                }
+            }
+            conn.commit();
+        }
+        return tableName;
+    }
+
+    @Test
+    public void testRowTimestampDefault() throws Exception {
+
+        String tableName =  generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE IF NOT EXISTS " + tableName
+                    + " (PK INTEGER NOT NULL PRIMARY KEY, KV1 VARCHAR, KV2 VARCHAR)"
+                    + this.tableDDLOptions;
+            conn.createStatement().execute(ddl);
+
+            String dml = "UPSERT INTO " + tableName + " (PK, KV1, KV2) VALUES (?, ?, ?)";
+            PreparedStatement stmt = conn.prepareStatement(dml);
+
+            int count = 5;
+            for (int id = 0; id < count; ++id) {
+                stmt.setInt(1, id);
+                stmt.setString(2, "KV1_" + id);
+                stmt.setString(3, "KV2_" + id);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+
+            String dql = "SELECT PHOENIX_ROW_TIMESTAMP() FROM " + tableName;
+
+            ResultSet rs = conn.createStatement().executeQuery(dql);
+            // verify row timestamp returned by the query matches the empty column cell timestamp
+            verifyHbaseAllRowsTimestamp(tableName, rs, count);
+
+            // update one row
+            conn.createStatement().execute("UPSERT INTO " + tableName
+                    + " (PK, KV1) VALUES (2, 'KV1_foo')");
+            conn.commit();
+
+            rs = conn.createStatement().executeQuery(dql);
+            // verify again after update
+            verifyHbaseAllRowsTimestamp(tableName, rs, count);
+
+            dql = "SELECT ROWKEY_BYTES_STRING(), PHOENIX_ROW_TIMESTAMP() FROM " + tableName
+                    + " WHERE PK >= 1 AND PK <=3 ";
+            rs = conn.createStatement().executeQuery(dql);
 
 Review comment:
   ditto

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #724: PHOENIX-5629 Phoenix Function to Return HBase row timestamp
URL: https://github.com/apache/phoenix/pull/724#discussion_r392450505
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/expression/function/PhoenixRowTimestampFunction.java
 ##########
 @@ -0,0 +1,136 @@
+/*
+ * 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 org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.phoenix.expression.Determinism;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.parse.PhoenixRowTimestampParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Date;
+import java.util.List;
+
+/**
+ * Function to return the timestamp of the empty column which functions as the row timestamp. The
+ * result returned can be used for debugging(eg. using HBase shell), logging etc.
+ * Can also be used in sql predicates.
+ */
+@BuiltInFunction(name = PhoenixRowTimestampFunction.NAME,
+        nodeClass= PhoenixRowTimestampParseNode.class,
+        args = {})
+public class PhoenixRowTimestampFunction extends ScalarFunction {
+    private static final Logger LOGGER = LoggerFactory.getLogger(PhoenixRowTimestampFunction.class);
 
 Review comment:
   nit: Remove this logger since it's not used. Also remove corresponding imports.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services