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/17 18:10:45 UTC

[GitHub] [phoenix] gjacoby126 opened a new pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

gjacoby126 opened a new pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735
 
 
   …kBack age

----------------------------------------------------------------
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] priyankporwal commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
priyankporwal commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r398340808
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyTableOutput.java
 ##########
 @@ -68,14 +68,18 @@
             "    SOURCE_TS BIGINT,\n" +
             "    TARGET_TS BIGINT,\n" +
             "    HAS_TARGET_ROW BOOLEAN,\n" +
+            "    BEYOND_MAX_LOOKBACK BOOLEAN,\n" +
             "    CONSTRAINT PK PRIMARY KEY\n" +
             "    (\n" +
             "        " + SOURCE_TABLE_COL_NAME + ",\n" +
             "        " + TARGET_TABLE_COL_NAME + ",\n" +
             "        " + SCRUTINY_EXECUTE_TIME_COL_NAME + ",\n" + // time at which the scrutiny ran
             "        SOURCE_ROW_PK_HASH\n" + //  this hash makes the PK unique
             "    )\n" + // dynamic columns consisting of the source and target columns will follow
-            ")";
+            ")  COLUMN_ENCODED_BYTES = 0 "; //column encoding not supported with dyn columns (PHOENIX-5107)
 
 Review comment:
   Do we need to do anything to disable COLUMN_ENCODIG for clusters with this table already existing?

----------------------------------------------------------------
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] gokceni commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
gokceni commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r394572034
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexScrutinyWithMaxLookbackIT.java
 ##########
 @@ -0,0 +1,206 @@
