haridsv commented on code in PR #2379:
URL: https://github.com/apache/phoenix/pull/2379#discussion_r3056893019


##########
phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java:
##########
@@ -0,0 +1,351 @@
+/*
+ * 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.mapreduce.PhoenixSyncTableTool.DEFAULT_PHOENIX_SYNC_TABLE_CHUNK_SIZE_BYTES;
+import static org.apache.phoenix.query.QueryConstants.AGG_TIMESTAMP;
+import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN_FAMILY;
+import static org.apache.phoenix.schema.types.PDataType.FALSE_BYTES;
+import static org.apache.phoenix.schema.types.PDataType.TRUE_BYTES;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.regionserver.PhoenixScannerContext;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.coprocessorclient.BaseScannerRegionObserverConstants;
+import org.apache.phoenix.util.PhoenixKeyValueUtil;
+import org.apache.phoenix.util.SHA256DigestUtil;
+import org.bouncycastle.crypto.digests.SHA256Digest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Server-side coprocessor that performs chunk formation and SHA-256 hashing 
for
+ * PhoenixSyncTableTool.
+ * <p>
+ * Accumulates rows into chunks (based on size limits) and computes a hash of 
all row data (keys,
+ * column families, qualifiers, timestamps, cell types, values). In case of 
paging timeout, return
+ * whatever is accumulated in chunk. If nothing is accumulated return dummy 
row either with prev
+ * result rowKey or max possible key < currentRowKey
+ * <p>
+ * Source scan (isTargetClusterScan=false): Returns complete chunks(if paging 
dint timeout) bounded
+ * by region boundaries. Sets hasMoreRows=false when region is exhausted.
+ * <p>
+ * Target scan (isTargetClusterScan=true): Returns partial chunks with 
serialized digest state when
+ * region boundary is reached, allowing cross-region hash continuation.
+ * <p>
+ * Returns chunk metadata cells: START_KEY, END_KEY, HASH (or digest state), 
ROW_COUNT,
+ * IS_PARTIAL_CHUNK
+ */
+public class PhoenixSyncTableRegionScanner extends BaseRegionScanner {
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(PhoenixSyncTableRegionScanner.class);
+  private static final byte[] CHUNK_METADATA_FAMILY = SINGLE_COLUMN_FAMILY;
+  private final Region region;
+  private final Scan scan;
+  private final RegionCoprocessorEnvironment env;
+  private final UngroupedAggregateRegionObserver 
ungroupedAggregateRegionObserver;
+  private final long chunkSizeBytes;
+  private boolean isTargetClusterScan = false;
+  private boolean hasMoreRows = true;
+  private final long pageSizeMs;
+  private Chunk currentChunk;
+
+  /**
+   * Creates a PhoenixSyncTableRegionScanner for chunk-based hashing.
+   * @param innerScanner                     The underlying region scanner
+   * @param region                           The region being scanned
+   * @param scan                             The scan request
+   * @param env                              The coprocessor environment
+   * @param ungroupedAggregateRegionObserver Parent observer for region state 
checks
+   * @param pageSizeMs                       Paging timeout in milliseconds
+   */
+  public PhoenixSyncTableRegionScanner(final RegionScanner innerScanner, final 
Region region,
+    final Scan scan, final RegionCoprocessorEnvironment env,
+    final UngroupedAggregateRegionObserver ungroupedAggregateRegionObserver, 
long pageSizeMs) {
+    super(innerScanner);
+    this.region = region;
+    this.scan = scan;
+    this.env = env;
+    this.ungroupedAggregateRegionObserver = ungroupedAggregateRegionObserver;
+    byte[] chunkSizeAttr =
+      
scan.getAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CHUNK_SIZE_BYTES);
+    if (chunkSizeAttr == null) { // Since we don't set chunk size scan attr 
for target cluster scan
+      this.isTargetClusterScan = true;
+    }
+    this.chunkSizeBytes = chunkSizeAttr != null
+      ? Bytes.toLong(chunkSizeAttr)
+      : DEFAULT_PHOENIX_SYNC_TABLE_CHUNK_SIZE_BYTES;
+    this.pageSizeMs = pageSizeMs;
+  }
+
+  @Override
+  public boolean next(List<Cell> results) throws IOException {
+    return next(results, null);
+  }
+
+  /**
+   * Accumulates rows into a chunk and returns chunk metadata cells. Supports 
server-side paging via
+   * {@link PhoenixScannerContext} following the same pattern as
+   * {@link GroupedAggregateRegionObserver} and {@link 
UncoveredIndexRegionScanner}.
+   * @param results        Output list to populate with chunk metadata cells
+   * @param scannerContext Phoenix scanner context for paging timeout detection
+   * @return true if more chunks available, false if scanning complete
+   */
+  @Override
+  public boolean next(List<Cell> results, ScannerContext scannerContext) 
throws IOException {
+    region.startRegionOperation();
+    try {
+      currentChunk = createNewChunk();
+      RegionScanner localScanner = delegate;
+      synchronized (localScanner) {
+        List<Cell> rowCells = new ArrayList<>();
+        while (hasMoreRows) {
+          ungroupedAggregateRegionObserver.checkForRegionClosingOrSplitting();
+          rowCells.clear();
+          hasMoreRows = (scannerContext == null)
+            ? localScanner.nextRaw(rowCells)
+            : localScanner.nextRaw(rowCells, scannerContext);
+
+          if (rowCells.isEmpty()) {
+            break;
+          }
+
+          byte[] rowKey = CellUtil.cloneRow(rowCells.get(0));
+          currentChunk.addRow(rowKey, rowCells);

Review Comment:
   nit comment, but since rowKey is not needed locally, it would be cleaner to 
push it into the addRow().



##########
phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java:
##########
@@ -0,0 +1,351 @@
+/*
+ * 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.mapreduce.PhoenixSyncTableTool.DEFAULT_PHOENIX_SYNC_TABLE_CHUNK_SIZE_BYTES;
+import static org.apache.phoenix.query.QueryConstants.AGG_TIMESTAMP;
+import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN_FAMILY;
+import static org.apache.phoenix.schema.types.PDataType.FALSE_BYTES;
+import static org.apache.phoenix.schema.types.PDataType.TRUE_BYTES;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.regionserver.PhoenixScannerContext;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.coprocessorclient.BaseScannerRegionObserverConstants;
+import org.apache.phoenix.util.PhoenixKeyValueUtil;
+import org.apache.phoenix.util.SHA256DigestUtil;
+import org.bouncycastle.crypto.digests.SHA256Digest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Server-side coprocessor that performs chunk formation and SHA-256 hashing 
for
+ * PhoenixSyncTableTool.
+ * <p>
+ * Accumulates rows into chunks (based on size limits) and computes a hash of 
all row data (keys,
+ * column families, qualifiers, timestamps, cell types, values). In case of 
paging timeout, return
+ * whatever is accumulated in chunk. If nothing is accumulated return dummy 
row either with prev
+ * result rowKey or max possible key < currentRowKey
+ * <p>
+ * Source scan (isTargetClusterScan=false): Returns complete chunks(if paging 
dint timeout) bounded
+ * by region boundaries. Sets hasMoreRows=false when region is exhausted.
+ * <p>
+ * Target scan (isTargetClusterScan=true): Returns partial chunks with 
serialized digest state when
+ * region boundary is reached, allowing cross-region hash continuation.
+ * <p>
+ * Returns chunk metadata cells: START_KEY, END_KEY, HASH (or digest state), 
ROW_COUNT,
+ * IS_PARTIAL_CHUNK
+ */
+public class PhoenixSyncTableRegionScanner extends BaseRegionScanner {
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(PhoenixSyncTableRegionScanner.class);
+  private static final byte[] CHUNK_METADATA_FAMILY = SINGLE_COLUMN_FAMILY;
+  private final Region region;
+  private final Scan scan;
+  private final RegionCoprocessorEnvironment env;
+  private final UngroupedAggregateRegionObserver 
ungroupedAggregateRegionObserver;
+  private final long chunkSizeBytes;
+  private boolean isTargetClusterScan = false;
+  private boolean hasMoreRows = true;
+  private final long pageSizeMs;
+  private Chunk currentChunk;
+
+  /**
+   * Creates a PhoenixSyncTableRegionScanner for chunk-based hashing.
+   * @param innerScanner                     The underlying region scanner
+   * @param region                           The region being scanned
+   * @param scan                             The scan request
+   * @param env                              The coprocessor environment
+   * @param ungroupedAggregateRegionObserver Parent observer for region state 
checks
+   * @param pageSizeMs                       Paging timeout in milliseconds
+   */
+  public PhoenixSyncTableRegionScanner(final RegionScanner innerScanner, final 
Region region,
+    final Scan scan, final RegionCoprocessorEnvironment env,
+    final UngroupedAggregateRegionObserver ungroupedAggregateRegionObserver, 
long pageSizeMs) {
+    super(innerScanner);
+    this.region = region;
+    this.scan = scan;
+    this.env = env;
+    this.ungroupedAggregateRegionObserver = ungroupedAggregateRegionObserver;
+    byte[] chunkSizeAttr =
+      
scan.getAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CHUNK_SIZE_BYTES);
+    if (chunkSizeAttr == null) { // Since we don't set chunk size scan attr 
for target cluster scan
+      this.isTargetClusterScan = true;
+    }
+    this.chunkSizeBytes = chunkSizeAttr != null
+      ? Bytes.toLong(chunkSizeAttr)
+      : DEFAULT_PHOENIX_SYNC_TABLE_CHUNK_SIZE_BYTES;
+    this.pageSizeMs = pageSizeMs;
+  }
+
+  @Override
+  public boolean next(List<Cell> results) throws IOException {
+    return next(results, null);
+  }
+
+  /**
+   * Accumulates rows into a chunk and returns chunk metadata cells. Supports 
server-side paging via
+   * {@link PhoenixScannerContext} following the same pattern as
+   * {@link GroupedAggregateRegionObserver} and {@link 
UncoveredIndexRegionScanner}.
+   * @param results        Output list to populate with chunk metadata cells
+   * @param scannerContext Phoenix scanner context for paging timeout detection
+   * @return true if more chunks available, false if scanning complete
+   */
+  @Override
+  public boolean next(List<Cell> results, ScannerContext scannerContext) 
throws IOException {
+    region.startRegionOperation();
+    try {
+      currentChunk = createNewChunk();
+      RegionScanner localScanner = delegate;
+      synchronized (localScanner) {
+        List<Cell> rowCells = new ArrayList<>();
+        while (hasMoreRows) {
+          ungroupedAggregateRegionObserver.checkForRegionClosingOrSplitting();
+          rowCells.clear();
+          hasMoreRows = (scannerContext == null)
+            ? localScanner.nextRaw(rowCells)
+            : localScanner.nextRaw(rowCells, scannerContext);
+
+          if (rowCells.isEmpty()) {
+            break;
+          }
+
+          byte[] rowKey = CellUtil.cloneRow(rowCells.get(0));
+          currentChunk.addRow(rowKey, rowCells);
+          if (!isTargetClusterScan && 
currentChunk.exceedsSize(chunkSizeBytes)) {
+            break;
+          }
+
+          if (
+            hasMoreRows && 
(PhoenixScannerContext.isReturnImmediately(scannerContext)
+              || PhoenixScannerContext.isTimedOut(scannerContext, pageSizeMs))
+          ) {
+            LOGGER.info("Paging timeout after {} rows ({} bytes) in region {}, 
chunk [{}:{}]",
+              currentChunk.getRowCount(), currentChunk.getSize(),
+              region.getRegionInfo().getRegionNameAsString(),
+              Bytes.toStringBinary(currentChunk.getStartKey()),
+              Bytes.toStringBinary(currentChunk.getEndKey()));
+            PhoenixScannerContext.setReturnImmediately(scannerContext);
+            break;
+          }
+        }
+      }
+
+      if (currentChunk.isEmpty()) {
+        return false;
+      }
+      buildChunkMetadataResult(results, currentChunk, isTargetClusterScan);
+      return hasMoreRows;
+    } catch (Throwable t) {
+      LOGGER.error(
+        "Exception during chunk scanning in region {} table {} at chunk 
startKey: {}, endKey: {})",
+        region.getRegionInfo().getRegionNameAsString(),
+        region.getRegionInfo().getTable().getNameAsString(),
+        currentChunk != null && !currentChunk.isEmpty()
+          ? Bytes.toStringBinary(currentChunk.getStartKey())
+          : "null",
+        currentChunk != null && !currentChunk.isEmpty()
+          ? Bytes.toStringBinary(currentChunk.getEndKey())
+          : "null",
+        t);
+      throw t;
+    } finally {
+      region.closeRegionOperation();
+    }
+  }
+
+  /**
+   * Creates a new chunk, with continued digest for first chunk if applicable. 
Only the very first
+   * chunk (when previousResultRowKey is null) can have a continued digest. 
All subsequent chunks
+   * get a fresh digest.
+   */
+  private Chunk createNewChunk() {
+    byte[] continuedDigestStateAttr =
+      
scan.getAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CONTINUED_DIGEST_STATE);
+    if (continuedDigestStateAttr != null) {
+      try {
+        SHA256Digest continuedDigest = 
SHA256DigestUtil.decodeDigestState(continuedDigestStateAttr);
+        return new Chunk(continuedDigest);
+      } catch (IOException e) {
+        throw new IllegalStateException("Failed to restore continued digest 
state", e);
+      }
+    }
+    return new Chunk(null);
+  }
+
+  /**
+   * Builds chunk metadata result cells and adds them to the results list. 
Returns a single
+   * "row"[rowKey=chunkEndKey] with multiple cells containing chunk 
metadata[chunkStartKey,
+   * hash/digest, rowCount, isPartialChunk]. For complete chunks: includes 
final SHA-256 hash (32
+   * bytes) For partial chunks: includes serialized MessageDigest state for 
continuation
+   * @param results             Output list to populate with chunk metadata 
cells
+   * @param chunk               The chunk to build metadata for
+   * @param isTargetClusterScan Whether this is a target cluster scan (partial 
chunk)
+   */
+  static void buildChunkMetadataResult(List<Cell> results, Chunk chunk,
+    boolean isTargetClusterScan) {
+    byte[] resultRowKey = chunk.getEndKey();
+    // If we are scanning target cluster, we always return partial chunk digest
+    // and compute final checksum in Mapper itself when scanner.next() returns 
null
+    boolean isPartialChunk = isTargetClusterScan;
+    results.add(PhoenixKeyValueUtil.newKeyValue(resultRowKey, 
CHUNK_METADATA_FAMILY,
+      BaseScannerRegionObserverConstants.SYNC_TABLE_START_KEY_QUALIFIER, 
AGG_TIMESTAMP,
+      chunk.getStartKey()));
+    results.add(PhoenixKeyValueUtil.newKeyValue(resultRowKey, 
CHUNK_METADATA_FAMILY,
+      BaseScannerRegionObserverConstants.SYNC_TABLE_ROW_COUNT_QUALIFIER, 
AGG_TIMESTAMP,
+      Bytes.toBytes(chunk.getRowCount())));
+    if (isPartialChunk) {
+      // Partial chunk - return serialized digest state
+      byte[] digestState = chunk.getDigestState();
+      results.add(PhoenixKeyValueUtil.newKeyValue(resultRowKey, 
CHUNK_METADATA_FAMILY,
+        
BaseScannerRegionObserverConstants.SYNC_TABLE_IS_PARTIAL_CHUNK_QUALIFIER, 
AGG_TIMESTAMP,
+        TRUE_BYTES));
+      results.add(PhoenixKeyValueUtil.newKeyValue(resultRowKey, 
CHUNK_METADATA_FAMILY,
+        BaseScannerRegionObserverConstants.SYNC_TABLE_HASH_QUALIFIER, 
AGG_TIMESTAMP, digestState));
+    } else {
+      // Complete chunk - finalize and return hash
+      byte[] hash = chunk.finalizeHash();
+      results.add(PhoenixKeyValueUtil.newKeyValue(resultRowKey, 
CHUNK_METADATA_FAMILY,
+        BaseScannerRegionObserverConstants.SYNC_TABLE_HASH_QUALIFIER, 
AGG_TIMESTAMP, hash));
+      results.add(PhoenixKeyValueUtil.newKeyValue(resultRowKey, 
CHUNK_METADATA_FAMILY,
+        
BaseScannerRegionObserverConstants.SYNC_TABLE_IS_PARTIAL_CHUNK_QUALIFIER, 
AGG_TIMESTAMP,
+        FALSE_BYTES));
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    try {
+      super.close();
+    } catch (Exception e) {
+      LOGGER.error("Error closing PhoenixSyncTableRegionScanner", e);
+    }
+  }
+
+  /**
+   * Encapsulates a chunk of rows being accumulated for sync verification. 
Manages chunk boundaries,
+   * size tracking, row counting, and SHA-256 digest computation.
+   */
+  static class Chunk {
+    private byte[] startKey;
+    private byte[] endKey;
+    private long size;
+    private long rowCount;
+    private final SHA256Digest digest;
+
+    /**
+     * Creates a new chunk with optional continued digest state.
+     * @param continuedDigest If not null, continues hashing from previous 
chunk's state (used for
+     *                        target scans spanning multiple regions)
+     */
+    Chunk(SHA256Digest continuedDigest) {
+      this.digest = continuedDigest != null ? continuedDigest : new 
SHA256Digest();
+      this.size = 0;
+      this.rowCount = 0;
+    }
+
+    /**
+     * Adds a row to this chunk, updating size, count, and digest.
+     */
+    void addRow(byte[] rowKey, List<Cell> cells) {
+      if (startKey == null) {
+        startKey = rowKey;
+      }
+      endKey = rowKey;
+      size += calculateRowSize(cells);
+      rowCount++;
+      updateDigest(rowKey, cells);
+    }
+
+    private long calculateRowSize(List<Cell> cells) {
+      long rowSize = 0;
+      for (Cell cell : cells) {
+        rowSize += PrivateCellUtil.estimatedSerializedSizeOf(cell);

Review Comment:
   PrivateCellUtil is marked as `@InterfaceAudience.Private`, which in theory 
means we shouldn't use it, but Phoenix is already using many such internal 
API's so not sure if we should continue to add more or put a conscious effort 
to avoid the creep.



##########
phoenix-core-client/src/main/java/org/apache/phoenix/util/SHA256DigestUtil.java:
##########
@@ -32,53 +28,49 @@
 public class SHA256DigestUtil {
 
   /**
-   * Maximum allowed size for encoded SHA-256 digest state. SHA-256 state is 
~96 bytes, we allow up
-   * to 128 bytes as buffer.
+   * Maximum allowed size for encoded SHA-256 digest state. BouncyCastle's 
SHA256Digest encoded
+   * state ranges from 53 to 113 bytes (52 base + 0-60 buffered words + 1 
purpose byte). We allow up
+   * to 128 bytes as headroom.
    */
   public static final int MAX_SHA256_DIGEST_STATE_SIZE = 128;
 
   /**
-   * Encodes a SHA256Digest state to a byte array with length prefix for 
validation. Format: [4-byte
-   * integer length][encoded digest state bytes]
+   * Encodes a SHA256Digest state to a byte array.
    * @param digest The digest whose state should be encoded
-   * @return Byte array containing integer length prefix + encoded state
+   * @return Byte array containing the raw BouncyCastle encoded state
    */
   public static byte[] encodeDigestState(SHA256Digest digest) {
     byte[] encoded = digest.getEncodedState();
-    ByteBuffer buffer = ByteBuffer.allocate(Bytes.SIZEOF_INT + encoded.length);
-    buffer.putInt(encoded.length);
-    buffer.put(encoded);
-    return buffer.array();
+    if (encoded.length > MAX_SHA256_DIGEST_STATE_SIZE) {
+      throw new IllegalArgumentException(
+        String.format("SHA256 encoded state too large: %d, expected <= %d", 
encoded.length,
+          MAX_SHA256_DIGEST_STATE_SIZE));
+    }

Review Comment:
   The byte[] is being produced and consumed by the same class, so my question 
is on why it could become invalid in the middle of this pass.



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to