[ 
https://issues.apache.org/jira/browse/PHOENIX-6181?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17213446#comment-17213446
 ] 

ASF GitHub Bot commented on PHOENIX-6181:
-----------------------------------------

gokceni commented on a change in pull request #915:
URL: https://github.com/apache/phoenix/pull/915#discussion_r504279338



##########
File path: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GlobalIndexRegionScanner.java
##########
@@ -240,4 +320,1033 @@ protected boolean isColumnIncluded(Cell cell) {
         byte[] qualifier = CellUtil.cloneQualifier(cell);
         return set.contains(qualifier);
     }
+    @VisibleForTesting
+    public boolean shouldVerify(IndexTool.IndexVerifyType verifyType,
+                                byte[] indexRowKey, Scan scan, Region region, 
IndexMaintainer indexMaintainer,
+                                IndexVerificationResultRepository 
verificationResultRepository, boolean shouldVerifyCheckDone) throws IOException 
{
+        this.verifyType = verifyType;
+        this.indexRowKeyforReadRepair = indexRowKey;
+        this.scan = scan;
+        this.region = region;
+        this.indexMaintainer = indexMaintainer;
+        this.verificationResultRepository = verificationResultRepository;
+        this.shouldVerifyCheckDone = shouldVerifyCheckDone;
+        return shouldVerify();
+    }
+
+    protected boolean shouldVerify() throws IOException {
+        // In case of read repair, proceed with rebuild
+        // All other types of rebuilds/verification should be incrementally 
performed if appropriate param is passed
+        byte[] lastVerifyTimeValue = 
scan.getAttribute(UngroupedAggregateRegionObserver.INDEX_RETRY_VERIFY);
+        Long lastVerifyTime = lastVerifyTimeValue == null ? 0 : 
Bytes.toLong(lastVerifyTimeValue);
+        if(indexRowKeyforReadRepair != null || lastVerifyTime == 0 || 
shouldVerifyCheckDone) {
+            return true;
+        }
+
+        IndexToolVerificationResult verificationResultTemp = 
verificationResultRepository
+                .getVerificationResult(lastVerifyTime, scan, region, 
indexMaintainer.getIndexTableName()) ;
+        if(verificationResultTemp != null) {
+            verificationResult = verificationResultTemp;
+        }
+        shouldVerifyCheckDone = true;
+        return verificationResultTemp == null;
+    }
+
+    @Override
+    public HRegionInfo getRegionInfo() {
+        return region.getRegionInfo();
+    }
+
+    @Override
+    public boolean isFilterDone() {
+        return false;
+    }
+
+    private void closeTables() throws IOException {
+        hTableFactory.shutdown();
+        if (indexHTable != null) {
+            indexHTable.close();
+        }
+        if (dataHTable != null) {
+            dataHTable.close();
+        }
+    }
+    @Override
+    public void close() throws IOException {
+        innerScanner.close();
+        if (indexRowKeyforReadRepair != null) {
+            closeTables();
+            return;
+        }
+        if (verify) {
+            try {
+                if (verificationResultRepository != null) {
+                    
verificationResultRepository.logToIndexToolResultTable(verificationResult,
+                            verifyType, 
region.getRegionInfo().getRegionName(), skipped);
+                }
+            } finally {
+                this.pool.stop("IndexRegionObserverRegionScanner is closing");
+                closeTables();
+                if (verificationResultRepository != null) {
+                    verificationResultRepository.close();
+                }
+                if (verificationOutputRepository != null) {
+                    verificationOutputRepository.close();
+                }
+            }
+        }
+        else {
+            this.pool.stop("IndexRegionObserverRegionScanner is closing");
+            closeTables();
+        }
+    }
+
+    @VisibleForTesting
+    public int setIndexTableTTL(int ttl) {
+        indexTableTTL = ttl;
+        return 0;

Review comment:
       What is the purpose of always returning 0?

##########
File path: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/IndexRepairRegionScanner.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.coprocessor;
+
+import static 
org.apache.phoenix.coprocessor.BaseScannerRegionObserver.PHYSICAL_DATA_TABLE_NAME;
+import static org.apache.phoenix.query.QueryConstants.AGG_TIMESTAMP;
+import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN;
+import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN_FAMILY;
+import static org.apache.phoenix.query.QueryConstants.UNGROUPED_AGG_ROW_KEY;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.query.HBaseFactoryProvider;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.compile.ScanRanges;
+import org.apache.phoenix.filter.SkipScanFilter;
+import org.apache.phoenix.hbase.index.parallel.Task;
+import org.apache.phoenix.hbase.index.parallel.TaskBatch;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.mapreduce.index.IndexTool;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.types.PLong;
+import org.apache.phoenix.schema.types.PVarbinary;
+import org.apache.phoenix.util.KeyValueUtil;
+import org.apache.phoenix.util.ServerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Maps;
+

Review comment:
       Most of the other classes have small descriptions of what they do. Let's 
add one here as well.

##########
File path: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GlobalIndexRegionScanner.java
##########
@@ -240,4 +320,1033 @@ protected boolean isColumnIncluded(Cell cell) {
         byte[] qualifier = CellUtil.cloneQualifier(cell);
         return set.contains(qualifier);
     }
+    @VisibleForTesting
+    public boolean shouldVerify(IndexTool.IndexVerifyType verifyType,
+                                byte[] indexRowKey, Scan scan, Region region, 
IndexMaintainer indexMaintainer,
+                                IndexVerificationResultRepository 
verificationResultRepository, boolean shouldVerifyCheckDone) throws IOException 
{
+        this.verifyType = verifyType;
+        this.indexRowKeyforReadRepair = indexRowKey;
+        this.scan = scan;
+        this.region = region;
+        this.indexMaintainer = indexMaintainer;
+        this.verificationResultRepository = verificationResultRepository;
+        this.shouldVerifyCheckDone = shouldVerifyCheckDone;
+        return shouldVerify();
+    }
+
+    protected boolean shouldVerify() throws IOException {
+        // In case of read repair, proceed with rebuild
+        // All other types of rebuilds/verification should be incrementally 
performed if appropriate param is passed
+        byte[] lastVerifyTimeValue = 
scan.getAttribute(UngroupedAggregateRegionObserver.INDEX_RETRY_VERIFY);
+        Long lastVerifyTime = lastVerifyTimeValue == null ? 0 : 
Bytes.toLong(lastVerifyTimeValue);
+        if(indexRowKeyforReadRepair != null || lastVerifyTime == 0 || 
shouldVerifyCheckDone) {

Review comment:
       Once this function is called, shouldVerifyCheckDone is set to true and 
from that point on, it will always be true. 
   Even if we want incremental, we will always do verify then why do we check 
lastVerifyTime? If lastVerifyTime is not 0, then shouldVerify check will not be 
set to true.
   Can we have just 1 global variable to decide if we should verify than 
multiple to make this easier?

##########
File path: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GlobalIndexRegionScanner.java
##########
@@ -240,4 +320,1033 @@ protected boolean isColumnIncluded(Cell cell) {
         byte[] qualifier = CellUtil.cloneQualifier(cell);
         return set.contains(qualifier);
     }
+    @VisibleForTesting
+    public boolean shouldVerify(IndexTool.IndexVerifyType verifyType,
+                                byte[] indexRowKey, Scan scan, Region region, 
IndexMaintainer indexMaintainer,
+                                IndexVerificationResultRepository 
verificationResultRepository, boolean shouldVerifyCheckDone) throws IOException 
{
+        this.verifyType = verifyType;
+        this.indexRowKeyforReadRepair = indexRowKey;
+        this.scan = scan;
+        this.region = region;
+        this.indexMaintainer = indexMaintainer;
+        this.verificationResultRepository = verificationResultRepository;
+        this.shouldVerifyCheckDone = shouldVerifyCheckDone;
+        return shouldVerify();
+    }
+
+    protected boolean shouldVerify() throws IOException {
+        // In case of read repair, proceed with rebuild
+        // All other types of rebuilds/verification should be incrementally 
performed if appropriate param is passed
+        byte[] lastVerifyTimeValue = 
scan.getAttribute(UngroupedAggregateRegionObserver.INDEX_RETRY_VERIFY);
+        Long lastVerifyTime = lastVerifyTimeValue == null ? 0 : 
Bytes.toLong(lastVerifyTimeValue);
+        if(indexRowKeyforReadRepair != null || lastVerifyTime == 0 || 
shouldVerifyCheckDone) {
+            return true;
+        }
+
+        IndexToolVerificationResult verificationResultTemp = 
verificationResultRepository
+                .getVerificationResult(lastVerifyTime, scan, region, 
indexMaintainer.getIndexTableName()) ;
+        if(verificationResultTemp != null) {
+            verificationResult = verificationResultTemp;
+        }
+        shouldVerifyCheckDone = true;
+        return verificationResultTemp == null;
+    }
+
+    @Override
+    public HRegionInfo getRegionInfo() {
+        return region.getRegionInfo();
+    }
+
+    @Override
+    public boolean isFilterDone() {
+        return false;
+    }
+
+    private void closeTables() throws IOException {
+        hTableFactory.shutdown();
+        if (indexHTable != null) {
+            indexHTable.close();
+        }
+        if (dataHTable != null) {
+            dataHTable.close();
+        }
+    }
+    @Override
+    public void close() throws IOException {
+        innerScanner.close();
+        if (indexRowKeyforReadRepair != null) {
+            closeTables();
+            return;
+        }
+        if (verify) {
+            try {
+                if (verificationResultRepository != null) {
+                    
verificationResultRepository.logToIndexToolResultTable(verificationResult,
+                            verifyType, 
region.getRegionInfo().getRegionName(), skipped);
+                }
+            } finally {
+                this.pool.stop("IndexRegionObserverRegionScanner is closing");
+                closeTables();
+                if (verificationResultRepository != null) {
+                    verificationResultRepository.close();
+                }
+                if (verificationOutputRepository != null) {
+                    verificationOutputRepository.close();
+                }
+            }
+        }
+        else {
+            this.pool.stop("IndexRegionObserverRegionScanner is closing");
+            closeTables();
+        }
+    }
+
+    @VisibleForTesting
+    public int setIndexTableTTL(int ttl) {
+        indexTableTTL = ttl;
+        return 0;
+    }
+
+    @VisibleForTesting
+    public int setIndexMaintainer(IndexMaintainer indexMaintainer) {
+        this.indexMaintainer = indexMaintainer;
+        return 0;
+    }
+
+    @VisibleForTesting
+    public long setMaxLookBackInMills(long maxLookBackInMills) {
+        this.maxLookBackInMills = maxLookBackInMills;
+        return 0;
+    }
+
+    public void logToIndexToolOutputTable(byte[] dataRowKey, byte[] 
indexRowKey, long dataRowTs, long indexRowTs,
+                                          String errorMsg, boolean 
isBeforeRebuild,
+                                          
IndexVerificationOutputRepository.IndexVerificationErrorType errorType) throws 
IOException {
+        logToIndexToolOutputTable(dataRowKey, indexRowKey, dataRowTs, 
indexRowTs, errorMsg, null,
+                null, isBeforeRebuild, errorType);
+    }
+
+    @VisibleForTesting
+    public void logToIndexToolOutputTable(byte[] dataRowKey, byte[] 
indexRowKey, long dataRowTs, long indexRowTs,
+                                          String errorMsg, byte[] 
expectedVaue, byte[] actualValue, boolean isBeforeRebuild,
+                                          
IndexVerificationOutputRepository.IndexVerificationErrorType errorType) throws 
IOException {
+        ungroupedAggregateRegionObserver.checkForRegionClosing();
+        verificationOutputRepository.logToIndexToolOutputTable(dataRowKey, 
indexRowKey, dataRowTs, indexRowTs,
+                errorMsg, expectedVaue, actualValue, 
scan.getTimeRange().getMax(),
+                region.getRegionInfo().getTable().getName(), isBeforeRebuild, 
errorType);
+    }
+
+    private static Cell getCell(Mutation m, byte[] family, byte[] qualifier) {
+        List<Cell> cellList = m.getFamilyCellMap().get(family);
+        if (cellList == null) {
+            return null;
+        }
+        for (Cell cell : cellList) {
+            if (CellUtil.matchingQualifier(cell, qualifier)) {
+                return cell;
+            }
+        }
+        return null;
+    }
+
+    private void logMismatch(Mutation expected, Mutation actual, int 
iteration, IndexToolVerificationResult.PhaseResult verificationPhaseResult, 
boolean isBeforeRebuild) throws IOException {
+        if (getTimestamp(expected) != getTimestamp(actual)) {
+            String errorMsg = "Not matching timestamp";
+            byte[] dataKey = indexMaintainer.buildDataRowKey(new 
ImmutableBytesWritable(expected.getRow()), viewConstants);
+            logToIndexToolOutputTable(dataKey, expected.getRow(), 
getTimestamp(expected), getTimestamp(actual),
+                    errorMsg, null, null, isBeforeRebuild, INVALID_ROW);
+            return;
+        }
+        int expectedCellCount = 0;
+        for (List<Cell> cells : expected.getFamilyCellMap().values()) {
+            if (cells == null) {
+                continue;
+            }
+            for (Cell expectedCell : cells) {
+                expectedCellCount++;
+                byte[] family = CellUtil.cloneFamily(expectedCell);
+                byte[] qualifier = CellUtil.cloneQualifier(expectedCell);
+                Cell actualCell = getCell(actual, family, qualifier);
+                if (actualCell == null ||
+                        !CellUtil.matchingType(expectedCell, actualCell)) {
+                    byte[] dataKey = indexMaintainer.buildDataRowKey(new 
ImmutableBytesWritable(expected.getRow()), viewConstants);
+                    String errorMsg = "Missing cell (in iteration " + 
iteration + ") " + Bytes.toString(family) + ":" + Bytes.toString(qualifier);
+                    logToIndexToolOutputTable(dataKey, expected.getRow(), 
getTimestamp(expected),
+                            getTimestamp(actual), errorMsg, isBeforeRebuild, 
INVALID_ROW);
+                    
verificationPhaseResult.setIndexHasMissingCellsCount(verificationPhaseResult.getIndexHasMissingCellsCount()
 + 1);
+                    return;
+                }
+                if (!CellUtil.matchingValue(actualCell, expectedCell)) {
+                    String errorMsg = "Not matching value (in iteration " + 
iteration + ") for " + Bytes.toString(family) + ":" + Bytes.toString(qualifier);

Review comment:
       Family qualifier will not be super useful if we do 5928

##########
File path: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/IndexRepairRegionScanner.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.coprocessor;
+
+import static 
org.apache.phoenix.coprocessor.BaseScannerRegionObserver.PHYSICAL_DATA_TABLE_NAME;
+import static org.apache.phoenix.query.QueryConstants.AGG_TIMESTAMP;
+import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN;
+import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN_FAMILY;
+import static org.apache.phoenix.query.QueryConstants.UNGROUPED_AGG_ROW_KEY;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.query.HBaseFactoryProvider;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.compile.ScanRanges;
+import org.apache.phoenix.filter.SkipScanFilter;
+import org.apache.phoenix.hbase.index.parallel.Task;
+import org.apache.phoenix.hbase.index.parallel.TaskBatch;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.mapreduce.index.IndexTool;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.types.PLong;
+import org.apache.phoenix.schema.types.PVarbinary;
+import org.apache.phoenix.util.KeyValueUtil;
+import org.apache.phoenix.util.ServerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Maps;
+
+public class IndexRepairRegionScanner extends GlobalIndexRegionScanner {
+
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(IndexRepairRegionScanner.class);
+
+    public IndexRepairRegionScanner(final RegionScanner innerScanner,
+                                     final Region region,
+                                     final Scan scan,
+                                     final RegionCoprocessorEnvironment env,
+                                     final UngroupedAggregateRegionObserver 
ungroupedAggregateRegionObserver)
+            throws IOException {
+        super(innerScanner, region, scan, env, 
ungroupedAggregateRegionObserver);
+
+        byte[] dataTableName = scan.getAttribute(PHYSICAL_DATA_TABLE_NAME);
+        dataHTable = hTableFactory.getTable(new 
ImmutableBytesPtr(dataTableName));
+        indexTableTTL = 
region.getTableDesc().getColumnFamilies()[0].getTimeToLive();
+        try (org.apache.hadoop.hbase.client.Connection connection =
+                     
HBaseFactoryProvider.getHConnectionFactory().createConnection(env.getConfiguration()))
 {
+            regionEndKeys = 
connection.getRegionLocator(dataHTable.getName()).getEndKeys();
+        }
+    }
+
+    public void prepareExpectedIndexMutations(Result dataRow, Map<byte[], 
List<Mutation>> expectedIndexMutationMap) throws IOException {
+        Put put = null;
+        Delete del = null;
+        for (Cell cell : dataRow.rawCells()) {
+            if (KeyValue.Type.codeToType(cell.getTypeByte()) == 
KeyValue.Type.Put) {
+                if (put == null) {
+                    put = new Put(CellUtil.cloneRow(cell));
+                }
+                put.add(cell);
+            } else {
+                if (del == null) {
+                    del = new Delete(CellUtil.cloneRow(cell));
+                }
+                del.addDeleteMarker(cell);
+            }
+        }
+        List<Mutation> indexMutations = 
prepareIndexMutationsForRebuild(indexMaintainer, put, del);
+        Collections.reverse(indexMutations);
+        for (Mutation mutation : indexMutations) {
+            byte[] indexRowKey = mutation.getRow();
+            List<Mutation> mutationList = 
expectedIndexMutationMap.get(indexRowKey);
+            if (mutationList == null) {
+                mutationList = new ArrayList<>();
+                mutationList.add(mutation);
+                expectedIndexMutationMap.put(indexRowKey, mutationList);
+            } else {
+                mutationList.add(mutation);
+            }
+        }
+    }
+
+    private void repairIndexRows(Map<byte[], List<Mutation>> indexMutationMap,
+                                 List<Mutation> indexRowsToBeDeleted,
+                                 IndexToolVerificationResult 
verificationResult) throws IOException {
+        try {
+            int batchSize = 0;
+            List<Mutation> indexUpdates = new 
ArrayList<Mutation>(maxBatchSize);
+            for (List<Mutation> mutationList : indexMutationMap.values()) {
+                indexUpdates.addAll(mutationList);
+                batchSize += mutationList.size();
+                if (batchSize >= maxBatchSize) {
+                    ungroupedAggregateRegionObserver.checkForRegionClosing();
+                    region.batchMutate(indexUpdates.toArray(new 
Mutation[indexUpdates.size()]),
+                            HConstants.NO_NONCE, HConstants.NO_NONCE);
+                    batchSize = 0;
+                    indexUpdates = new ArrayList<Mutation>(maxBatchSize);
+                }
+            }
+            if (batchSize > 0) {
+                ungroupedAggregateRegionObserver.checkForRegionClosing();
+                region.batchMutate(indexUpdates.toArray(new 
Mutation[indexUpdates.size()]),
+                        HConstants.NO_NONCE, HConstants.NO_NONCE);
+            }
+            batchSize = 0;
+            indexUpdates = new ArrayList<Mutation>(maxBatchSize);
+            for (Mutation mutation : indexRowsToBeDeleted) {
+                indexUpdates.add(mutation);
+                batchSize ++;
+                if (batchSize >= maxBatchSize) {
+                    ungroupedAggregateRegionObserver.checkForRegionClosing();
+                    region.batchMutate(indexUpdates.toArray(new 
Mutation[indexUpdates.size()]),
+                            HConstants.NO_NONCE, HConstants.NO_NONCE);
+                    batchSize = 0;
+                    indexUpdates = new ArrayList<Mutation>(maxBatchSize);
+                }
+            }
+            if (batchSize > 0) {
+                ungroupedAggregateRegionObserver.checkForRegionClosing();
+                region.batchMutate(indexUpdates.toArray(new 
Mutation[indexUpdates.size()]),
+                        HConstants.NO_NONCE, HConstants.NO_NONCE);

Review comment:
       +1

##########
File path: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
##########
@@ -1103,13 +1104,21 @@ private RegionScanner rebuildIndices(final 
RegionScanner innerScanner, final Reg
             if (oldCoproc) {
                 return new IndexerRegionScanner(scanner, region, scan, env, 
this);
             } else {
-                return new IndexRebuildRegionScanner(scanner, region, scan, 
env, this);
+                if 
(region.getTableDesc().hasCoprocessor(IndexRegionObserver.class.getCanonicalName()))
 {

Review comment:
       Don't we have IndexRegionObserver for new design all the time? What is 
the case when the index doesn't have both Indexer and IndexRegionObserver?




----------------------------------------------------------------
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:
[email protected]


> IndexRepairRegionScanner to verify and repair every global index row
> --------------------------------------------------------------------
>
>                 Key: PHOENIX-6181
>                 URL: https://issues.apache.org/jira/browse/PHOENIX-6181
>             Project: Phoenix
>          Issue Type: Improvement
>    Affects Versions: 5.0.0, 4.14.3
>            Reporter: Kadir OZDEMIR
>            Assignee: Kadir OZDEMIR
>            Priority: Major
>         Attachments: PHOENIX-6181.4.x.001.patch
>
>
> IndexRebuildRegionScanner is the server side engine to rebuild and verify 
> every index row pointed by the data table. IndexRebuildRegionScanner runs on 
> data table regions and scans every data table rows locally, and then rebuilds 
> and verifies index table rows referenced by the data table rows over 
> server-to-server RPCs using the HBase client installed on region servers. 
> However, IndexRebuildRegionScanner cannot clean up the index rows that are 
> not referenced by the data table if there are such index rows. In order to do 
> that we need another region scanner that scans index table regions and makes 
> sure that every index row is valid. This region scanner will be called 
> IndexRepairRegionScanner.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to