+/*
+ * 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.Maps;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.regionserver.ScanInfoUtil;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyMapper;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyTool;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
+import org.apache.phoenix.util.ManualEnvironmentEdge;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.phoenix.mapreduce.index.IndexScrutinyMapperForTest.MAX_LOOKBACK;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.INVALID_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.BEYOND_MAX_LOOKBACK;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.VALID_ROW_COUNT;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class IndexScrutinyWithMaxLookbackIT extends IndexScrutinyToolBaseIT {
+
+    private static PreparedStatement upsertDataStmt;
+    private static String dataTableFullName;
+    private static String schema;
+    private static String dataTableName;
+    private static String indexTableName;
+    private static String viewName;
+    private static boolean isViewIndex;
+    private static ManualEnvironmentEdge testClock;
+    public static final String UPSERT_DATA = "UPSERT INTO %s VALUES (?, ?, ?)";
+
+    @BeforeClass
+    public static synchronized void doSetup() throws Exception {
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
+        props.put(QueryServices.GLOBAL_INDEX_ROW_AGE_THRESHOLD_TO_DELETE_MS_ATTRIB, Long.toString(0));
+        props.put(ScanInfoUtil.PHOENIX_MAX_LOOKBACK_AGE_CONF_KEY,
+            Integer.toString(MAX_LOOKBACK));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    }
+
+    @Test
+    public void testScrutinyOnRowsBeyondMaxLookBack() throws Exception {
+        schema = generateUniqueName();
+        dataTableName = generateUniqueName();
+        indexTableName = generateUniqueName();
+        dataTableFullName = SchemaUtil.getTableName(schema, dataTableName);
+        isViewIndex = false;
+        String dataTableDDL = "CREATE TABLE %s (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, "
+            + "ZIP INTEGER) COLUMN_ENCODED_BYTES=0, VERSIONS=1";
+        String indexTableDDL = "CREATE INDEX %s ON %s (NAME) INCLUDE (ZIP)";
+        testClock = new ManualEnvironmentEdge();
+
+        try (Connection conn =
+                 DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            conn.createStatement().execute(String.format(dataTableDDL, dataTableFullName));
+            conn.createStatement().execute(String.format(indexTableDDL, indexTableName,
+                dataTableFullName));
+            conn.commit();
+        }
+        upsertDataAndScrutinize(dataTableName, dataTableFullName, testClock);
+    }
+
+    @Test
+    public void testScrutinyOnRowsBeyondMaxLookback_viewIndex() throws Exception {
+        schema = "S"+generateUniqueName();
+        dataTableName = "T"+generateUniqueName();
+        dataTableFullName = SchemaUtil.getTableName(schema,dataTableName);
+        indexTableName = "VI"+generateUniqueName();
+        isViewIndex = true;
+        viewName = "V"+generateUniqueName();
+        String viewFullName = SchemaUtil.getTableName(schema,viewName);
+        String dataTableDDL = "CREATE TABLE %s (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, "
+            + "ZIP INTEGER) COLUMN_ENCODED_BYTES = 0, VERSIONS = 1 ";
+        String viewDDL = "CREATE VIEW %s AS SELECT * FROM %s";
+        String indexTableDDL = "CREATE INDEX %s ON %s (NAME) INCLUDE (ZIP) VERSIONS = 1";
+        testClock = new ManualEnvironmentEdge();
+
+        try (Connection conn =
+                 DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            conn.createStatement().execute(String.format(dataTableDDL, dataTableFullName));
+            conn.createStatement().execute(String.format(viewDDL, viewFullName, dataTableFullName));
+            conn.createStatement().execute(String.format(indexTableDDL, indexTableName,
+                viewFullName));
+            conn.commit();
+        }
+        upsertDataAndScrutinize(viewName, viewFullName, testClock);
+    }
+
+    private void upsertDataAndScrutinize(String tableName, String tableFullName,
+                                         ManualEnvironmentEdge testClock)
+        throws Exception {
+        try(Connection conn =
+                DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            // insert two rows
+            upsertDataStmt = getUpsertDataStmt(tableFullName, conn);
+
+            NonParameterizedIndexScrutinyToolIT.upsertRow(upsertDataStmt, 1, "name-1", 98051);
+            NonParameterizedIndexScrutinyToolIT.upsertRow(upsertDataStmt, 2, "name-2", 98052);
+            conn.commit();
+            long afterInsertSCN = EnvironmentEdgeManager.currentTimeMillis() + 1;
+            testClock.setValue(afterInsertSCN);
+            EnvironmentEdgeManager.injectEdge(testClock);
+            testClock.incrementValue(1);
 
 Review comment:
   nit: Would be good to have a comment on what the clock increments are doing.

----------------------------------------------------------------
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] gokceni commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
gokceni commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r394572294
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexScrutinyWithMaxLookbackIT.java
 ##########
 @@ -0,0 +1,206 @@
+/*
+ * 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.Maps;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.regionserver.ScanInfoUtil;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyMapper;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyTool;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
+import org.apache.phoenix.util.ManualEnvironmentEdge;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.phoenix.mapreduce.index.IndexScrutinyMapperForTest.MAX_LOOKBACK;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.INVALID_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.BEYOND_MAX_LOOKBACK;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.VALID_ROW_COUNT;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class IndexScrutinyWithMaxLookbackIT extends IndexScrutinyToolBaseIT {
+
+    private static PreparedStatement upsertDataStmt;
+    private static String dataTableFullName;
+    private static String schema;
+    private static String dataTableName;
+    private static String indexTableName;
+    private static String viewName;
+    private static boolean isViewIndex;
+    private static ManualEnvironmentEdge testClock;
+    public static final String UPSERT_DATA = "UPSERT INTO %s VALUES (?, ?, ?)";
+
+    @BeforeClass
+    public static synchronized void doSetup() throws Exception {
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
+        props.put(QueryServices.GLOBAL_INDEX_ROW_AGE_THRESHOLD_TO_DELETE_MS_ATTRIB, Long.toString(0));
+        props.put(ScanInfoUtil.PHOENIX_MAX_LOOKBACK_AGE_CONF_KEY,
+            Integer.toString(MAX_LOOKBACK));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    }
+
+    @Test
+    public void testScrutinyOnRowsBeyondMaxLookBack() throws Exception {
+        schema = generateUniqueName();
+        dataTableName = generateUniqueName();
+        indexTableName = generateUniqueName();
+        dataTableFullName = SchemaUtil.getTableName(schema, dataTableName);
+        isViewIndex = false;
+        String dataTableDDL = "CREATE TABLE %s (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, "
+            + "ZIP INTEGER) COLUMN_ENCODED_BYTES=0, VERSIONS=1";
+        String indexTableDDL = "CREATE INDEX %s ON %s (NAME) INCLUDE (ZIP)";
+        testClock = new ManualEnvironmentEdge();
+
+        try (Connection conn =
+                 DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            conn.createStatement().execute(String.format(dataTableDDL, dataTableFullName));
+            conn.createStatement().execute(String.format(indexTableDDL, indexTableName,
+                dataTableFullName));
+            conn.commit();
+        }
+        upsertDataAndScrutinize(dataTableName, dataTableFullName, testClock);
+    }
+
+    @Test
+    public void testScrutinyOnRowsBeyondMaxLookback_viewIndex() throws Exception {
+        schema = "S"+generateUniqueName();
+        dataTableName = "T"+generateUniqueName();
+        dataTableFullName = SchemaUtil.getTableName(schema,dataTableName);
+        indexTableName = "VI"+generateUniqueName();
+        isViewIndex = true;
+        viewName = "V"+generateUniqueName();
+        String viewFullName = SchemaUtil.getTableName(schema,viewName);
+        String dataTableDDL = "CREATE TABLE %s (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, "
+            + "ZIP INTEGER) COLUMN_ENCODED_BYTES = 0, VERSIONS = 1 ";
+        String viewDDL = "CREATE VIEW %s AS SELECT * FROM %s";
+        String indexTableDDL = "CREATE INDEX %s ON %s (NAME) INCLUDE (ZIP) VERSIONS = 1";
+        testClock = new ManualEnvironmentEdge();
+
+        try (Connection conn =
+                 DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            conn.createStatement().execute(String.format(dataTableDDL, dataTableFullName));
+            conn.createStatement().execute(String.format(viewDDL, viewFullName, dataTableFullName));
+            conn.createStatement().execute(String.format(indexTableDDL, indexTableName,
+                viewFullName));
+            conn.commit();
+        }
+        upsertDataAndScrutinize(viewName, viewFullName, testClock);
+    }
+
+    private void upsertDataAndScrutinize(String tableName, String tableFullName,
+                                         ManualEnvironmentEdge testClock)
+        throws Exception {
+        try(Connection conn =
+                DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            // insert two rows
+            upsertDataStmt = getUpsertDataStmt(tableFullName, conn);
+
+            NonParameterizedIndexScrutinyToolIT.upsertRow(upsertDataStmt, 1, "name-1", 98051);
+            NonParameterizedIndexScrutinyToolIT.upsertRow(upsertDataStmt, 2, "name-2", 98052);
+            conn.commit();
+            long afterInsertSCN = EnvironmentEdgeManager.currentTimeMillis() + 1;
+            testClock.setValue(afterInsertSCN);
+            EnvironmentEdgeManager.injectEdge(testClock);
+            testClock.incrementValue(1);
+            testClock.incrementValue(MAX_LOOKBACK /2  * 1000);
+            long scrutinyTs = EnvironmentEdgeManager.currentTimeMillis();
+            changeIndexValue();
+            testClock.incrementValue(MAX_LOOKBACK /2  * 1000);
+            List<Job> completedJobs = runScrutiny(schema, tableName, indexTableName, scrutinyTs);
+            Job job = completedJobs.get(0);
+            assertTrue(job.isSuccessful());
+            assertCounters(job.getCounters());
+        }
+    }
+    private void changeIndexValue() throws SQLException {
 
 Review comment:
   Suggest to rename this function to be updateIndexRow

----------------------------------------------------------------
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] gjacoby126 commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
gjacoby126 commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r397990506
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexScrutinyWithMaxLookbackIT.java
 ##########
 @@ -0,0 +1,206 @@
+/*
+ * 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.Maps;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.regionserver.ScanInfoUtil;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyMapper;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyTool;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
+import org.apache.phoenix.util.ManualEnvironmentEdge;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.phoenix.mapreduce.index.IndexScrutinyMapperForTest.MAX_LOOKBACK;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.INVALID_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.BEYOND_MAX_LOOKBACK;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.VALID_ROW_COUNT;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class IndexScrutinyWithMaxLookbackIT extends IndexScrutinyToolBaseIT {
+
+    private static PreparedStatement upsertDataStmt;
+    private static String dataTableFullName;
+    private static String schema;
+    private static String dataTableName;
+    private static String indexTableName;
+    private static String viewName;
+    private static boolean isViewIndex;
+    private static ManualEnvironmentEdge testClock;
+    public static final String UPSERT_DATA = "UPSERT INTO %s VALUES (?, ?, ?)";
+
+    @BeforeClass
+    public static synchronized void doSetup() throws Exception {
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
+        props.put(QueryServices.GLOBAL_INDEX_ROW_AGE_THRESHOLD_TO_DELETE_MS_ATTRIB, Long.toString(0));
+        props.put(ScanInfoUtil.PHOENIX_MAX_LOOKBACK_AGE_CONF_KEY,
+            Integer.toString(MAX_LOOKBACK));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    }
+
+    @Test
+    public void testScrutinyOnRowsBeyondMaxLookBack() throws Exception {
+        schema = generateUniqueName();
+        dataTableName = generateUniqueName();
+        indexTableName = generateUniqueName();
+        dataTableFullName = SchemaUtil.getTableName(schema, dataTableName);
+        isViewIndex = false;
+        String dataTableDDL = "CREATE TABLE %s (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, "
+            + "ZIP INTEGER) COLUMN_ENCODED_BYTES=0, VERSIONS=1";
+        String indexTableDDL = "CREATE INDEX %s ON %s (NAME) INCLUDE (ZIP)";
+        testClock = new ManualEnvironmentEdge();
+
+        try (Connection conn =
+                 DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            conn.createStatement().execute(String.format(dataTableDDL, dataTableFullName));
+            conn.createStatement().execute(String.format(indexTableDDL, indexTableName,
+                dataTableFullName));
+            conn.commit();
+        }
+        upsertDataAndScrutinize(dataTableName, dataTableFullName, testClock);
+    }
+
+    @Test
+    public void testScrutinyOnRowsBeyondMaxLookback_viewIndex() throws Exception {
+        schema = "S"+generateUniqueName();
+        dataTableName = "T"+generateUniqueName();
+        dataTableFullName = SchemaUtil.getTableName(schema,dataTableName);
+        indexTableName = "VI"+generateUniqueName();
+        isViewIndex = true;
+        viewName = "V"+generateUniqueName();
+        String viewFullName = SchemaUtil.getTableName(schema,viewName);
+        String dataTableDDL = "CREATE TABLE %s (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, "
+            + "ZIP INTEGER) COLUMN_ENCODED_BYTES = 0, VERSIONS = 1 ";
+        String viewDDL = "CREATE VIEW %s AS SELECT * FROM %s";
+        String indexTableDDL = "CREATE INDEX %s ON %s (NAME) INCLUDE (ZIP) VERSIONS = 1";
+        testClock = new ManualEnvironmentEdge();
+
+        try (Connection conn =
+                 DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            conn.createStatement().execute(String.format(dataTableDDL, dataTableFullName));
+            conn.createStatement().execute(String.format(viewDDL, viewFullName, dataTableFullName));
+            conn.createStatement().execute(String.format(indexTableDDL, indexTableName,
+                viewFullName));
+            conn.commit();
+        }
+        upsertDataAndScrutinize(viewName, viewFullName, testClock);
+    }
+
+    private void upsertDataAndScrutinize(String tableName, String tableFullName,
+                                         ManualEnvironmentEdge testClock)
+        throws Exception {
+        try(Connection conn =
+                DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            // insert two rows
+            upsertDataStmt = getUpsertDataStmt(tableFullName, conn);
+
+            NonParameterizedIndexScrutinyToolIT.upsertRow(upsertDataStmt, 1, "name-1", 98051);
+            NonParameterizedIndexScrutinyToolIT.upsertRow(upsertDataStmt, 2, "name-2", 98052);
+            conn.commit();
+            long afterInsertSCN = EnvironmentEdgeManager.currentTimeMillis() + 1;
+            testClock.setValue(afterInsertSCN);
+            EnvironmentEdgeManager.injectEdge(testClock);
+            testClock.incrementValue(1);
+            testClock.incrementValue(MAX_LOOKBACK /2  * 1000);
+            long scrutinyTs = EnvironmentEdgeManager.currentTimeMillis();
+            changeIndexValue();
+            testClock.incrementValue(MAX_LOOKBACK /2  * 1000);
+            List<Job> completedJobs = runScrutiny(schema, tableName, indexTableName, scrutinyTs);
+            Job job = completedJobs.get(0);
+            assertTrue(job.isSuccessful());
+            assertCounters(job.getCounters());
+        }
+    }
+    private void changeIndexValue() throws SQLException {
+        try (Connection conn =
+                 DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            String tableName = isViewIndex ?
+                SchemaUtil.getTableName(schema, viewName) : dataTableFullName;
+            PreparedStatement stmt = getUpsertDataStmt(tableName, conn);
+            NonParameterizedIndexScrutinyToolIT.upsertRow(stmt, 1, "name-1", 38139);
+            conn.commit();
+        }
+    }
+
+    private static PreparedStatement getUpsertDataStmt(String tableFullName, Connection conn) throws SQLException {
+        return conn.prepareStatement(String.format(UPSERT_DATA, tableFullName));
+    }
+
+    private void assertCounters(Counters counters) {
+        assertEquals(1, getCounterValue(counters, VALID_ROW_COUNT));
+        assertEquals(1, getCounterValue(counters, BEYOND_MAX_LOOKBACK));
+        assertEquals(0, getCounterValue(counters, INVALID_ROW_COUNT));
+    }
+
+    private List<Job> runScrutiny(String schemaName, String dataTableName, String indexTableName,
+                                  Long scrutinyTs)
+        throws Exception {
+        return runScrutiny(schemaName, dataTableName, indexTableName, null, null, scrutinyTs);
+    }
+
+    private List<Job> runScrutiny(String schemaName, String dataTableName, String indexTableName,
+                                  Long batchSize, IndexScrutinyTool.SourceTable sourceTable,
+                                  Long scrutinyTs) throws Exception {
+        final String[]
+            cmdArgs =
+            getArgValues(schemaName, dataTableName, indexTableName, batchSize, sourceTable,
+                false, null, null, null, scrutinyTs);
+        return runScrutiny(MaxLookbackIndexScrutinyMapper.class, cmdArgs);
+    }
+
+    private static class MaxLookbackIndexScrutinyMapper extends IndexScrutinyMapper {
+        @Override
+        public void postSetup(){
+            try {
+                String tableToCompact;
+                if (isViewIndex){
+                    String physicalDataTableName =
+                        SchemaUtil.getPhysicalHBaseTableName(schema, dataTableName, false).getString();
 
 Review comment:
   I know it's going to be false. 

----------------------------------------------------------------
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 #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
swaroopak commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r394593712
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyMapper.java
 ##########
 @@ -288,8 +301,15 @@ protected void checkIfInvalidRowsExpired(Context context,
             Pair<Long, List<Object>> sourceValues = entry.getValue();
             Long sourceTS = sourceValues.getFirst();
             if (hasRowExpiredOnSource(sourceTS, ttl)) {
-                context.getCounter(PhoenixScrutinyJobCounters.EXPIRED_ROW_COUNT).increment(1);
-                itr.remove();
+                context.getCounter(PhoenixScrutinyJobCounters.EXPIRED_ROW_COUNT).increment(1L);
+                itr.remove(); //don't output to the scrutiny table
+            } else if (isRowOlderThanMaxLookback(sourceTS)){
+                context.getCounter(PhoenixScrutinyJobCounters.BEYOND_MAX_LOOKBACK).increment(1L);
+                //still output to the scrutiny table just in case it's useful
 
 Review comment:
   why do we need them around?

----------------------------------------------------------------
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] gjacoby126 commented on issue #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
gjacoby126 commented on issue #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#issuecomment-604587057
 
 
   Thanks for the reviews @swaroopak @priyankporwal @kadirozde 

----------------------------------------------------------------
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] kadirozde commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
kadirozde commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r395236000
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexScrutinyWithMaxLookbackIT.java
 ##########
 @@ -0,0 +1,209 @@
+/*
+ * 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.Maps;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.regionserver.ScanInfoUtil;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyMapper;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyTool;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
+import org.apache.phoenix.util.ManualEnvironmentEdge;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.INVALID_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.BEYOND_MAX_LOOKBACK;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.VALID_ROW_COUNT;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class IndexScrutinyWithMaxLookbackIT extends IndexScrutinyToolBaseIT {
+
+    private static PreparedStatement upsertDataStmt;
+    private static String dataTableFullName;
+    private static String schema;
+    private static String dataTableName;
+    private static String indexTableName;
+    private static String viewName;
+    private static boolean isViewIndex;
+    private static ManualEnvironmentEdge testClock;
+    public static final String UPSERT_DATA = "UPSERT INTO %s VALUES (?, ?, ?)";
+    public static final int MAX_LOOKBACK = 6;
+
+
+    @BeforeClass
+    public static synchronized void doSetup() throws Exception {
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(2);
+        props.put(QueryServices.GLOBAL_INDEX_ROW_AGE_THRESHOLD_TO_DELETE_MS_ATTRIB, Long.toString(0));
+        props.put(ScanInfoUtil.PHOENIX_MAX_LOOKBACK_AGE_CONF_KEY,
+            Integer.toString(MAX_LOOKBACK));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    }
+
+    @Test
+    public void testScrutinyOnRowsBeyondMaxLookBack() throws Exception {
+        schema = "S" + generateUniqueName();
+        dataTableName = "T" + generateUniqueName();
+        indexTableName = "I" + generateUniqueName();
+        dataTableFullName = SchemaUtil.getTableName(schema, dataTableName);
+        isViewIndex = false;
+        String dataTableDDL = "CREATE TABLE %s (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, "
+            + "ZIP INTEGER) COLUMN_ENCODED_BYTES=0, VERSIONS=1";
+        String indexTableDDL = "CREATE INDEX %s ON %s (NAME) INCLUDE (ZIP)";
+        testClock = new ManualEnvironmentEdge();
+
+        try (Connection conn =
+                 DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            conn.createStatement().execute(String.format(dataTableDDL, dataTableFullName));
+            conn.createStatement().execute(String.format(indexTableDDL, indexTableName,
+                dataTableFullName));
+            conn.commit();
+        }
+        upsertDataAndScrutinize(dataTableName, dataTableFullName, testClock);
+    }
+
+    @Test
+    public void testScrutinyOnRowsBeyondMaxLookback_viewIndex() throws Exception {
+        schema = "S"+generateUniqueName();
+        dataTableName = "T"+generateUniqueName();
+        dataTableFullName = SchemaUtil.getTableName(schema,dataTableName);
+        indexTableName = "VI"+generateUniqueName();
+        isViewIndex = true;
+        viewName = "V"+generateUniqueName();
+        String viewFullName = SchemaUtil.getTableName(schema,viewName);
+        String dataTableDDL = "CREATE TABLE %s (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, "
+            + "ZIP INTEGER) COLUMN_ENCODED_BYTES = 0, VERSIONS = 1 ";
+        String viewDDL = "CREATE VIEW %s AS SELECT * FROM %s";
+        String indexTableDDL = "CREATE INDEX %s ON %s (NAME) INCLUDE (ZIP) VERSIONS = 1";
+        testClock = new ManualEnvironmentEdge();
+
+        try (Connection conn =
+                 DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            conn.createStatement().execute(String.format(dataTableDDL, dataTableFullName));
+            conn.createStatement().execute(String.format(viewDDL, viewFullName, dataTableFullName));
+            conn.createStatement().execute(String.format(indexTableDDL, indexTableName,
+                viewFullName));
+            conn.commit();
+        }
+        upsertDataAndScrutinize(viewName, viewFullName, testClock);
+    }
+
+    private void upsertDataAndScrutinize(String tableName, String tableFullName,
 
 Review comment:
   Can we have tests for deletes in addition to upserts? I am also interested in the cases where rows are deleted within the max loopback window and before and after the scn.

----------------------------------------------------------------
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] gokceni commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
gokceni commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r394611347
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyTool.java
 ##########
 @@ -505,6 +509,18 @@ public int run(String[] args) throws Exception {
         }
     }
 
+    private void validateTimestamp(Configuration configuration, long ts) {
+        long maxLookBackAge = ScanInfoUtil.getMaxLookbackInMillis(configuration);
+        if (maxLookBackAge != ScanInfoUtil.DEFAULT_PHOENIX_MAX_LOOKBACK_AGE * 1000L) {
 
 Review comment:
   Is there a chance that people coincidentally pick the same Max lookback age as the default one?

----------------------------------------------------------------
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] kadirozde commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
kadirozde commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r395805615
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/MaxLookbackIT.java
 ##########
 @@ -317,16 +300,16 @@ public void testRecentMaxVersionsNotCompactedAway() throws Exception {
             //after flush, check to make sure we can see all three versions at the appropriate times
             assertMultiVersionLookbacks(dataTableSelectSql, allValues, allSCNs);
             assertMultiVersionLookbacks(indexTableSelectSql, allValues, allSCNs);
-            majorCompact(dataTable, EnvironmentEdgeManager.currentTimeMillis());
-            majorCompact(indexTable, EnvironmentEdgeManager.currentTimeMillis());
+            majorCompact(dataTable);
+            majorCompact(indexTable);
             //after major compaction, check to make sure we can see all three versions
             // at the appropriate times
             assertMultiVersionLookbacks(dataTableSelectSql, allValues, allSCNs);
             assertMultiVersionLookbacks(indexTableSelectSql, allValues, allSCNs);
             injectEdge.incrementValue(MAX_LOOKBACK_AGE * 1000);
             long afterLookbackAgeSCN = EnvironmentEdgeManager.currentTimeMillis();
-            majorCompact(dataTable, afterLookbackAgeSCN);
-            majorCompact(indexTable, afterLookbackAgeSCN);
+            majorCompact(dataTable);
+            majorCompact(indexTable);
 
 Review comment:
   That is fair. Thanks.

----------------------------------------------------------------
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 #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
swaroopak commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r394594635
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyMapper.java
 ##########
 @@ -299,7 +319,16 @@ protected boolean hasRowExpiredOnSource(Long sourceTS, Long ttl) {
         return ttl != Integer.MAX_VALUE && sourceTS + ttl*1000 < currentTS;
     }
 
-    private long getTableTtl() throws SQLException, IOException {
+    protected boolean isRowOlderThanMaxLookback(Long sourceTS){
 
 Review comment:
   nit: space before '{' this and some other places. 

----------------------------------------------------------------
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] gokceni commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
gokceni commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r394570511
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexScrutinyWithMaxLookbackIT.java
 ##########
 @@ -0,0 +1,206 @@
+/*
+ * 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.Maps;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.regionserver.ScanInfoUtil;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyMapper;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyTool;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
+import org.apache.phoenix.util.ManualEnvironmentEdge;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.phoenix.mapreduce.index.IndexScrutinyMapperForTest.MAX_LOOKBACK;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.INVALID_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.BEYOND_MAX_LOOKBACK;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.VALID_ROW_COUNT;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class IndexScrutinyWithMaxLookbackIT extends IndexScrutinyToolBaseIT {
+
+    private static PreparedStatement upsertDataStmt;
+    private static String dataTableFullName;
+    private static String schema;
+    private static String dataTableName;
+    private static String indexTableName;
+    private static String viewName;
+    private static boolean isViewIndex;
+    private static ManualEnvironmentEdge testClock;
+    public static final String UPSERT_DATA = "UPSERT INTO %s VALUES (?, ?, ?)";
+
+    @BeforeClass
+    public static synchronized void doSetup() throws Exception {
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
+        props.put(QueryServices.GLOBAL_INDEX_ROW_AGE_THRESHOLD_TO_DELETE_MS_ATTRIB, Long.toString(0));
+        props.put(ScanInfoUtil.PHOENIX_MAX_LOOKBACK_AGE_CONF_KEY,
+            Integer.toString(MAX_LOOKBACK));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    }
+
+    @Test
+    public void testScrutinyOnRowsBeyondMaxLookBack() throws Exception {
+        schema = generateUniqueName();
+        dataTableName = generateUniqueName();
+        indexTableName = generateUniqueName();
 
 Review comment:
   nit: consider prepending "I_" and "D_" to names

----------------------------------------------------------------
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] gokceni commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
gokceni commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r394572294
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexScrutinyWithMaxLookbackIT.java
 ##########
 @@ -0,0 +1,206 @@
+/*
+ * 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.Maps;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.regionserver.ScanInfoUtil;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyMapper;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyTool;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
+import org.apache.phoenix.util.ManualEnvironmentEdge;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.phoenix.mapreduce.index.IndexScrutinyMapperForTest.MAX_LOOKBACK;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.INVALID_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.BEYOND_MAX_LOOKBACK;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.VALID_ROW_COUNT;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class IndexScrutinyWithMaxLookbackIT extends IndexScrutinyToolBaseIT {
+
+    private static PreparedStatement upsertDataStmt;
+    private static String dataTableFullName;
+    private static String schema;
+    private static String dataTableName;
+    private static String indexTableName;
+    private static String viewName;
+    private static boolean isViewIndex;
+    private static ManualEnvironmentEdge testClock;
+    public static final String UPSERT_DATA = "UPSERT INTO %s VALUES (?, ?, ?)";
+
+    @BeforeClass
+    public static synchronized void doSetup() throws Exception {
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
+        props.put(QueryServices.GLOBAL_INDEX_ROW_AGE_THRESHOLD_TO_DELETE_MS_ATTRIB, Long.toString(0));
+        props.put(ScanInfoUtil.PHOENIX_MAX_LOOKBACK_AGE_CONF_KEY,
+            Integer.toString(MAX_LOOKBACK));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    }
+
+    @Test
+    public void testScrutinyOnRowsBeyondMaxLookBack() throws Exception {
+        schema = generateUniqueName();
+        dataTableName = generateUniqueName();
+        indexTableName = generateUniqueName();
+        dataTableFullName = SchemaUtil.getTableName(schema, dataTableName);
+        isViewIndex = false;
+        String dataTableDDL = "CREATE TABLE %s (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, "
+            + "ZIP INTEGER) COLUMN_ENCODED_BYTES=0, VERSIONS=1";
+        String indexTableDDL = "CREATE INDEX %s ON %s (NAME) INCLUDE (ZIP)";
+        testClock = new ManualEnvironmentEdge();
+
+        try (Connection conn =
+                 DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            conn.createStatement().execute(String.format(dataTableDDL, dataTableFullName));
+            conn.createStatement().execute(String.format(indexTableDDL, indexTableName,
+                dataTableFullName));
+            conn.commit();
+        }
+        upsertDataAndScrutinize(dataTableName, dataTableFullName, testClock);
+    }
+
+    @Test
+    public void testScrutinyOnRowsBeyondMaxLookback_viewIndex() throws Exception {
+        schema = "S"+generateUniqueName();
+        dataTableName = "T"+generateUniqueName();
+        dataTableFullName = SchemaUtil.getTableName(schema,dataTableName);
+        indexTableName = "VI"+generateUniqueName();
+        isViewIndex = true;
+        viewName = "V"+generateUniqueName();
+        String viewFullName = SchemaUtil.getTableName(schema,viewName);
+        String dataTableDDL = "CREATE TABLE %s (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, "
+            + "ZIP INTEGER) COLUMN_ENCODED_BYTES = 0, VERSIONS = 1 ";
+        String viewDDL = "CREATE VIEW %s AS SELECT * FROM %s";
+        String indexTableDDL = "CREATE INDEX %s ON %s (NAME) INCLUDE (ZIP) VERSIONS = 1";
+        testClock = new ManualEnvironmentEdge();
+
+        try (Connection conn =
+                 DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            conn.createStatement().execute(String.format(dataTableDDL, dataTableFullName));
+            conn.createStatement().execute(String.format(viewDDL, viewFullName, dataTableFullName));
+            conn.createStatement().execute(String.format(indexTableDDL, indexTableName,
+                viewFullName));
+            conn.commit();
+        }
+        upsertDataAndScrutinize(viewName, viewFullName, testClock);
+    }
+
+    private void upsertDataAndScrutinize(String tableName, String tableFullName,
+                                         ManualEnvironmentEdge testClock)
+        throws Exception {
+        try(Connection conn =
+                DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            // insert two rows
+            upsertDataStmt = getUpsertDataStmt(tableFullName, conn);
+
+            NonParameterizedIndexScrutinyToolIT.upsertRow(upsertDataStmt, 1, "name-1", 98051);
+            NonParameterizedIndexScrutinyToolIT.upsertRow(upsertDataStmt, 2, "name-2", 98052);
+            conn.commit();
+            long afterInsertSCN = EnvironmentEdgeManager.currentTimeMillis() + 1;
+            testClock.setValue(afterInsertSCN);
+            EnvironmentEdgeManager.injectEdge(testClock);
+            testClock.incrementValue(1);
+            testClock.incrementValue(MAX_LOOKBACK /2  * 1000);
+            long scrutinyTs = EnvironmentEdgeManager.currentTimeMillis();
+            changeIndexValue();
+            testClock.incrementValue(MAX_LOOKBACK /2  * 1000);
+            List<Job> completedJobs = runScrutiny(schema, tableName, indexTableName, scrutinyTs);
+            Job job = completedJobs.get(0);
+            assertTrue(job.isSuccessful());
+            assertCounters(job.getCounters());
+        }
+    }
+    private void changeIndexValue() throws SQLException {
 
 Review comment:
   Suggest to rename this function to be add index row

----------------------------------------------------------------
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] gjacoby126 commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
gjacoby126 commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r395874364
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixScrutinyJobCounters.java
 ##########
 @@ -41,5 +41,11 @@
     /**
      * Number of batches processed
      */
