tkhurana commented on code in PR #2379: URL: https://github.com/apache/phoenix/pull/2379#discussion_r2967701021
########## phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java: ########## @@ -0,0 +1,383 @@ +/* + * 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.util.PhoenixConfigurationUtil.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 static org.apache.phoenix.util.ScanUtil.getDummyResult; + +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.HConstants; +import org.apache.hadoop.hbase.PrivateCellUtil; +import org.apache.hadoop.hbase.client.RegionInfo; +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.ByteUtil; +import org.apache.phoenix.util.PhoenixKeyValueUtil; +import org.apache.phoenix.util.SHA256DigestUtil; +import org.apache.phoenix.util.ScanUtil; +import org.bouncycastle.crypto.digests.SHA256Digest; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; + +/** + * 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 (isTargetScan=false): Returns complete chunks(if paging dint timeout) bounded by + * region boundaries. Sets hasMoreRows=false when region is exhausted. + * <p> + * Target scan (isTargetScan=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 isTargetScan = false; + private byte[] chunkStartKey = null; + private byte[] chunkEndKey = null; + private long currentChunkSize = 0L; + private long currentChunkRowCount = 0L; + private final SHA256Digest digest; + private boolean hasMoreRows = true; + private boolean isUsingContinuedDigest; + private byte[] previousResultRowKey = null; + private final byte[] initStartRowKey; + private final boolean includeInitStartRowKey; + private final long pageSizeMs; + + /** + * 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 + * @throws IllegalStateException if digest state restoration fails + */ + @VisibleForTesting + 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.isTargetScan = true; + } + this.chunkSizeBytes = chunkSizeAttr != null + ? Bytes.toLong(chunkSizeAttr) + : DEFAULT_PHOENIX_SYNC_TABLE_CHUNK_SIZE_BYTES; + + // Check if we should continue from a previous digest state (cross-region continuation) + byte[] continuedDigestStateAttr = + scan.getAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CONTINUED_DIGEST_STATE); + if (continuedDigestStateAttr != null) { + try { + this.digest = SHA256DigestUtil.decodeDigestState(continuedDigestStateAttr); + this.isUsingContinuedDigest = true; + } catch (IOException e) { + throw new IllegalStateException("Failed to restore continued digest state", e); + } + } else { + this.digest = new SHA256Digest(); + this.isUsingContinuedDigest = false; + } + this.initStartRowKey = scan.getStartRow(); + this.includeInitStartRowKey = scan.includeStartRow(); + 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 { + resetChunkState(); Review Comment: If you have a notion of a chunk object then you don't need reset you can simply create a new chunk -- 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]