-    BATCHES_PROCESSED_COUNT;
+    BATCHES_PROCESSED_COUNT,
+    /**
+     * Number of rows in source that became older than the max lookback age while scrutiny
+     * was comparing them with the target, and didn't match. We break these out separately because
+     * they could be due to extra versions being compacted, and are harmless.
+     */
+    BEYOND_MAX_LOOKBACK;
 
 Review comment:
   No objection to appending _COUNT to the existing enum value, but NO_LOOKBACK_ROW_COUNT seems more confusing to me. 

----------------------------------------------------------------
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] gokceni commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
gokceni commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r394609771
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyMapper.java
 ##########
 @@ -161,12 +168,22 @@ protected void setup(final Context context) throws IOException, InterruptedExcep
             LOGGER.info("Target table base query: " + targetTableQuery);
             md5 = MessageDigest.getInstance("MD5");
             ttl = getTableTtl();
+            maxLookbackAgeMillis = ScanInfoUtil.getMaxLookbackInMillis(configuration);
         } catch (SQLException | NoSuchAlgorithmException e) {
             tryClosingResourceSilently(this.outputUpsertStmt);
             tryClosingResourceSilently(this.connection);
             tryClosingResourceSilently(this.outputConn);
             throw new RuntimeException(e);
         }
+        postSetup();
+    }
+
+    protected void preSetup() {
 
 Review comment:
   Any reasons to have these empty methods?

----------------------------------------------------------------
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] gjacoby126 commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
gjacoby126 commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r395316724
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexScrutinyWithMaxLookbackIT.java
 ##########
 @@ -0,0 +1,209 @@
+/*
+ * 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.Maps;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.regionserver.ScanInfoUtil;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyMapper;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyTool;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
+import org.apache.phoenix.util.ManualEnvironmentEdge;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.INVALID_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.BEYOND_MAX_LOOKBACK;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.VALID_ROW_COUNT;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class IndexScrutinyWithMaxLookbackIT extends IndexScrutinyToolBaseIT {
+
+    private static PreparedStatement upsertDataStmt;
+    private static String dataTableFullName;
+    private static String schema;
+    private static String dataTableName;
+    private static String indexTableName;
+    private static String viewName;
+    private static boolean isViewIndex;
+    private static ManualEnvironmentEdge testClock;
+    public static final String UPSERT_DATA = "UPSERT INTO %s VALUES (?, ?, ?)";
+    public static final int MAX_LOOKBACK = 6;
+
+
+    @BeforeClass
+    public static synchronized void doSetup() throws Exception {
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(2);
+        props.put(QueryServices.GLOBAL_INDEX_ROW_AGE_THRESHOLD_TO_DELETE_MS_ATTRIB, Long.toString(0));
+        props.put(ScanInfoUtil.PHOENIX_MAX_LOOKBACK_AGE_CONF_KEY,
+            Integer.toString(MAX_LOOKBACK));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    }
+
+    @Test
+    public void testScrutinyOnRowsBeyondMaxLookBack() throws Exception {
+        schema = "S" + generateUniqueName();
+        dataTableName = "T" + generateUniqueName();
+        indexTableName = "I" + generateUniqueName();
+        dataTableFullName = SchemaUtil.getTableName(schema, dataTableName);
+        isViewIndex = false;
+        String dataTableDDL = "CREATE TABLE %s (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, "
+            + "ZIP INTEGER) COLUMN_ENCODED_BYTES=0, VERSIONS=1";
+        String indexTableDDL = "CREATE INDEX %s ON %s (NAME) INCLUDE (ZIP)";
+        testClock = new ManualEnvironmentEdge();
+
+        try (Connection conn =
+                 DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            conn.createStatement().execute(String.format(dataTableDDL, dataTableFullName));
+            conn.createStatement().execute(String.format(indexTableDDL, indexTableName,
+                dataTableFullName));
+            conn.commit();
+        }
+        upsertDataAndScrutinize(dataTableName, dataTableFullName, testClock);
+    }
+
+    @Test
+    public void testScrutinyOnRowsBeyondMaxLookback_viewIndex() throws Exception {
+        schema = "S"+generateUniqueName();
+        dataTableName = "T"+generateUniqueName();
+        dataTableFullName = SchemaUtil.getTableName(schema,dataTableName);
+        indexTableName = "VI"+generateUniqueName();
+        isViewIndex = true;
+        viewName = "V"+generateUniqueName();
+        String viewFullName = SchemaUtil.getTableName(schema,viewName);
+        String dataTableDDL = "CREATE TABLE %s (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, "
+            + "ZIP INTEGER) COLUMN_ENCODED_BYTES = 0, VERSIONS = 1 ";
+        String viewDDL = "CREATE VIEW %s AS SELECT * FROM %s";
+        String indexTableDDL = "CREATE INDEX %s ON %s (NAME) INCLUDE (ZIP) VERSIONS = 1";
+        testClock = new ManualEnvironmentEdge();
+
+        try (Connection conn =
+                 DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            conn.createStatement().execute(String.format(dataTableDDL, dataTableFullName));
+            conn.createStatement().execute(String.format(viewDDL, viewFullName, dataTableFullName));
+            conn.createStatement().execute(String.format(indexTableDDL, indexTableName,
+                viewFullName));
+            conn.commit();
+        }
+        upsertDataAndScrutinize(viewName, viewFullName, testClock);
+    }
+
+    private void upsertDataAndScrutinize(String tableName, String tableFullName,
 
 Review comment:
   Good idea, I will add a test for deletes. 

----------------------------------------------------------------
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 #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
swaroopak commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r396775278
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexScrutinyWithMaxLookbackIT.java
 ##########
 @@ -0,0 +1,206 @@
+/*
+ * 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.Maps;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.regionserver.ScanInfoUtil;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyMapper;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyTool;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
+import org.apache.phoenix.util.ManualEnvironmentEdge;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.phoenix.mapreduce.index.IndexScrutinyMapperForTest.MAX_LOOKBACK;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.INVALID_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.BEYOND_MAX_LOOKBACK;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.VALID_ROW_COUNT;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class IndexScrutinyWithMaxLookbackIT extends IndexScrutinyToolBaseIT {
+
+    private static PreparedStatement upsertDataStmt;
+    private static String dataTableFullName;
+    private static String schema;
+    private static String dataTableName;
+    private static String indexTableName;
+    private static String viewName;
+    private static boolean isViewIndex;
+    private static ManualEnvironmentEdge testClock;
+    public static final String UPSERT_DATA = "UPSERT INTO %s VALUES (?, ?, ?)";
+
+    @BeforeClass
+    public static synchronized void doSetup() throws Exception {
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
+        props.put(QueryServices.GLOBAL_INDEX_ROW_AGE_THRESHOLD_TO_DELETE_MS_ATTRIB, Long.toString(0));
+        props.put(ScanInfoUtil.PHOENIX_MAX_LOOKBACK_AGE_CONF_KEY,
+            Integer.toString(MAX_LOOKBACK));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    }
+
+    @Test
+    public void testScrutinyOnRowsBeyondMaxLookBack() throws Exception {
+        schema = generateUniqueName();
+        dataTableName = generateUniqueName();
+        indexTableName = generateUniqueName();
+        dataTableFullName = SchemaUtil.getTableName(schema, dataTableName);
+        isViewIndex = false;
+        String dataTableDDL = "CREATE TABLE %s (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, "
+            + "ZIP INTEGER) COLUMN_ENCODED_BYTES=0, VERSIONS=1";
+        String indexTableDDL = "CREATE INDEX %s ON %s (NAME) INCLUDE (ZIP)";
+        testClock = new ManualEnvironmentEdge();
+
+        try (Connection conn =
+                 DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            conn.createStatement().execute(String.format(dataTableDDL, dataTableFullName));
+            conn.createStatement().execute(String.format(indexTableDDL, indexTableName,
+                dataTableFullName));
+            conn.commit();
+        }
+        upsertDataAndScrutinize(dataTableName, dataTableFullName, testClock);
+    }
+
+    @Test
+    public void testScrutinyOnRowsBeyondMaxLookback_viewIndex() throws Exception {
+        schema = "S"+generateUniqueName();
+        dataTableName = "T"+generateUniqueName();
+        dataTableFullName = SchemaUtil.getTableName(schema,dataTableName);
+        indexTableName = "VI"+generateUniqueName();
+        isViewIndex = true;
+        viewName = "V"+generateUniqueName();
+        String viewFullName = SchemaUtil.getTableName(schema,viewName);
+        String dataTableDDL = "CREATE TABLE %s (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, "
+            + "ZIP INTEGER) COLUMN_ENCODED_BYTES = 0, VERSIONS = 1 ";
+        String viewDDL = "CREATE VIEW %s AS SELECT * FROM %s";
+        String indexTableDDL = "CREATE INDEX %s ON %s (NAME) INCLUDE (ZIP) VERSIONS = 1";
+        testClock = new ManualEnvironmentEdge();
+
+        try (Connection conn =
+                 DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            conn.createStatement().execute(String.format(dataTableDDL, dataTableFullName));
+            conn.createStatement().execute(String.format(viewDDL, viewFullName, dataTableFullName));
+            conn.createStatement().execute(String.format(indexTableDDL, indexTableName,
+                viewFullName));
+            conn.commit();
+        }
+        upsertDataAndScrutinize(viewName, viewFullName, testClock);
+    }
+
+    private void upsertDataAndScrutinize(String tableName, String tableFullName,
+                                         ManualEnvironmentEdge testClock)
+        throws Exception {
+        try(Connection conn =
+                DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            // insert two rows
+            upsertDataStmt = getUpsertDataStmt(tableFullName, conn);
+
+            NonParameterizedIndexScrutinyToolIT.upsertRow(upsertDataStmt, 1, "name-1", 98051);
+            NonParameterizedIndexScrutinyToolIT.upsertRow(upsertDataStmt, 2, "name-2", 98052);
+            conn.commit();
+            long afterInsertSCN = EnvironmentEdgeManager.currentTimeMillis() + 1;
+            testClock.setValue(afterInsertSCN);
+            EnvironmentEdgeManager.injectEdge(testClock);
+            testClock.incrementValue(1);
+            testClock.incrementValue(MAX_LOOKBACK /2  * 1000);
+            long scrutinyTs = EnvironmentEdgeManager.currentTimeMillis();
+            changeIndexValue();
+            testClock.incrementValue(MAX_LOOKBACK /2  * 1000);
+            List<Job> completedJobs = runScrutiny(schema, tableName, indexTableName, scrutinyTs);
+            Job job = completedJobs.get(0);
+            assertTrue(job.isSuccessful());
+            assertCounters(job.getCounters());
+        }
+    }
+    private void changeIndexValue() throws SQLException {
+        try (Connection conn =
+                 DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            String tableName = isViewIndex ?
+                SchemaUtil.getTableName(schema, viewName) : dataTableFullName;
+            PreparedStatement stmt = getUpsertDataStmt(tableName, conn);
+            NonParameterizedIndexScrutinyToolIT.upsertRow(stmt, 1, "name-1", 38139);
+            conn.commit();
+        }
+    }
+
+    private static PreparedStatement getUpsertDataStmt(String tableFullName, Connection conn) throws SQLException {
+        return conn.prepareStatement(String.format(UPSERT_DATA, tableFullName));
+    }
+
+    private void assertCounters(Counters counters) {
+        assertEquals(1, getCounterValue(counters, VALID_ROW_COUNT));
+        assertEquals(1, getCounterValue(counters, BEYOND_MAX_LOOKBACK));
+        assertEquals(0, getCounterValue(counters, INVALID_ROW_COUNT));
+    }
+
+    private List<Job> runScrutiny(String schemaName, String dataTableName, String indexTableName,
+                                  Long scrutinyTs)
+        throws Exception {
+        return runScrutiny(schemaName, dataTableName, indexTableName, null, null, scrutinyTs);
+    }
+
+    private List<Job> runScrutiny(String schemaName, String dataTableName, String indexTableName,
+                                  Long batchSize, IndexScrutinyTool.SourceTable sourceTable,
+                                  Long scrutinyTs) throws Exception {
+        final String[]
+            cmdArgs =
+            getArgValues(schemaName, dataTableName, indexTableName, batchSize, sourceTable,
+                false, null, null, null, scrutinyTs);
+        return runScrutiny(MaxLookbackIndexScrutinyMapper.class, cmdArgs);
+    }
+
+    private static class MaxLookbackIndexScrutinyMapper extends IndexScrutinyMapper {
+        @Override
+        public void postSetup(){
+            try {
+                String tableToCompact;
+                if (isViewIndex){
+                    String physicalDataTableName =
+                        SchemaUtil.getPhysicalHBaseTableName(schema, dataTableName, false).getString();
 
 Review comment:
   @gjacoby126 

----------------------------------------------------------------
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] priyankporwal commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
priyankporwal commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r395864325
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
 ##########
 @@ -1183,6 +1186,22 @@ public static int getIsNullableInt(boolean isNullable) {
 		return isNullable ? ResultSetMetaData.columnNullable : ResultSetMetaData.columnNoNulls;
 	}
 
+	public static int getTimeToLive(PhoenixConnection conn, String physicalName) throws SQLException {
+        byte[] tableQualifier = Bytes.toBytes(physicalName);
+        return getTimeToLive(conn, tableQualifier);
+    }
+
+    public static int getTimeToLive(PhoenixConnection conn, byte[] tableQualifier)
+     throws SQLException {
+        HTableDescriptor td = conn.getQueryServices().getTableDescriptor(tableQualifier);
+        HColumnDescriptor[] cds = td.getColumnFamilies();
+        if (cds.length > 0){
+            return cds[0].getTimeToLive();
+        } else {
+            return HConstants.FOREVER;
 
 Review comment:
    when would this be the case? i.e. #CF==0

----------------------------------------------------------------
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] priyankporwal commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
priyankporwal commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r395862690
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixScrutinyJobCounters.java
 ##########
 @@ -41,5 +41,11 @@
     /**
      * Number of batches processed
      */
-    BATCHES_PROCESSED_COUNT;
+    BATCHES_PROCESSED_COUNT,
+    /**
+     * Number of rows in source that became older than the max lookback age while scrutiny
+     * was comparing them with the target, and didn't match. We break these out separately because
+     * they could be due to extra versions being compacted, and are harmless.
+     */
+    BEYOND_MAX_LOOKBACK;
 
 Review comment:
   All counters end with '_COUNT' here ... Perhaps we should remain consistent. That said, 'BEYOND_MAX_LOOKBACK_ROW_COUNT' looks too long. How about 'NO_LOOKBACK_ROW_COUNT'?

----------------------------------------------------------------
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] gokceni commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
gokceni commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r394569926
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexScrutinyWithMaxLookbackIT.java
 ##########
 @@ -0,0 +1,206 @@
+/*
+ * 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.Maps;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.regionserver.ScanInfoUtil;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyMapper;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyTool;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
+import org.apache.phoenix.util.ManualEnvironmentEdge;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.phoenix.mapreduce.index.IndexScrutinyMapperForTest.MAX_LOOKBACK;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.INVALID_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.BEYOND_MAX_LOOKBACK;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.VALID_ROW_COUNT;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class IndexScrutinyWithMaxLookbackIT extends IndexScrutinyToolBaseIT {
+
+    private static PreparedStatement upsertDataStmt;
+    private static String dataTableFullName;
+    private static String schema;
+    private static String dataTableName;
+    private static String indexTableName;
+    private static String viewName;
+    private static boolean isViewIndex;
+    private static ManualEnvironmentEdge testClock;
+    public static final String UPSERT_DATA = "UPSERT INTO %s VALUES (?, ?, ?)";
+
+    @BeforeClass
+    public static synchronized void doSetup() throws Exception {
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
 
 Review comment:
   nit : consider expected size = 2

----------------------------------------------------------------
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 #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
swaroopak commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r394589194
 
 

 ##########
 File path: phoenix-core/src/test/java/org/apache/phoenix/mapreduce/index/IndexScrutinyMapperForTest.java
 ##########
 @@ -23,6 +23,8 @@
 public class IndexScrutinyMapperForTest extends IndexScrutinyMapper {
 
     public static final int TEST_TABLE_TTL = 3600;
+    public static final int MAX_LOOKBACK = 6;
 
 Review comment:
   nit: unnecessary change

----------------------------------------------------------------
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] gjacoby126 commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
gjacoby126 commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r394672881
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyTool.java
 ##########
 @@ -505,6 +509,18 @@ public int run(String[] args) throws Exception {
         }
     }
 
+    private void validateTimestamp(Configuration configuration, long ts) {
+        long maxLookBackAge = ScanInfoUtil.getMaxLookbackInMillis(configuration);
+        if (maxLookBackAge != ScanInfoUtil.DEFAULT_PHOENIX_MAX_LOOKBACK_AGE * 1000L) {
 
 Review comment:
   The default age is the same as "turn off max lookback". It happens to be 0, which makes the 1000L not necessary, but I wanted to keep the discipline of _always_ converting to millis from seconds to avoid accidentally missing it elsewhere.

----------------------------------------------------------------
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] gjacoby126 commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
gjacoby126 commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r395880099
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
 ##########
 @@ -1183,6 +1186,22 @@ public static int getIsNullableInt(boolean isNullable) {
 		return isNullable ? ResultSetMetaData.columnNullable : ResultSetMetaData.columnNoNulls;
 	}
 
+	public static int getTimeToLive(PhoenixConnection conn, String physicalName) throws SQLException {
+        byte[] tableQualifier = Bytes.toBytes(physicalName);
+        return getTimeToLive(conn, tableQualifier);
+    }
+
+    public static int getTimeToLive(PhoenixConnection conn, byte[] tableQualifier)
+     throws SQLException {
+        HTableDescriptor td = conn.getQueryServices().getTableDescriptor(tableQualifier);
+        HColumnDescriptor[] cds = td.getColumnFamilies();
+        if (cds.length > 0){
+            return cds[0].getTimeToLive();
+        } else {
+            return HConstants.FOREVER;
 
 Review comment:
   I couldn't think of one, but decided to code the function defensively. 

----------------------------------------------------------------
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] gjacoby126 commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
gjacoby126 commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r394674341
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyMapper.java
 ##########
 @@ -288,8 +301,15 @@ protected void checkIfInvalidRowsExpired(Context context,
             Pair<Long, List<Object>> sourceValues = entry.getValue();
             Long sourceTS = sourceValues.getFirst();
             if (hasRowExpiredOnSource(sourceTS, ttl)) {
-                context.getCounter(PhoenixScrutinyJobCounters.EXPIRED_ROW_COUNT).increment(1);
-                itr.remove();
+                context.getCounter(PhoenixScrutinyJobCounters.EXPIRED_ROW_COUNT).increment(1L);
+                itr.remove(); //don't output to the scrutiny table
+            } else if (isRowOlderThanMaxLookback(sourceTS)){
+                context.getCounter(PhoenixScrutinyJobCounters.BEYOND_MAX_LOOKBACK).increment(1L);
+                //still output to the scrutiny table just in case it's useful
 
 Review comment:
   Humans need to evaluate beyond max lookback counters to see if they're valid issues or not. 

----------------------------------------------------------------
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 #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
swaroopak commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r394601232
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexScrutinyWithMaxLookbackIT.java
 ##########
 @@ -0,0 +1,206 @@
+/*
+ * 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.Maps;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.regionserver.ScanInfoUtil;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyMapper;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyTool;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
+import org.apache.phoenix.util.ManualEnvironmentEdge;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.phoenix.mapreduce.index.IndexScrutinyMapperForTest.MAX_LOOKBACK;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.INVALID_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.BEYOND_MAX_LOOKBACK;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.VALID_ROW_COUNT;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class IndexScrutinyWithMaxLookbackIT extends IndexScrutinyToolBaseIT {
+
+    private static PreparedStatement upsertDataStmt;
+    private static String dataTableFullName;
+    private static String schema;
+    private static String dataTableName;
+    private static String indexTableName;
+    private static String viewName;
+    private static boolean isViewIndex;
+    private static ManualEnvironmentEdge testClock;
+    public static final String UPSERT_DATA = "UPSERT INTO %s VALUES (?, ?, ?)";
+
+    @BeforeClass
+    public static synchronized void doSetup() throws Exception {
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
+        props.put(QueryServices.GLOBAL_INDEX_ROW_AGE_THRESHOLD_TO_DELETE_MS_ATTRIB, Long.toString(0));
+        props.put(ScanInfoUtil.PHOENIX_MAX_LOOKBACK_AGE_CONF_KEY,
+            Integer.toString(MAX_LOOKBACK));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    }
+
+    @Test
+    public void testScrutinyOnRowsBeyondMaxLookBack() throws Exception {
+        schema = generateUniqueName();
+        dataTableName = generateUniqueName();
+        indexTableName = generateUniqueName();
+        dataTableFullName = SchemaUtil.getTableName(schema, dataTableName);
+        isViewIndex = false;
+        String dataTableDDL = "CREATE TABLE %s (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, "
+            + "ZIP INTEGER) COLUMN_ENCODED_BYTES=0, VERSIONS=1";
+        String indexTableDDL = "CREATE INDEX %s ON %s (NAME) INCLUDE (ZIP)";
+        testClock = new ManualEnvironmentEdge();
+
+        try (Connection conn =
+                 DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            conn.createStatement().execute(String.format(dataTableDDL, dataTableFullName));
+            conn.createStatement().execute(String.format(indexTableDDL, indexTableName,
+                dataTableFullName));
+            conn.commit();
+        }
+        upsertDataAndScrutinize(dataTableName, dataTableFullName, testClock);
+    }
+
+    @Test
+    public void testScrutinyOnRowsBeyondMaxLookback_viewIndex() throws Exception {
+        schema = "S"+generateUniqueName();
+        dataTableName = "T"+generateUniqueName();
+        dataTableFullName = SchemaUtil.getTableName(schema,dataTableName);
+        indexTableName = "VI"+generateUniqueName();
+        isViewIndex = true;
+        viewName = "V"+generateUniqueName();
+        String viewFullName = SchemaUtil.getTableName(schema,viewName);
+        String dataTableDDL = "CREATE TABLE %s (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, "
+            + "ZIP INTEGER) COLUMN_ENCODED_BYTES = 0, VERSIONS = 1 ";
+        String viewDDL = "CREATE VIEW %s AS SELECT * FROM %s";
+        String indexTableDDL = "CREATE INDEX %s ON %s (NAME) INCLUDE (ZIP) VERSIONS = 1";
+        testClock = new ManualEnvironmentEdge();
+
+        try (Connection conn =
+                 DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            conn.createStatement().execute(String.format(dataTableDDL, dataTableFullName));
+            conn.createStatement().execute(String.format(viewDDL, viewFullName, dataTableFullName));
+            conn.createStatement().execute(String.format(indexTableDDL, indexTableName,
+                viewFullName));
+            conn.commit();
+        }
+        upsertDataAndScrutinize(viewName, viewFullName, testClock);
+    }
+
+    private void upsertDataAndScrutinize(String tableName, String tableFullName,
+                                         ManualEnvironmentEdge testClock)
+        throws Exception {
+        try(Connection conn =
+                DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            // insert two rows
+            upsertDataStmt = getUpsertDataStmt(tableFullName, conn);
+
+            NonParameterizedIndexScrutinyToolIT.upsertRow(upsertDataStmt, 1, "name-1", 98051);
+            NonParameterizedIndexScrutinyToolIT.upsertRow(upsertDataStmt, 2, "name-2", 98052);
+            conn.commit();
+            long afterInsertSCN = EnvironmentEdgeManager.currentTimeMillis() + 1;
+            testClock.setValue(afterInsertSCN);
+            EnvironmentEdgeManager.injectEdge(testClock);
+            testClock.incrementValue(1);
+            testClock.incrementValue(MAX_LOOKBACK /2  * 1000);
+            long scrutinyTs = EnvironmentEdgeManager.currentTimeMillis();
+            changeIndexValue();
+            testClock.incrementValue(MAX_LOOKBACK /2  * 1000);
+            List<Job> completedJobs = runScrutiny(schema, tableName, indexTableName, scrutinyTs);
+            Job job = completedJobs.get(0);
+            assertTrue(job.isSuccessful());
+            assertCounters(job.getCounters());
+        }
+    }
+    private void changeIndexValue() throws SQLException {
+        try (Connection conn =
+                 DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            String tableName = isViewIndex ?
+                SchemaUtil.getTableName(schema, viewName) : dataTableFullName;
+            PreparedStatement stmt = getUpsertDataStmt(tableName, conn);
+            NonParameterizedIndexScrutinyToolIT.upsertRow(stmt, 1, "name-1", 38139);
+            conn.commit();
+        }
+    }
+
+    private static PreparedStatement getUpsertDataStmt(String tableFullName, Connection conn) throws SQLException {
+        return conn.prepareStatement(String.format(UPSERT_DATA, tableFullName));
+    }
+
+    private void assertCounters(Counters counters) {
+        assertEquals(1, getCounterValue(counters, VALID_ROW_COUNT));
+        assertEquals(1, getCounterValue(counters, BEYOND_MAX_LOOKBACK));
+        assertEquals(0, getCounterValue(counters, INVALID_ROW_COUNT));
+    }
+
+    private List<Job> runScrutiny(String schemaName, String dataTableName, String indexTableName,
+                                  Long scrutinyTs)
+        throws Exception {
+        return runScrutiny(schemaName, dataTableName, indexTableName, null, null, scrutinyTs);
+    }
+
+    private List<Job> runScrutiny(String schemaName, String dataTableName, String indexTableName,
+                                  Long batchSize, IndexScrutinyTool.SourceTable sourceTable,
+                                  Long scrutinyTs) throws Exception {
+        final String[]
+            cmdArgs =
+            getArgValues(schemaName, dataTableName, indexTableName, batchSize, sourceTable,
+                false, null, null, null, scrutinyTs);
+        return runScrutiny(MaxLookbackIndexScrutinyMapper.class, cmdArgs);
+    }
+
+    private static class MaxLookbackIndexScrutinyMapper extends IndexScrutinyMapper {
+        @Override
+        public void postSetup(){
+            try {
+                String tableToCompact;
+                if (isViewIndex){
+                    String physicalDataTableName =
+                        SchemaUtil.getPhysicalHBaseTableName(schema, dataTableName, false).getString();
 
 Review comment:
   why not take isNamespaceEnabled property from the config?

----------------------------------------------------------------
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] priyankporwal commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
priyankporwal commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r395906713
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyMapper.java
 ##########
 @@ -288,8 +301,15 @@ protected void checkIfInvalidRowsExpired(Context context,
             Pair<Long, List<Object>> sourceValues = entry.getValue();
             Long sourceTS = sourceValues.getFirst();
             if (hasRowExpiredOnSource(sourceTS, ttl)) {
-                context.getCounter(PhoenixScrutinyJobCounters.EXPIRED_ROW_COUNT).increment(1);
-                itr.remove();
+                context.getCounter(PhoenixScrutinyJobCounters.EXPIRED_ROW_COUNT).increment(1L);
+                itr.remove(); //don't output to the scrutiny table
+            } else if (isRowOlderThanMaxLookback(sourceTS)){
+                context.getCounter(PhoenixScrutinyJobCounters.BEYOND_MAX_LOOKBACK).increment(1L);
+                //still output to the scrutiny table just in case it's useful
 
 Review comment:
   @gjacoby126 Any reason to be concerned about bloating the scrutiny table? This is an expected race condition that can happen pretty regularly if rows are updated at the cadence of around MaxLookbackAge. Humans may not necessarily have bandwidth to look at potentially large number of rows. If we are going to emit to scrutiny table, perhaps we should add a way to filter these out as well to make them manageable.

----------------------------------------------------------------
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] gjacoby126 commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
gjacoby126 commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r395304627
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/MaxLookbackIT.java
 ##########
 @@ -317,16 +300,16 @@ public void testRecentMaxVersionsNotCompactedAway() throws Exception {
             //after flush, check to make sure we can see all three versions at the appropriate times
             assertMultiVersionLookbacks(dataTableSelectSql, allValues, allSCNs);
             assertMultiVersionLookbacks(indexTableSelectSql, allValues, allSCNs);
-            majorCompact(dataTable, EnvironmentEdgeManager.currentTimeMillis());
-            majorCompact(indexTable, EnvironmentEdgeManager.currentTimeMillis());
+            majorCompact(dataTable);
+            majorCompact(indexTable);
             //after major compaction, check to make sure we can see all three versions
             // at the appropriate times
             assertMultiVersionLookbacks(dataTableSelectSql, allValues, allSCNs);
             assertMultiVersionLookbacks(indexTableSelectSql, allValues, allSCNs);
             injectEdge.incrementValue(MAX_LOOKBACK_AGE * 1000);
             long afterLookbackAgeSCN = EnvironmentEdgeManager.currentTimeMillis();
-            majorCompact(dataTable, afterLookbackAgeSCN);
-            majorCompact(indexTable, afterLookbackAgeSCN);
+            majorCompact(dataTable);
+            majorCompact(indexTable);
 
 Review comment:
   But if there are additional indexing tests we need, definitely please suggest them in a JIRA!

----------------------------------------------------------------
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] priyankporwal commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
priyankporwal commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r395904364
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixScrutinyJobCounters.java
 ##########
 @@ -41,5 +41,11 @@
     /**
      * Number of batches processed
      */
-    BATCHES_PROCESSED_COUNT;
+    BATCHES_PROCESSED_COUNT,
+    /**
+     * Number of rows in source that became older than the max lookback age while scrutiny
+     * was comparing them with the target, and didn't match. We break these out separately because
+     * they could be due to extra versions being compacted, and are harmless.
+     */
+    BEYOND_MAX_LOOKBACK;
 
 Review comment:
   True. They are similar to 'EXPIRED' but can't use the same name... 'UNREADABLE' can also be confusing. Anyways, pick what sounds best to you.

----------------------------------------------------------------
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] gjacoby126 commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
gjacoby126 commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r395303792
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/MaxLookbackIT.java
 ##########
 @@ -317,16 +300,16 @@ public void testRecentMaxVersionsNotCompactedAway() throws Exception {
             //after flush, check to make sure we can see all three versions at the appropriate times
             assertMultiVersionLookbacks(dataTableSelectSql, allValues, allSCNs);
             assertMultiVersionLookbacks(indexTableSelectSql, allValues, allSCNs);
-            majorCompact(dataTable, EnvironmentEdgeManager.currentTimeMillis());
-            majorCompact(indexTable, EnvironmentEdgeManager.currentTimeMillis());
+            majorCompact(dataTable);
+            majorCompact(indexTable);
             //after major compaction, check to make sure we can see all three versions
             // at the appropriate times
             assertMultiVersionLookbacks(dataTableSelectSql, allValues, allSCNs);
             assertMultiVersionLookbacks(indexTableSelectSql, allValues, allSCNs);
             injectEdge.incrementValue(MAX_LOOKBACK_AGE * 1000);
             long afterLookbackAgeSCN = EnvironmentEdgeManager.currentTimeMillis();
-            majorCompact(dataTable, afterLookbackAgeSCN);
-            majorCompact(indexTable, afterLookbackAgeSCN);
+            majorCompact(dataTable);
+            majorCompact(indexTable);
 
 Review comment:
   First, this test suite only was modified because I needed to steal its major compaction code and put it into TestUtil. It's otherwise unrelated to this JIRA, and if it needs changes, that should be a separate issue. 
   
   Second, this isn't really a test of indexing per se, but of whether the max lookback coproc hooks correctly prevent data from being purged during the lookback window. 

----------------------------------------------------------------
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] gjacoby126 merged pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
gjacoby126 merged pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735
 
 
   

----------------------------------------------------------------
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] priyankporwal commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
priyankporwal commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r398340808
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyTableOutput.java
 ##########
 @@ -68,14 +68,18 @@
             "    SOURCE_TS BIGINT,\n" +
             "    TARGET_TS BIGINT,\n" +
             "    HAS_TARGET_ROW BOOLEAN,\n" +
+            "    BEYOND_MAX_LOOKBACK BOOLEAN,\n" +
             "    CONSTRAINT PK PRIMARY KEY\n" +
             "    (\n" +
             "        " + SOURCE_TABLE_COL_NAME + ",\n" +
             "        " + TARGET_TABLE_COL_NAME + ",\n" +
             "        " + SCRUTINY_EXECUTE_TIME_COL_NAME + ",\n" + // time at which the scrutiny ran
             "        SOURCE_ROW_PK_HASH\n" + //  this hash makes the PK unique
             "    )\n" + // dynamic columns consisting of the source and target columns will follow
-            ")";
+            ")  COLUMN_ENCODED_BYTES = 0 "; //column encoding not supported with dyn columns (PHOENIX-5107)
 
 Review comment:
   Do we need to do anything to disable COLUMN_ENCODING for clusters with this table already existing?

----------------------------------------------------------------
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] kadirozde commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
kadirozde commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r395238910
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/MaxLookbackIT.java
 ##########
 @@ -317,16 +300,16 @@ public void testRecentMaxVersionsNotCompactedAway() throws Exception {
             //after flush, check to make sure we can see all three versions at the appropriate times
             assertMultiVersionLookbacks(dataTableSelectSql, allValues, allSCNs);
             assertMultiVersionLookbacks(indexTableSelectSql, allValues, allSCNs);
-            majorCompact(dataTable, EnvironmentEdgeManager.currentTimeMillis());
-            majorCompact(indexTable, EnvironmentEdgeManager.currentTimeMillis());
+            majorCompact(dataTable);
+            majorCompact(indexTable);
             //after major compaction, check to make sure we can see all three versions
             // at the appropriate times
             assertMultiVersionLookbacks(dataTableSelectSql, allValues, allSCNs);
             assertMultiVersionLookbacks(indexTableSelectSql, allValues, allSCNs);
             injectEdge.incrementValue(MAX_LOOKBACK_AGE * 1000);
             long afterLookbackAgeSCN = EnvironmentEdgeManager.currentTimeMillis();
-            majorCompact(dataTable, afterLookbackAgeSCN);
-            majorCompact(indexTable, afterLookbackAgeSCN);
+            majorCompact(dataTable);
+            majorCompact(indexTable);
 
 Review comment:
   In the tests above, major compactions run on both the data and index table. Can we pick only one table for one test, the other for another test, and so on? The interesting problems arise when compaction runs on one table but not on both

----------------------------------------------------------------
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] gokceni commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
gokceni commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r394572294
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexScrutinyWithMaxLookbackIT.java
 ##########
 @@ -0,0 +1,206 @@
+/*
+ * 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.Maps;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.regionserver.ScanInfoUtil;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyMapper;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyTool;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
+import org.apache.phoenix.util.ManualEnvironmentEdge;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.phoenix.mapreduce.index.IndexScrutinyMapperForTest.MAX_LOOKBACK;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.INVALID_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.BEYOND_MAX_LOOKBACK;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.VALID_ROW_COUNT;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class IndexScrutinyWithMaxLookbackIT extends IndexScrutinyToolBaseIT {
+
+    private static PreparedStatement upsertDataStmt;
+    private static String dataTableFullName;
+    private static String schema;
+    private static String dataTableName;
+    private static String indexTableName;
+    private static String viewName;
+    private static boolean isViewIndex;
+    private static ManualEnvironmentEdge testClock;
+    public static final String UPSERT_DATA = "UPSERT INTO %s VALUES (?, ?, ?)";
+
+    @BeforeClass
+    public static synchronized void doSetup() throws Exception {
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
+        props.put(QueryServices.GLOBAL_INDEX_ROW_AGE_THRESHOLD_TO_DELETE_MS_ATTRIB, Long.toString(0));
+        props.put(ScanInfoUtil.PHOENIX_MAX_LOOKBACK_AGE_CONF_KEY,
+            Integer.toString(MAX_LOOKBACK));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    }
+
+    @Test
+    public void testScrutinyOnRowsBeyondMaxLookBack() throws Exception {
+        schema = generateUniqueName();
+        dataTableName = generateUniqueName();
+        indexTableName = generateUniqueName();
+        dataTableFullName = SchemaUtil.getTableName(schema, dataTableName);
+        isViewIndex = false;
+        String dataTableDDL = "CREATE TABLE %s (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, "
+            + "ZIP INTEGER) COLUMN_ENCODED_BYTES=0, VERSIONS=1";
+        String indexTableDDL = "CREATE INDEX %s ON %s (NAME) INCLUDE (ZIP)";
+        testClock = new ManualEnvironmentEdge();
+
+        try (Connection conn =
+                 DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            conn.createStatement().execute(String.format(dataTableDDL, dataTableFullName));
+            conn.createStatement().execute(String.format(indexTableDDL, indexTableName,
+                dataTableFullName));
+            conn.commit();
+        }
+        upsertDataAndScrutinize(dataTableName, dataTableFullName, testClock);
+    }
+
+    @Test
+    public void testScrutinyOnRowsBeyondMaxLookback_viewIndex() throws Exception {
+        schema = "S"+generateUniqueName();
+        dataTableName = "T"+generateUniqueName();
+        dataTableFullName = SchemaUtil.getTableName(schema,dataTableName);
+        indexTableName = "VI"+generateUniqueName();
+        isViewIndex = true;
+        viewName = "V"+generateUniqueName();
+        String viewFullName = SchemaUtil.getTableName(schema,viewName);
+        String dataTableDDL = "CREATE TABLE %s (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, "
+            + "ZIP INTEGER) COLUMN_ENCODED_BYTES = 0, VERSIONS = 1 ";
+        String viewDDL = "CREATE VIEW %s AS SELECT * FROM %s";
+        String indexTableDDL = "CREATE INDEX %s ON %s (NAME) INCLUDE (ZIP) VERSIONS = 1";
+        testClock = new ManualEnvironmentEdge();
+
+        try (Connection conn =
+                 DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            conn.createStatement().execute(String.format(dataTableDDL, dataTableFullName));
+            conn.createStatement().execute(String.format(viewDDL, viewFullName, dataTableFullName));
+            conn.createStatement().execute(String.format(indexTableDDL, indexTableName,
+                viewFullName));
+            conn.commit();
+        }
+        upsertDataAndScrutinize(viewName, viewFullName, testClock);
+    }
+
+    private void upsertDataAndScrutinize(String tableName, String tableFullName,
+                                         ManualEnvironmentEdge testClock)
+        throws Exception {
+        try(Connection conn =
+                DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            // insert two rows
+            upsertDataStmt = getUpsertDataStmt(tableFullName, conn);
+
+            NonParameterizedIndexScrutinyToolIT.upsertRow(upsertDataStmt, 1, "name-1", 98051);
+            NonParameterizedIndexScrutinyToolIT.upsertRow(upsertDataStmt, 2, "name-2", 98052);
+            conn.commit();
+            long afterInsertSCN = EnvironmentEdgeManager.currentTimeMillis() + 1;
+            testClock.setValue(afterInsertSCN);
+            EnvironmentEdgeManager.injectEdge(testClock);
+            testClock.incrementValue(1);
+            testClock.incrementValue(MAX_LOOKBACK /2  * 1000);
+            long scrutinyTs = EnvironmentEdgeManager.currentTimeMillis();
+            changeIndexValue();
+            testClock.incrementValue(MAX_LOOKBACK /2  * 1000);
+            List<Job> completedJobs = runScrutiny(schema, tableName, indexTableName, scrutinyTs);
+            Job job = completedJobs.get(0);
+            assertTrue(job.isSuccessful());
+            assertCounters(job.getCounters());
+        }
+    }
+    private void changeIndexValue() throws SQLException {
 
 Review comment:
   Suggest to rename this function to be addIndexRow

----------------------------------------------------------------
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 #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…

Posted by GitBox <gi...@apache.org>.
swaroopak commented on a change in pull request #735: PHOENIX-5734 - IndexScrutinyTool should not report rows beyond maxLoo…
URL: https://github.com/apache/phoenix/pull/735#discussion_r394547036
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexScrutinyToolBaseIT.java
 ##########
 @@ -54,8 +54,9 @@
                 new ReadOnlyProps(clientProps.entrySet().iterator()));
     }
 
-    protected List<Job> runScrutiny(String[] cmdArgs) throws Exception {
-        IndexScrutinyTool scrutiny = new IndexScrutinyTool(IndexScrutinyMapperForTest.class);
+    protected List<Job> runScrutiny(Class<? extends IndexScrutinyMapper> mapperClass,
 
 Review comment:
   aah! nice :) 

----------------------------------------------------------------
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