From d70fd132b185595d923b698bf08d5f610697305b Mon Sep 17 00:00:00 2001 From: Rahul Kumar Date: Tue, 10 Feb 2026 23:34:46 +0530 Subject: [PATCH 1/9] PHOENIX-7751 : PhoenixSyncTable tool to validate data b/w source and target cluster --- .../BaseScannerRegionObserverConstants.java | 23 + .../apache/phoenix/query/QueryServices.java | 7 + .../phoenix/query/QueryServicesOptions.java | 5 + .../org/apache/phoenix/util/ScanUtil.java | 4 + .../PhoenixSyncTableRegionScanner.java | 376 +++++++++++ .../UngroupedAggregateRegionObserver.java | 8 + .../PhoenixNoOpSingleRecordReader.java | 113 ++++ .../PhoenixSyncTableInputFormat.java | 197 ++++++ .../mapreduce/PhoenixSyncTableMapper.java | 601 ++++++++++++++++++ .../PhoenixSyncTableOutputRepository.java | 295 +++++++++ .../mapreduce/PhoenixSyncTableOutputRow.java | 83 +++ .../mapreduce/PhoenixSyncTableTool.java | 439 +++++++++++++ .../mapreduce/index/IndexScrutinyTool.java | 17 +- .../phoenix/mapreduce/index/IndexTool.java | 15 +- .../mapreduce/transform/TransformTool.java | 8 +- .../util/PhoenixConfigurationUtil.java | 96 +++ .../mapreduce/util/PhoenixMapReduceUtil.java | 104 +++ 17 files changed, 2358 insertions(+), 33 deletions(-) create mode 100644 phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java create mode 100644 phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixNoOpSingleRecordReader.java create mode 100644 phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java create mode 100644 phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java create mode 100644 phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java create mode 100644 phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java create mode 100644 phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/BaseScannerRegionObserverConstants.java b/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/BaseScannerRegionObserverConstants.java index dee99dc463f..3de94ba9739 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/BaseScannerRegionObserverConstants.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/BaseScannerRegionObserverConstants.java @@ -194,6 +194,29 @@ public static long getMaxLookbackInMillis(Configuration conf) { */ public static final String CUSTOM_ANNOTATIONS = "_Annot"; + /** + * SyncTableTool scan attributes for server-side chunk formation and checksum + */ + public static final String SYNC_TABLE_CHUNK_FORMATION = "_SyncTableChunk"; + public static final String SYNC_TABLE_MAPPER_REGION_END_KEY = "_SyncTableMapperRegionEnd"; + public static final String SYNC_TABLE_CHUNK_SIZE_BYTES = "_SyncTableChunkSizeBytes"; + public static final String SYNC_TABLE_CHUNK_MAX_ROWS = "_SyncTableChunkMaxRows"; + public static final String SYNC_TABLE_FORCE_FULL_RANGE = "_SyncTableForceFullRange"; + public static final String SYNC_TABLE_CONTINUED_DIGEST_STATE = "_SyncTableContinuedDigestState"; + + /** + * SyncTableTool chunk metadata cell qualifiers. These define the wire protocol between + * PhoenixSyncTableRegionScanner (server-side coprocessor) and PhoenixSyncTableMapper (client-side + * mapper). The coprocessor returns chunk metadata as HBase cells with these qualifiers, and the + * mapper parses them to extract chunk information. + */ + public static final byte[] SYNC_TABLE_START_KEY_QUALIFIER = Bytes.toBytes("START_KEY"); + public static final byte[] SYNC_TABLE_END_KEY_QUALIFIER = Bytes.toBytes("END_KEY"); + public static final byte[] SYNC_TABLE_HASH_QUALIFIER = Bytes.toBytes("HASH"); + public static final byte[] SYNC_TABLE_ROW_COUNT_QUALIFIER = Bytes.toBytes("ROW_COUNT"); + public static final byte[] SYNC_TABLE_IS_PARTIAL_QUALIFIER = Bytes.toBytes("IS_PARTIAL"); + public static final byte[] SYNC_TABLE_HAS_MORE_ROWS_QUALIFIER = Bytes.toBytes("HAS_MORE_ROWS"); + /** Exposed for testing */ public static final String SCANNER_OPENED_TRACE_INFO = "Scanner opened on server"; } diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServices.java index 5ed24f77a6d..5f85bd48d0c 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServices.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServices.java @@ -212,6 +212,13 @@ public interface QueryServices extends SQLCloseable { public static final String SERVER_SIDE_PRIOIRTY_ATTRIB = "phoenix.serverside.rpc.priority"; public static final String ALLOW_LOCAL_INDEX_ATTRIB = "phoenix.index.allowLocalIndex"; + public static final String SYNC_TABLE_QUERY_TIMEOUT_ATTRIB = "phoenix.synctable.query.timeout"; + public static final String SYNC_TABLE_RPC_TIMEOUT_ATTRIB = "phoenix.synctable.rpc.timeout"; + public static final String SYNC_TABLE_CLIENT_SCANNER_TIMEOUT_ATTRIB = + "phoenix.synctable.client.scanner.timeout"; + public static final String SYNC_TABLE_RPC_RETRIES_COUNTER = + "phoenix.synctable.rpc.retries.counter"; + // Retries when doing server side writes to SYSTEM.CATALOG public static final String METADATA_WRITE_RETRIES_NUMBER = "phoenix.metadata.rpc.retries.number"; public static final String METADATA_WRITE_RETRY_PAUSE = "phoenix.metadata.rpc.pause"; diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java index c4c44108566..e3854ae4f49 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java @@ -248,6 +248,11 @@ public class QueryServicesOptions { // hrs public static final long DEFAULT_INDEX_PENDING_DISABLE_THRESHOLD = 30000; // 30 secs + public static final long DEFAULT_SYNC_TABLE_QUERY_TIMEOUT = (5 * 30000 * 60) + 2100; + public static final long DEFAULT_SYNC_TABLE_RPC_TIMEOUT = 30000 * 60; // 30 mins + public static final long DEFAULT_SYNC_TABLE_CLIENT_SCANNER_TIMEOUT = 30000 * 60; // 30 mins + public static final int DEFAULT_SYNC_TABLE_RPC_RETRIES_COUNTER = 5; // 5 total tries at rpc level + /** * HConstants#HIGH_QOS is the max we will see to a standard table. We go higher to differentiate * and give some room for things in the middle diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/util/ScanUtil.java b/phoenix-core-client/src/main/java/org/apache/phoenix/util/ScanUtil.java index 7f66b1d8ddd..4bd75fd119b 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/util/ScanUtil.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/util/ScanUtil.java @@ -1011,6 +1011,10 @@ public static boolean isIndexRebuild(Scan scan) { return scan.getAttribute((BaseScannerRegionObserverConstants.REBUILD_INDEXES)) != null; } + public static boolean isSyncTableChunkFormation(Scan scan) { + return scan.getAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CHUNK_FORMATION) != null; + } + public static int getClientVersion(Scan scan) { int clientVersion = UNKNOWN_CLIENT_VERSION; byte[] clientVersionBytes = diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java new file mode 100644 index 00000000000..38b677d0a07 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java @@ -0,0 +1,376 @@ +/* + * 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.*; +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.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +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.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.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. + *

+ * Accumulates rows into chunks (based on size/row limits) and computes a hash of all row data + * (keys, column families, qualifiers, timestamps, cell types, values). + *

+ * Source mode (forceFullRange=false): Returns complete chunks bounded by region boundaries. Sets + * hasMoreRows=false when region is exhausted. + *

+ * Target mode (forceFullRange=true): Returns partial chunks with serialized digest state when + * region boundary is reached, allowing cross-region hash continuation. + *

+ * Returns chunk metadata cells: START_KEY, END_KEY, HASH (or digest state), ROW_COUNT, IS_PARTIAL, + * HAS_MORE_ROWS. + */ +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 static final int MAX_SHA256_STATE_SIZE = 128; + private final Region region; + private final Scan scan; + private final RegionCoprocessorEnvironment env; + private final UngroupedAggregateRegionObserver ungroupedAggregateRegionObserver; + + private final byte[] mapperRegionEndKey; + private final long chunkSizeBytes; + private final long chunkMaxRows; + private final boolean forceFullRange; + private byte[] chunkStartKey = null; + private byte[] chunkEndKey = null; + private long currentChunkSize = 0L; + private long currentChunkRowCount = 0L; + private SHA256Digest digest; + private boolean hasMoreRows = true; + private boolean isUsingContinuedDigest = false; + + /** + * @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 + */ + @VisibleForTesting + public PhoenixSyncTableRegionScanner(final RegionScanner innerScanner, final Region region, + final Scan scan, final RegionCoprocessorEnvironment env, + final UngroupedAggregateRegionObserver ungroupedAggregateRegionObserver) { + super(innerScanner); + this.region = region; + this.scan = scan; + this.env = env; + this.ungroupedAggregateRegionObserver = ungroupedAggregateRegionObserver; + + byte[] mapperEndAttr = + scan.getAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_MAPPER_REGION_END_KEY); + byte[] chunkSizeAttr = + scan.getAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CHUNK_SIZE_BYTES); + byte[] chunkMaxRowsAttr = + scan.getAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CHUNK_MAX_ROWS); + this.mapperRegionEndKey = mapperEndAttr != null ? mapperEndAttr : new byte[0]; + this.chunkSizeBytes = chunkSizeAttr != null + ? Bytes.toLong(chunkSizeAttr) + : DEFAULT_PHOENIX_SYNC_TABLE_CHUNK_SIZE_BYTES; + this.chunkMaxRows = chunkMaxRowsAttr != null + ? Bytes.toLong(chunkMaxRowsAttr) + : DEFAULT_PHOENIX_SYNC_TABLE_CHUNK_MAX_ROWS; + byte[] forceFullRangeAttr = + scan.getAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_FORCE_FULL_RANGE); + this.forceFullRange = Bytes.toBoolean(forceFullRangeAttr); + // 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 = decodeDigestState(continuedDigestStateAttr); + this.isUsingContinuedDigest = true; + } catch (IOException e) { + throw new RuntimeException("Failed to restore continued digest state", e); + } + } else { + this.digest = new SHA256Digest(); + this.isUsingContinuedDigest = false; + } + } + + /** + * Accumulates rows into a chunk and returns chunk metadata cells. + * @param results Output list to populate with chunk metadata cells + * @return true if more chunks available, false if scanning complete + */ + @Override + public boolean next(List results) throws IOException { + region.startRegionOperation(); + try { + resetChunkState(); + RegionScanner localScanner = delegate; + synchronized (localScanner) { + List rowCells = new ArrayList<>(); + while (hasMoreRows) { + // Check region state INSIDE loop for long-running scans + ungroupedAggregateRegionObserver.checkForRegionClosingOrSplitting(); + rowCells.clear(); + hasMoreRows = localScanner.nextRaw(rowCells); + if (rowCells.isEmpty()) { + break; + } + + byte[] rowKey = CellUtil.cloneRow(rowCells.get(0)); + long rowSize = calculateRowSize(rowCells); + addRowToChunk(rowKey, rowCells, rowSize); + if (!forceFullRange && willExceedChunkLimits(rowSize)) { + break; + } + } + } + if (chunkStartKey == null) { + // TODO: Add throw exception here and test if it returns null ? + return false; + } + + boolean isPartial = + forceFullRange && !hasMoreRows && Bytes.compareTo(chunkEndKey, mapperRegionEndKey) < 0; + buildChunkMetadataResult(results, isPartial); + LOGGER.info("Chunk metadata being sent with startKey {}, endKey {}, forceFullRange {}", + chunkStartKey, chunkEndKey, forceFullRange); + 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(), + chunkStartKey != null ? Bytes.toStringBinary(chunkStartKey) : "null", + chunkEndKey != null ? Bytes.toStringBinary(chunkEndKey) : "null", t); + throw t; + } finally { + region.closeRegionOperation(); + } + } + + @Override + public boolean next(List result, ScannerContext scannerContext) throws IOException { + return next(result); + } + + /** + * Resets chunk state for a new chunk. Note: If this scanner was initialized with continued digest + * state, the first call to this method will NOT reset the digest, allowing us to continue hashing + * from the previous region's state. + */ + private void resetChunkState() { + chunkStartKey = null; + chunkEndKey = null; + currentChunkSize = 0; + currentChunkRowCount = 0; + if (!isUsingContinuedDigest) { + digest.reset(); + } + isUsingContinuedDigest = false; + } + + private long calculateRowSize(List cells) { + long size = 0; + for (Cell cell : cells) { + size += PrivateCellUtil.estimatedSerializedSizeOf(cell); + } + return size; + } + + private boolean willExceedChunkLimits(long rowSize) { + // Check if adding this row would exceed size limit + if (currentChunkSize + rowSize > chunkSizeBytes) { + return true; + } + return currentChunkRowCount + 1 > chunkMaxRows; + } + + /** + * Adds a row to the current chunk and updates digest + */ + private void addRowToChunk(byte[] rowKey, List cells, long rowSize) { + // Set chunk start key on first row + if (chunkStartKey == null) { + chunkStartKey = rowKey; + } + chunkEndKey = rowKey; + currentChunkSize += rowSize; + currentChunkRowCount++; + updateDigestWithRow(rowKey, cells); + } + + /** + * Updates the SHA-256 digest with data from a row. Hash includes: row key + cell family + cell + * qualifier + cell timestamp + cell type + cell value. This ensures that any difference in the + * data will result in different hashes. Optimized to avoid cloning - reads directly from cell's + * backing arrays (zero-copy). + */ + private void updateDigestWithRow(byte[] rowKey, List cells) { + digest.update(rowKey, 0, rowKey.length); + for (Cell cell : cells) { + digest.update(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()); + digest.update(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()); + long ts = cell.getTimestamp(); + digest.update((byte) (ts >>> 56)); + digest.update((byte) (ts >>> 48)); + digest.update((byte) (ts >>> 40)); + digest.update((byte) (ts >>> 32)); + digest.update((byte) (ts >>> 24)); + digest.update((byte) (ts >>> 16)); + digest.update((byte) (ts >>> 8)); + digest.update((byte) (ts)); + + digest.update(cell.getType().getCode()); + digest.update(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()); + } + } + + /** + * Encodes a SHA256Digest state to a byte array with length prefix for validation. This + * production-grade implementation adds security checks for critical deployment: - Length prefix + * for validation and extensibility - Prevents malicious large allocations - Enables detection of + * corrupted serialization + * @param digest The digest whose state should be encoded + * @return Byte array containing 4-byte length prefix + encoded state + * @throws IOException if encoding fails + */ + private byte[] encodeDigestState(SHA256Digest digest) throws IOException { + byte[] encoded = digest.getEncodedState(); + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + DataOutputStream dos = new DataOutputStream(bos); + dos.writeInt(encoded.length); + dos.write(encoded); + dos.flush(); + return bos.toByteArray(); + } + + /** + * Decodes a SHA256Digest state from a byte array. + * @param encodedState Byte array containing 4-byte length prefix + encoded state + * @return SHA256Digest restored to the saved state + * @throws IOException if state is invalid, corrupted, or security checks fail + */ + private SHA256Digest decodeDigestState(byte[] encodedState) throws IOException { + // Why always > 4 ? + if (encodedState == null) { + String regionName = region.getRegionInfo().getRegionNameAsString(); + String tableName = region.getRegionInfo().getTable().getNameAsString(); + throw new IllegalArgumentException( + String.format("Invalid encoded digest state in region %s table %s: encodedState is null", + regionName, tableName)); + } + + DataInputStream dis = new DataInputStream(new ByteArrayInputStream(encodedState)); + int stateLength = dis.readInt(); + // Prevent malicious large allocations, hash digest can never go beyond ~96 bytes + if (stateLength > MAX_SHA256_STATE_SIZE) { + String regionName = region.getRegionInfo().getRegionNameAsString(); + String tableName = region.getRegionInfo().getTable().getNameAsString(); + throw new IllegalArgumentException( + String.format("Invalid SHA256 state length in region %s table %s: %d expected <= %d", + regionName, tableName, stateLength, MAX_SHA256_STATE_SIZE)); + } + byte[] state = new byte[stateLength]; + dis.readFully(state); + return new SHA256Digest(state); + } + + /** + * Builds chunk metadata result cells and adds them to the results list. Returns a single "row" + * with multiple cells containing chunk metadata. 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 isPartial true if this is a partial chunk (region boundary reached before completion) + */ + private void buildChunkMetadataResult(List results, boolean isPartial) throws IOException { + byte[] resultRowKey = this.chunkStartKey; + results.add(PhoenixKeyValueUtil.newKeyValue(resultRowKey, CHUNK_METADATA_FAMILY, + BaseScannerRegionObserverConstants.SYNC_TABLE_END_KEY_QUALIFIER, AGG_TIMESTAMP, chunkEndKey)); + results.add(PhoenixKeyValueUtil.newKeyValue(resultRowKey, CHUNK_METADATA_FAMILY, + BaseScannerRegionObserverConstants.SYNC_TABLE_ROW_COUNT_QUALIFIER, AGG_TIMESTAMP, + Bytes.toBytes(currentChunkRowCount))); + results.add(PhoenixKeyValueUtil.newKeyValue(resultRowKey, CHUNK_METADATA_FAMILY, + BaseScannerRegionObserverConstants.SYNC_TABLE_HAS_MORE_ROWS_QUALIFIER, AGG_TIMESTAMP, + Bytes.toBytes(hasMoreRows))); + if (isPartial) { + // Partial chunk digest + SHA256Digest cloned = new SHA256Digest(digest); + byte[] digestState = encodeDigestState(cloned); + results.add(PhoenixKeyValueUtil.newKeyValue(resultRowKey, CHUNK_METADATA_FAMILY, + BaseScannerRegionObserverConstants.SYNC_TABLE_IS_PARTIAL_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 = new byte[digest.getDigestSize()]; + digest.doFinal(hash, 0); + 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_QUALIFIER, AGG_TIMESTAMP, + FALSE_BYTES)); + } + } + + @Override + public void close() throws IOException { + try { + super.close(); + } catch (Exception e) { + LOGGER.error("Error closing PhoenixSyncTableRegionScanner", e); + } + } + + // Getters for testing + @VisibleForTesting + public long getChunkSizeBytes() { + return chunkSizeBytes; + } + + @VisibleForTesting + public long getChunkMaxRows() { + return chunkMaxRows; + } +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java index 2574fd67b12..5955bce543d 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java @@ -416,6 +416,14 @@ public RegionScanner run() throws Exception { return rebuildIndices(s, region, scan, env); } }); + } else if (ScanUtil.isSyncTableChunkFormation(scan)) { + return User.runAsLoginUser(new PrivilegedExceptionAction() { + @Override + public RegionScanner run() throws Exception { + return new PhoenixSyncTableRegionScanner(s, region, scan, env, + UngroupedAggregateRegionObserver.this); + } + }); } boolean useNewValueColumnQualifier = EncodedColumnsUtil.useNewValueColumnQualifier(scan); diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixNoOpSingleRecordReader.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixNoOpSingleRecordReader.java new file mode 100644 index 00000000000..8dc8860f426 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixNoOpSingleRecordReader.java @@ -0,0 +1,113 @@ +/* + * 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.mapreduce; + +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.db.DBInputFormat; +import org.apache.hadoop.mapreduce.lib.db.DBWritable; + +/** + * A minimal RecordReader that returns exactly one dummy record per InputSplit. + *

+ * Use this when your mapper: + *

+ *

+ * This avoids the overhead of scanning and returning all rows when the mapper only needs to be + * triggered once per region/split. The standard {@link PhoenixRecordReader} iterates through all + * rows, calling {@code map()} for each row - which is wasteful when the mapper ignores the row data + * entirely. + *

+ * How it works: + *

+ * @see PhoenixSyncTableInputFormat + * @see PhoenixRecordReader + */ +public class PhoenixNoOpSingleRecordReader extends RecordReader { + + private boolean hasRecord = true; + + /** + * Initialize the RecordReader. No initialization is needed since we return a single dummy record. + * @param split The InputSplit containing region boundaries + * @param context The task context + */ + @Override + public void initialize(InputSplit split, TaskAttemptContext context) { + // No initialization needed + } + + /** + * Returns true exactly once to trigger a single map() call per split. + * @return true on first call, false on all subsequent calls + */ + @Override + public boolean nextKeyValue() { + if (hasRecord) { + hasRecord = false; + return true; + } + return false; + } + + /** + * Returns a NullWritable key (mapper ignores this). + * @return NullWritable singleton + */ + @Override + public NullWritable getCurrentKey() { + return NullWritable.get(); + } + + /** + * Returns a NullDBWritable value (mapper ignores this). The mapper extracts what it needs from + * the InputSplit, not from this value. + * @return A new NullDBWritable instance + */ + @Override + public DBWritable getCurrentValue() { + return new DBInputFormat.NullDBWritable(); + } + + /** + * Returns progress: 0.0 before the record is consumed, 1.0 after. + * @return 0.0f if record not yet consumed, 1.0f otherwise + */ + @Override + public float getProgress() { + return hasRecord ? 0.0f : 1.0f; + } + + /** + * Close the RecordReader. Nothing to close since we hold no resources. + */ + @Override + public void close() { + // Nothing to close + } +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java new file mode 100644 index 00000000000..5326ffe883f --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java @@ -0,0 +1,197 @@ +/* + * 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.mapreduce; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.phoenix.mapreduce.util.ConnectionUtil; +import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil; +import org.apache.phoenix.query.KeyRange; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * InputFormat designed for PhoenixSyncTableTool that generates splits based on HBase region + * boundaries. Filters out already-processed mapper regions using checkpoint data, enabling + * resumable sync jobs. Uses {@link PhoenixNoOpSingleRecordReader} to invoke the mapper once per + * split (region). + */ +public class PhoenixSyncTableInputFormat extends PhoenixInputFormat { + + private static final Logger LOGGER = LoggerFactory.getLogger(PhoenixSyncTableInputFormat.class); + + /** + * Instantiated by MapReduce framework + */ + public PhoenixSyncTableInputFormat() { + super(); + } + + /** + * Returns a {@link PhoenixNoOpSingleRecordReader} that emits exactly one dummy record per split. + *

+ * PhoenixSyncTableMapper doesn't need actual row data from the RecordReader - it extracts region + * boundaries from the InputSplit and delegates all scanning to the PhoenixSyncTableRegionScanner + * coprocessor. Using PhoenixNoOpSingleRecordReader ensures that {@code map()} is called exactly + * once per region no matter what scan looks like, avoiding the overhead of the default + * PhoenixRecordReader which would call {@code map()} for every row of scan. + * @param split Input Split + * @return A SingleRecordReader instance + */ + @SuppressWarnings("rawtypes") + @Override + public RecordReader createRecordReader(InputSplit split, TaskAttemptContext context) { + return new PhoenixNoOpSingleRecordReader(); + } + + /** + * Generates InputSplits for the sync job, filtering out already-completed regions using synn + * checkpoint table. + */ + @Override + public List getSplits(JobContext context) throws IOException, InterruptedException { + Configuration conf = context.getConfiguration(); + String tableName = PhoenixConfigurationUtil.getPhoenixSyncTableName(conf); + String targetZkQuorum = PhoenixConfigurationUtil.getPhoenixSyncTableTargetZkQuorum(conf); + Long fromTime = PhoenixConfigurationUtil.getPhoenixSyncTableFromTime(conf); + Long toTime = PhoenixConfigurationUtil.getPhoenixSyncTableToTime(conf); + + List allSplits = super.getSplits(context); + if (allSplits == null || allSplits.isEmpty()) { + throw new IOException(String.format( + "PhoenixInputFormat generated no splits for table %s. Check table exists and has regions.", + tableName)); + } + LOGGER.info("Total splits generated {} of table {} for PhoenixSyncTable ", allSplits.size(), + tableName); + List completedRegions = null; + try { + completedRegions = + queryCompletedMapperRegions(conf, tableName, targetZkQuorum, fromTime, toTime); + } catch (SQLException e) { + throw new RuntimeException(e); + } + if (completedRegions.isEmpty()) { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("No completed regions for table {} - processing all {} splits", tableName, + allSplits.size()); + } + return allSplits; + } + + List unprocessedSplits = filterCompletedSplits(allSplits, completedRegions); + LOGGER.info("Found {} completed mapper regions for table {}, {} unprocessed splits remaining", + completedRegions.size(), tableName, unprocessedSplits.size()); + return unprocessedSplits; + } + + /** + * Queries Sync checkpoint table for completed mapper regions + */ + private List queryCompletedMapperRegions(Configuration conf, String tableName, + String targetZkQuorum, Long fromTime, Long toTime) throws SQLException { + List completedRegions = new ArrayList<>(); + try (Connection conn = ConnectionUtil.getInputConnection(conf)) { + PhoenixSyncTableOutputRepository repository = new PhoenixSyncTableOutputRepository(conn); + List completedRows = + repository.getProcessedMapperRegions(tableName, targetZkQuorum, fromTime, toTime); + for (PhoenixSyncTableOutputRow row : completedRows) { + KeyRange keyRange = KeyRange.getKeyRange(row.getStartRowKey(), row.getEndRowKey()); + completedRegions.add(keyRange); + } + } + return completedRegions; + } + + /** + * Filters out splits that are fully contained within already completed mapper region boundary. + * @param allSplits All splits generated from region boundaries + * @param completedRegions Regions already verified (from checkpoint table) + * @return Splits that need processing + */ + private List filterCompletedSplits(List allSplits, + List completedRegions) { + allSplits.sort((s1, s2) -> { + PhoenixInputSplit ps1 = (PhoenixInputSplit) s1; + PhoenixInputSplit ps2 = (PhoenixInputSplit) s2; + return KeyRange.COMPARATOR.compare(ps1.getKeyRange(), ps2.getKeyRange()); + }); + List unprocessedSplits = new ArrayList<>(); + int splitIdx = 0; + int completedIdx = 0; + while (splitIdx < allSplits.size() && completedIdx < completedRegions.size()) { + PhoenixInputSplit split = (PhoenixInputSplit) allSplits.get(splitIdx); + KeyRange splitRange = split.getKeyRange(); + KeyRange completedRange = completedRegions.get(completedIdx); + + // Normalize boundaries (null becomes empty byte array) + byte[] splitStart = normalizeKey(splitRange.getLowerRange()); + byte[] splitEnd = normalizeKey(splitRange.getUpperRange()); + byte[] completedStart = normalizeKey(completedRange.getLowerRange()); + byte[] completedEnd = normalizeKey(completedRange.getUpperRange()); + + // Completed region ends before split starts + if (Bytes.compareTo(completedEnd, splitStart) <= 0) { + completedIdx++; + } + // Completed region starts after split ends + else if (Bytes.compareTo(completedStart, splitEnd) >= 0) { + unprocessedSplits.add(allSplits.get(splitIdx)); + splitIdx++; + } + // Overlap exists - check if split fully contained in completed + else { + // Split is fully contained if: completedStart <= splitStart AND splitEnd <= completedEnd + boolean startContained = Bytes.compareTo(completedStart, splitStart) <= 0; + boolean endContained = Bytes.compareTo(splitEnd, completedEnd) <= 0; + boolean fullyContained = startContained && endContained; + if (!fullyContained) { + // Not fully contained - keep the split + unprocessedSplits.add(allSplits.get(splitIdx)); + } + splitIdx++; + } + } + + // Add any remaining splits (if completed regions exhausted) + // These splits cannot be contained since no completed regions left to check + while (splitIdx < allSplits.size()) { + unprocessedSplits.add(allSplits.get(splitIdx)); + splitIdx++; + } + return unprocessedSplits; + } + + /** + * Normalizes a key boundary for comparison + * @param key The key to normalize + * @return Empty byte array if key is null, otherwise the key unchanged + */ + private byte[] normalizeKey(byte[] key) { + return key == null ? new byte[0] : key; + } +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java new file mode 100644 index 00000000000..f5be5dae380 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java @@ -0,0 +1,601 @@ +/* + * 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.mapreduce; + +import static org.apache.phoenix.schema.types.PDataType.TRUE_BYTES; + +import java.io.IOException; +import java.security.MessageDigest; +import java.sql.Connection; +import java.sql.SQLException; +import java.sql.Timestamp; +import java.util.*; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.lib.db.DBInputFormat; +import org.apache.phoenix.coprocessorclient.BaseScannerRegionObserverConstants; +import org.apache.phoenix.jdbc.PhoenixConnection; +import org.apache.phoenix.mapreduce.util.ConnectionUtil; +import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil; +import org.apache.phoenix.mapreduce.util.PhoenixMapReduceUtil; +import org.apache.phoenix.query.KeyRange; +import org.apache.phoenix.schema.PTable; +import org.apache.phoenix.util.MetaDataUtil; +import org.apache.phoenix.util.PhoenixRuntime; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; + +/** + * Mapper that acts as a driver for synchronizing table between source and target clusters. The + * actual work of chunking and hashing is done server-side by the coprocessor. This mapper fetches + * chunk hashes from both clusters, compares them and write to checkpoint table. + */ +public class PhoenixSyncTableMapper + extends Mapper { + + private static final Logger LOGGER = LoggerFactory.getLogger(PhoenixSyncTableMapper.class); + + public enum SyncCounters { + CHUNKS_VERIFIED, + CHUNKS_MISMATCHED, + ROWS_PROCESSED + } + + private String tableName; + private String targetZkQuorum; + private Long fromTime; + private Long toTime; + private boolean isDryRun; + private long chunkSizeBytes; + private long chunkMaxRows; + private Configuration conf; + private Connection sourceConnection; + private Connection targetConnection; + private Connection globalConnection; + private PTable pTable; + private byte[] physicalTableName; + private byte[] mapperRegionStart; + private byte[] mapperRegionEnd; + private PhoenixSyncTableOutputRepository syncTableOutputRepository; + private Timestamp mapperStartTime; + + @Override + protected void setup(Context context) throws InterruptedException { + try { + super.setup(context); + mapperStartTime = new Timestamp(System.currentTimeMillis()); + this.conf = context.getConfiguration(); + tableName = PhoenixConfigurationUtil.getPhoenixSyncTableName(conf); + targetZkQuorum = PhoenixConfigurationUtil.getPhoenixSyncTableTargetZkQuorum(conf); + fromTime = PhoenixConfigurationUtil.getPhoenixSyncTableFromTime(conf); + toTime = PhoenixConfigurationUtil.getPhoenixSyncTableToTime(conf); + isDryRun = PhoenixConfigurationUtil.getPhoenixSyncTableDryRun(conf); + chunkSizeBytes = PhoenixConfigurationUtil.getPhoenixSyncTableChunkSizeBytes(conf); + chunkMaxRows = PhoenixConfigurationUtil.getPhoenixSyncTableChunkMaxRows(conf); + extractRegionBoundariesFromSplit(context); + sourceConnection = ConnectionUtil.getInputConnection(conf); + pTable = sourceConnection.unwrap(PhoenixConnection.class).getTable(tableName); + physicalTableName = pTable.getPhysicalName().getBytes(); + connectToTargetCluster(); + globalConnection = createGlobalConnection(conf); + syncTableOutputRepository = new PhoenixSyncTableOutputRepository(globalConnection); + } catch (SQLException | IOException e) { + tryClosingResources(); + throw new RuntimeException(String.format("Failed to setup mapper for table: %s", tableName), + e); + } + } + + /** + * Extracts mapper region boundaries from the PhoenixInputSplit + */ + private void extractRegionBoundariesFromSplit(Context context) { + PhoenixInputSplit split = (PhoenixInputSplit) context.getInputSplit(); + KeyRange keyRange = split.getKeyRange(); + if (keyRange == null) { + throw new IllegalStateException(String.format( + "PhoenixInputSplit has no KeyRange for table: %s . Cannot determine region boundaries for sync operation.", + tableName)); + } + mapperRegionStart = keyRange.getLowerRange(); + mapperRegionEnd = keyRange.getUpperRange(); + } + + /** + * Connects to the target cluster using the target ZK quorum, port, znode + */ + private void connectToTargetCluster() throws SQLException, IOException { + Configuration targetConf = + PhoenixMapReduceUtil.createConfigurationForZkQuorum(conf, targetZkQuorum); + if (toTime != null) { + targetConf.set(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(toTime)); + } + targetConnection = ConnectionUtil.getInputConnection(targetConf); + } + + /** + * Creates a global (non-tenant) connection for the checkpoint table. + */ + private Connection createGlobalConnection(Configuration conf) throws SQLException { + Configuration globalConf = new Configuration(conf); + globalConf.unset(PhoenixConfigurationUtil.MAPREDUCE_TENANT_ID); + globalConf.unset(PhoenixRuntime.CURRENT_SCN_ATTRIB); + return ConnectionUtil.getInputConnection(globalConf); + } + + /** + * Processes a mapper region by comparing chunks between source and target clusters. Gets already + * processed chunks from checkpoint table, resumes from checkpointed progress and records final + * status for chunks & mapper (VERIFIED/MISMATCHED). + */ + @Override + protected void map(NullWritable key, DBInputFormat.NullDBWritable value, Context context) + throws IOException, InterruptedException { + LOGGER.info("Processing mapper region: {} to {}", Bytes.toStringBinary(mapperRegionStart), + Bytes.toStringBinary(mapperRegionEnd)); + context.getCounter(PhoenixJobCounters.INPUT_RECORDS).increment(1); + try { + List processedChunks = + syncTableOutputRepository.getProcessedChunks(tableName, targetZkQuorum, fromTime, toTime, + mapperRegionStart, mapperRegionEnd); + List> unprocessedRanges = + calculateUnprocessedRanges(mapperRegionStart, mapperRegionEnd, processedChunks); + // TODO: Test with delete where mapper region boundary key is not present in region + for (Pair range : unprocessedRanges) { + processChunkRange(range.getFirst(), range.getSecond(), context); + } + + long mismatchedChunk = context.getCounter(SyncCounters.CHUNKS_MISMATCHED).getValue(); + long verifiedChunk = context.getCounter(SyncCounters.CHUNKS_VERIFIED).getValue(); + long rowsProcessed = context.getCounter(SyncCounters.ROWS_PROCESSED).getValue(); + Timestamp mapperEndTime = new Timestamp(System.currentTimeMillis()); + Map mapperCounters = new LinkedHashMap<>(); + mapperCounters.put(SyncCounters.CHUNKS_VERIFIED.name(), verifiedChunk); + mapperCounters.put(SyncCounters.CHUNKS_MISMATCHED.name(), mismatchedChunk); + mapperCounters.put(SyncCounters.CHUNKS_MISMATCHED.name(), rowsProcessed); + String counters = formatCounters(mapperCounters); + + if (mismatchedChunk == 0) { + context.getCounter(PhoenixJobCounters.OUTPUT_RECORDS).increment(1); + syncTableOutputRepository.checkpointSyncTableResult(tableName, targetZkQuorum, + PhoenixSyncTableOutputRow.Type.MAPPER_REGION, fromTime, toTime, isDryRun, + mapperRegionStart, mapperRegionEnd, PhoenixSyncTableOutputRow.Status.VERIFIED, + mapperStartTime, mapperEndTime, counters); + LOGGER.info( + "Mapper completed with verified: {} verifiedChunk chunks, {} mismatchedChunk chunks", + verifiedChunk, mismatchedChunk); + } else { + context.getCounter(PhoenixJobCounters.FAILED_RECORDS).increment(1); + LOGGER.warn( + "Mapper completed with mismatched: {} mismatchedChunk chunks, {} verifiedChunk chunks", + verifiedChunk, mismatchedChunk); + syncTableOutputRepository.checkpointSyncTableResult(tableName, targetZkQuorum, + PhoenixSyncTableOutputRow.Type.MAPPER_REGION, fromTime, toTime, isDryRun, + mapperRegionStart, mapperRegionEnd, PhoenixSyncTableOutputRow.Status.MISMATCHED, + mapperStartTime, mapperEndTime, counters); + } + } catch (SQLException e) { + tryClosingResources(); + throw new RuntimeException("Error processing mapper", e); + } + } + + /** + * Processes a chunk range by comparing source and target cluster data. Source chunking: Breaks + * data into size-based chunks within each physical region. Creates new scanner and new chunk when + * region boundary is crossed. This can happen if mapper region splits while the mapper job is + * running. Target chunking: Follows source chunk boundaries exactly. Source chunk boundary might + * be split across multiple target region, if so corpoc signal for partial chunk with partial + * digest. Once entire Source chunk is covered by target scanner, we calculate resulting checksum + * from combined digest. + * @param rangeStart Range start key + * @param rangeEnd Range end key + * @param context Mapper context for progress and counters + * @throws IOException if scan fails + * @throws SQLException if database operations fail + */ + private void processChunkRange(byte[] rangeStart, byte[] rangeEnd, Context context) + throws IOException, SQLException { + byte[] currentStartKey = rangeStart; + boolean isEntireMapperRegionChunked = false; + boolean isStartKeyInclusive = true; + while (!isEntireMapperRegionChunked) { + boolean isRegionChunked = false; + try (ChunkScannerContext sourceScanner = createChunkScanner(sourceConnection, currentStartKey, + rangeEnd, false, null, isStartKeyInclusive, false)) { + while (!isRegionChunked) { + ChunkInfo sourceChunk = sourceScanner.getNextChunk(); + if (sourceChunk == null) { + isEntireMapperRegionChunked = true; + break; + } + Timestamp chunkStartTime = new Timestamp(System.currentTimeMillis()); + sourceChunk.executionStartTime = chunkStartTime; + ChunkInfo targetChunk = getTargetChunkWithSourceBoundary(targetConnection, + sourceChunk.startKey, sourceChunk.endKey); + context.getCounter(SyncCounters.ROWS_PROCESSED).increment(sourceChunk.rowCount); + boolean matched = MessageDigest.isEqual(sourceChunk.hash, targetChunk.hash); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Chunk comparison {}, {}: source={} rows, target={} rows, matched={}", + Bytes.toStringBinary(sourceChunk.startKey), Bytes.toStringBinary(sourceChunk.endKey), + sourceChunk.rowCount, targetChunk.rowCount, matched); + } + + if (matched) { + handleVerifiedChunk(sourceChunk, context); + } else { + handleMismatchedChunk(sourceChunk, context); + } + + context.progress(); + currentStartKey = sourceChunk.endKey; + isStartKeyInclusive = false; + if (!sourceChunk.hasMoreRows) { + isRegionChunked = true; + } + } + } + } + + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Completed processing range {}, {}", Bytes.toStringBinary(rangeStart), + Bytes.toStringBinary(rangeEnd)); + } + } + + /** + * Scans target across multiple regions and returns a single combined ChunkInfo. Handles partial + * chunks by passing digest state to next scanner via scan attributes, enabling cross-region + * digest continuation. + * @param conn Target connection + * @param startKey Source chunk start key + * @param endKey Source chunk end key + * @return Single ChunkInfo with final hash from all target regions + */ + private ChunkInfo getTargetChunkWithSourceBoundary(Connection conn, byte[] startKey, + byte[] endKey) throws IOException, SQLException { + ChunkInfo targetChunk = new ChunkInfo(); + targetChunk.startKey = startKey; + targetChunk.endKey = endKey; + targetChunk.hash = new byte[0]; + targetChunk.rowCount = 0; + targetChunk.isPartial = false; + + byte[] currentStartKey = startKey; + byte[] continuedDigestState = null; + boolean isStartKeyInclusive = true; + while (currentStartKey != null && Bytes.compareTo(currentStartKey, endKey) < 0) { + try (ChunkScannerContext scanner = createChunkScanner(conn, currentStartKey, endKey, true, + continuedDigestState, isStartKeyInclusive, true)) { + ChunkInfo chunk = scanner.getNextChunk(); + if (chunk == null) { + break; + } + targetChunk.rowCount += chunk.rowCount; + if (chunk.isPartial) { + continuedDigestState = chunk.hash; + currentStartKey = chunk.endKey; + isStartKeyInclusive = false; + } else { + targetChunk.hash = chunk.hash; + currentStartKey = null; + } + } + } + return targetChunk; + } + + /** + * Creates a reusable scanner context for fetching chunks from a range. + * @param conn Connection to cluster (source or target) + * @param startKey Range start key (inclusive) + * @param endKey Range end key (exclusive) + * @param forceFullRange If true, coprocessor returns entire range without size-based + * breaking (for target cluster) + * @param continuedDigestState If not null, coprocessor will continue hashing from this state (for + * cross-region continuation on target) + * @param isStartKeyInclusive Whether StartKey Inclusive + * @param isEndKeyInclusive Whether EndKey Inclusive + * @return ChunkScannerContext that must be closed + * @throws IOException if scanner creation fails + * @throws SQLException if connection fails + */ + private ChunkScannerContext createChunkScanner(Connection conn, byte[] startKey, byte[] endKey, + boolean forceFullRange, byte[] continuedDigestState, boolean isStartKeyInclusive, + boolean isEndKeyInclusive) throws IOException, SQLException { + // Not using try-with-resources since ChunkScannerContext owns the table lifecycle + Table hTable = + conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(physicalTableName); + Scan scan = createChunkScan(startKey, endKey, isStartKeyInclusive, isEndKeyInclusive); + scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CHUNK_FORMATION, TRUE_BYTES); + scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_MAPPER_REGION_END_KEY, endKey); + scan.setAttribute(BaseScannerRegionObserverConstants.SKIP_REGION_BOUNDARY_CHECK, TRUE_BYTES); + scan.setAttribute(BaseScannerRegionObserverConstants.UNGROUPED_AGG, TRUE_BYTES); + if (continuedDigestState != null && continuedDigestState.length > 0) { + scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CONTINUED_DIGEST_STATE, + continuedDigestState); + LOGGER.info("Creating scanner with continued digest state ({} bytes) for range [{}, {})", + continuedDigestState.length, Bytes.toStringBinary(startKey), Bytes.toStringBinary(endKey)); + } + + if (forceFullRange) { + scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_FORCE_FULL_RANGE, TRUE_BYTES); + } else { + scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CHUNK_SIZE_BYTES, + Bytes.toBytes(chunkSizeBytes)); + scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CHUNK_MAX_ROWS, + Bytes.toBytes(chunkMaxRows)); + } + ResultScanner scanner = hTable.getScanner(scan); + return new ChunkScannerContext(hTable, scanner); + } + + /** + * Parses chunk information from the coprocessor result. The PhoenixSyncTableRegionScanner returns + * cells with chunk metadata including SHA-256 hash (for complete chunks) or MessageDigest state + * (for partial chunks). + */ + private ChunkInfo parseChunkInfo(Result result) { + List cells = Arrays.asList(result.rawCells()); + // Note: startKey is in the row key, not as a separate cell (optimization to avoid duplication) + Cell endKeyCell = + MetaDataUtil.getCell(cells, BaseScannerRegionObserverConstants.SYNC_TABLE_END_KEY_QUALIFIER); + Cell rowCountCell = MetaDataUtil.getCell(cells, + BaseScannerRegionObserverConstants.SYNC_TABLE_ROW_COUNT_QUALIFIER); + Cell isPartialCell = MetaDataUtil.getCell(cells, + BaseScannerRegionObserverConstants.SYNC_TABLE_IS_PARTIAL_QUALIFIER); + Cell hashCell = + MetaDataUtil.getCell(cells, BaseScannerRegionObserverConstants.SYNC_TABLE_HASH_QUALIFIER); + Cell hasMoreRowsCell = MetaDataUtil.getCell(cells, + BaseScannerRegionObserverConstants.SYNC_TABLE_HAS_MORE_ROWS_QUALIFIER); + + if ( + endKeyCell == null || rowCountCell == null || isPartialCell == null || hashCell == null + || hasMoreRowsCell == null + ) { + throw new RuntimeException("Missing required chunk metadata cells."); + } + + ChunkInfo info = new ChunkInfo(); + info.startKey = result.getRow(); // Read from row key instead of cell value + info.endKey = CellUtil.cloneValue(endKeyCell); + info.rowCount = Bytes.toLong(rowCountCell.getValueArray(), rowCountCell.getValueOffset(), + rowCountCell.getValueLength()); + info.isPartial = isPartialCell.getValueArray()[isPartialCell.getValueOffset()] != 0; + info.hash = CellUtil.cloneValue(hashCell); + info.hasMoreRows = hasMoreRowsCell.getValueArray()[hasMoreRowsCell.getValueOffset()] != 0; + return info; + } + + /** + * Formats counters as a comma-separated key=value string. Example: + * "CHUNKS_VERIFIED=10,CHUNKS_MISMATCHED=2,ROWS_PROCESSED=5678" + * @param counters Map of counter names to values + * @return Formatted string or null if counters is null/empty + */ + private String formatCounters(Map counters) { + if (counters == null || counters.isEmpty()) { + return null; + } + StringBuilder sb = new StringBuilder(); + boolean first = true; + for (Map.Entry entry : counters.entrySet()) { + if (!first) { + sb.append(","); + } + sb.append(entry.getKey()).append("=").append(entry.getValue()); + first = false; + } + return sb.toString(); + } + + /*** + * + */ + private void handleVerifiedChunk(ChunkInfo chunk, Context context) throws SQLException { + Timestamp chunkEndTime = new Timestamp(System.currentTimeMillis()); + String counters = "ROWS_PROCESSED=" + chunk.rowCount; + syncTableOutputRepository.checkpointSyncTableResult(tableName, targetZkQuorum, + PhoenixSyncTableOutputRow.Type.CHUNK, fromTime, toTime, isDryRun, chunk.startKey, + chunk.endKey, PhoenixSyncTableOutputRow.Status.VERIFIED, chunk.executionStartTime, + chunkEndTime, counters); + context.getCounter(SyncCounters.CHUNKS_VERIFIED).increment(1); + } + + /*** + * + */ + private void handleMismatchedChunk(ChunkInfo sourceChunk, Context context) throws SQLException { + Timestamp chunkEndTime = new Timestamp(System.currentTimeMillis()); + LOGGER.warn("Chunk mismatch detected: {} to {}", Bytes.toStringBinary(sourceChunk.startKey), + Bytes.toStringBinary(sourceChunk.endKey)); + String counters = "ROWS_PROCESSED=" + sourceChunk.rowCount; + syncTableOutputRepository.checkpointSyncTableResult(tableName, targetZkQuorum, + PhoenixSyncTableOutputRow.Type.CHUNK, fromTime, toTime, isDryRun, sourceChunk.startKey, + sourceChunk.endKey, PhoenixSyncTableOutputRow.Status.MISMATCHED, + sourceChunk.executionStartTime, chunkEndTime, counters); + context.getCounter(SyncCounters.CHUNKS_MISMATCHED).increment(1); + } + + /** + * Creates a Hbase raw scan for a chunk range to capture all cell versions and delete markers. + */ + private Scan createChunkScan(byte[] startKey, byte[] endKey, boolean isStartKeyInclusive, + boolean isEndKeyInclusive) throws IOException { + Scan scan = new Scan(); + scan.withStartRow(startKey, isStartKeyInclusive); + scan.withStopRow(endKey, isEndKeyInclusive); + scan.setRaw(true); + scan.readAllVersions(); + scan.setCacheBlocks(false); + if (fromTime != null || toTime != null) { + scan.setTimeRange(fromTime != null ? fromTime : 0L, toTime != null ? toTime : Long.MAX_VALUE); + } + return scan; + } + + /** + * Calculates unprocessed chunk ranges (gaps) within a mapper region. Given a list of processed + * chunks, returns the ranges that haven't been processed yet. + * @param mapperRegionStart Start of mapper region + * @param mapperRegionEnd End of mapper region + * @param processedChunks List of already-processed chunks from getProcessedChunks() + * @return List of (startKey, endKey) pairs representing unprocessed ranges + */ + @VisibleForTesting + public List> calculateUnprocessedRanges(byte[] mapperRegionStart, + byte[] mapperRegionEnd, List processedChunks) { + List> gaps = new ArrayList<>(); + if (processedChunks == null || processedChunks.isEmpty()) { + gaps.add(new Pair<>(mapperRegionStart, mapperRegionEnd)); + return gaps; + } + + byte[] currentPos = mapperRegionStart; + for (PhoenixSyncTableOutputRow chunk : processedChunks) { + byte[] chunkStart = chunk.getStartRowKey(); + byte[] chunkEnd = chunk.getEndRowKey(); + // Clip chunk boundaries to mapper region boundaries + // If Mapper region boundary is [20,85) and already processed chunked rows boundaries are + // [10,30] and [70,80] + // When we start mapper, it would identify already processed ranges as [10,30] and [70,80] + // and calculate unprocessed ranges as [30,70) and [80,85). It would then do further chunking + // on these two new boundaries. + byte[] effectiveStart = + Bytes.compareTo(chunkStart, mapperRegionStart) > 0 ? chunkStart : mapperRegionStart; + byte[] effectiveEnd = + Bytes.compareTo(chunkEnd, mapperRegionEnd) < 0 ? chunkEnd : mapperRegionEnd; + // Check for gap before this chunk's effective start + if (Bytes.compareTo(currentPos, effectiveStart) < 0) { + gaps.add(new Pair<>(currentPos, effectiveStart)); + } + currentPos = effectiveEnd; + } + if (Bytes.compareTo(currentPos, mapperRegionEnd) < 0) { + gaps.add(new Pair<>(currentPos, mapperRegionEnd)); + } + return gaps; + } + + @Override + protected void cleanup(Context context) throws IOException, InterruptedException { + tryClosingResources(); + super.cleanup(context); + LOGGER.info("Mapper cleanup complete - Chunk Verified: {}, Mismatched: {}", + context.getCounter(SyncCounters.CHUNKS_VERIFIED).getValue(), + context.getCounter(SyncCounters.CHUNKS_MISMATCHED).getValue()); + } + + private void tryClosingResources() { + if (sourceConnection != null) { + try { + sourceConnection.close(); + LOGGER.info("Source connection closed"); + } catch (SQLException e) { + LOGGER.error("Error while closing source connection in PhoenixSyncTableMapper", e); + } + } + if (targetConnection != null) { + try { + targetConnection.close(); + LOGGER.info("Target connection closed"); + } catch (SQLException e) { + LOGGER.error("Error while closing target connection in PhoenixSyncTableMapper", e); + } + } + if (globalConnection != null) { + try { + globalConnection.close(); + LOGGER.info("Output connection closed"); + } catch (SQLException e) { + LOGGER.error("Error while closing output connection in PhoenixSyncTableMapper", e); + } + } + } + + /** + * Hold chunk metadata returned from coprocessor + */ + private static class ChunkInfo { + byte[] startKey; + byte[] endKey; + byte[] hash; + long rowCount; + boolean isPartial; + boolean hasMoreRows; + Timestamp executionStartTime; + + @Override + public String toString() { + return String.format("Chunk[start=%s, end=%s, rows=%d, partial=%s, hasMoreRows=%s]", + Bytes.toStringBinary(startKey), Bytes.toStringBinary(endKey), rowCount, isPartial, + hasMoreRows); + } + } + + /** + * Holds a ResultScanner and Table reference for reuse across multiple chunks. + */ + private class ChunkScannerContext implements AutoCloseable { + private final Table table; + private final ResultScanner scanner; + + ChunkScannerContext(Table table, ResultScanner scanner) { + this.table = table; + this.scanner = scanner; + } + + /** + * Fetches the next chunk from the scanner. Each call retrieves one chunk's metadata from the + * server-side coprocessor. + * @return ChunkInfo or null if no more chunks available for region + * @throws IOException if scan fails + */ + ChunkInfo getNextChunk() throws IOException { + Result result = scanner.next(); + if (result == null || result.isEmpty()) { + return null; + } + return parseChunkInfo(result); + } + + @Override + public void close() throws IOException { + try { + if (scanner != null) { + scanner.close(); + } + } finally { + if (table != null) { + table.close(); + } + } + } + } +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java new file mode 100644 index 00000000000..f1cc51435ec --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java @@ -0,0 +1,295 @@ +/* + * 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.mapreduce; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import org.apache.hadoop.hbase.HConstants; +import org.apache.phoenix.mapreduce.PhoenixSyncTableOutputRow.Status; +import org.apache.phoenix.mapreduce.PhoenixSyncTableOutputRow.Type; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Repository for managing the PHOENIX_SYNC_TABLE_OUTPUT table. This table stores checkpoint + * information for the PhoenixSyncTableTool, enabling: 1. Job-level checkpointing (skip completed + * mapper regions on restart) 2. Mapper-level checkpointing (skip completed chunks within a region) + * 3. Audit trail of all sync operations + */ +public class PhoenixSyncTableOutputRepository { + + private static final Logger LOGGER = + LoggerFactory.getLogger(PhoenixSyncTableOutputRepository.class); + public static final String SYNC_TABLE_CHECKPOINT_TABLE_NAME = "PHOENIX_SYNC_TABLE_CHECKPOINT"; + private static final int OUTPUT_TABLE_TTL_SECONDS = 30 * 24 * 60 * 60; + private final Connection connection; + private static final byte[] EMPTY_START_KEY_SENTINEL = new byte[] { 0x00 }; + + /** + * Constructor + * @param connection Phoenix connection + */ + public PhoenixSyncTableOutputRepository(Connection connection) { + this.connection = connection; + } + + /** + * Creates the PHOENIX_SYNC_TABLE_OUTPUT table if it doesn't exist. Table schema: - Primary key: + * (TABLE_NAME, TARGET_CLUSTER, FROM_TIME, TO_TIME, ENTRY_TYPE, START_ROW_KEY, END_ROW_KEY) - TTL: + * 30 days - Salt buckets: 4 (for better distribution) + */ + public void createOutputTableIfNotExists() throws SQLException { + String ddl = "CREATE TABLE IF NOT EXISTS " + SYNC_TABLE_CHECKPOINT_TABLE_NAME + " (\n" + + " TABLE_NAME VARCHAR NOT NULL,\n" + " TARGET_CLUSTER VARCHAR NOT NULL,\n" + + " ENTRY_TYPE VARCHAR(20) NOT NULL,\n" + " FROM_TIME BIGINT NOT NULL,\n" + + " TO_TIME BIGINT NOT NULL,\n" + " IS_DRY_RUN BOOLEAN NOT NULL,\n" + + " START_ROW_KEY VARBINARY NOT NULL,\n" + " END_ROW_KEY VARBINARY,\n" + + " IS_FIRST_REGION BOOLEAN, \n" + " EXECUTION_START_TIME TIMESTAMP,\n" + + " EXECUTION_END_TIME TIMESTAMP,\n" + " STATUS VARCHAR(20),\n" + + " COUNTERS VARCHAR(255), \n" + " CONSTRAINT PK PRIMARY KEY (\n" + + " TABLE_NAME,\n" + " TARGET_CLUSTER,\n" + " ENTRY_TYPE ,\n" + + " FROM_TIME,\n" + " TO_TIME,\n" + " IS_DRY_RUN,\n" + + " START_ROW_KEY )" + ") TTL=" + OUTPUT_TABLE_TTL_SECONDS; + + try (Statement stmt = connection.createStatement()) { + stmt.execute(ddl); + connection.commit(); + LOGGER.info("Successfully created or verified existence of {} table", + SYNC_TABLE_CHECKPOINT_TABLE_NAME); + } + } + + /** + * Logs a chunk-level sync result to the output table. + * @param tableName Source table name + * @param targetCluster Target cluster ZK quorum + * @param fromTime Start timestamp for sync (nullable) + * @param toTime End timestamp for sync (nullable) + * @param isDryRun Whether this is a dry run + * @param startKey Chunk start row key + * @param endKey Chunk end row key + * @param status Sync status (IN_PROGRESS/VERIFIED) + * @param executionStartTime When chunk processing started + * @param executionEndTime When chunk processing completed + */ + // public void logChunkResult(String tableName, String targetCluster, Long fromTime, Long toTime, + // boolean isDryRun, byte[] startKey, byte[] endKey, Status status, + // Timestamp executionStartTime, Timestamp executionEndTime) + // throws SQLException { + // + // checkpointSyncTableResult(tableName, targetCluster, Type.CHUNK, fromTime, toTime, isDryRun, + // startKey, endKey, status, executionStartTime, executionEndTime); + // } + + /** + * Logs a mapper region completion to the output table. This indicates that all chunks within the + * region have been processed. + * @param tableName Source table name + * @param targetCluster Target cluster ZK quorum + * @param fromTime Start timestamp for sync (nullable) + * @param toTime End timestamp for sync (nullable) + * @param isDryRun Whether this is a dry run + * @param regionStart Mapper region start row key + * @param regionEnd Mapper region end row key + * @param status Overall status for the region + * @param executionStartTime When mapper started processing this region + * @param executionEndTime When mapper finished processing this region + */ + // public void logMapperRegionResult(String tableName, String targetCluster, Long fromTime, + // Long toTime, boolean isDryRun, byte[] regionStart, byte[] regionEnd, Status status, + // Timestamp executionStartTime, Timestamp executionEndTime) + // throws SQLException { + // + // checkpointSyncTableResult(tableName, targetCluster, Type.MAPPER_REGION, fromTime, toTime, + // isDryRun, + // regionStart, regionEnd, status, executionStartTime, executionEndTime); + // } + + /** + * Internal method to log sync results to the output table + */ + public void checkpointSyncTableResult(String tableName, String targetCluster, Type type, + Long fromTime, Long toTime, boolean isDryRun, byte[] startKey, byte[] endKey, Status status, + Timestamp executionStartTime, Timestamp executionEndTime, String counters) throws SQLException { + + // Validate required parameters + if (tableName == null || tableName.isEmpty()) { + throw new IllegalArgumentException("TableName cannot be null or empty for checkpoint"); + } + if (targetCluster == null || targetCluster.isEmpty()) { + throw new IllegalArgumentException("TargetCluster cannot be null or empty for checkpoint"); + } + if (type == null) { + throw new IllegalArgumentException("Type cannot be null for checkpoint"); + } + if (fromTime == null || toTime == null) { + throw new IllegalArgumentException("FromTime and ToTime cannot be null for checkpoint"); + } + + String upsert = "UPSERT INTO " + SYNC_TABLE_CHECKPOINT_TABLE_NAME + " (" + + "TABLE_NAME, TARGET_CLUSTER, ENTRY_TYPE, FROM_TIME, TO_TIME, IS_DRY_RUN, " + + "START_ROW_KEY, END_ROW_KEY, IS_FIRST_REGION, EXECUTION_START_TIME, EXECUTION_END_TIME, " + + "STATUS, COUNTERS) " + "VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)"; + + byte[] effectiveStartKey = + (startKey == null || startKey.length == 0) ? EMPTY_START_KEY_SENTINEL : startKey; + + // Determine if this is the first region (empty start key) + boolean isFirstRegion = (startKey == null || startKey.length == 0); + + try (PreparedStatement ps = connection.prepareStatement(upsert)) { + ps.setString(1, tableName); + ps.setString(2, targetCluster); + ps.setString(3, type.name()); + ps.setObject(4, fromTime); + ps.setObject(5, toTime); + ps.setBoolean(6, isDryRun); + ps.setBytes(7, effectiveStartKey); + ps.setBytes(8, endKey); + ps.setBoolean(9, isFirstRegion); + ps.setTimestamp(10, executionStartTime); + ps.setTimestamp(11, executionEndTime); + ps.setString(12, status != null ? status.name() : null); + ps.setString(13, counters); + ps.executeUpdate(); + connection.commit(); + } + } + + /** + * Helper to check if a key represents an empty boundary + */ + // private boolean isEmptyBoundary(byte[] key) { + // return key != null && key.length == 1 && Bytes.equals(key, HConstants.EMPTY_BYTE_ARRAY); + // } + + /** + * Converts stored key back to HBase empty key if needed. For first region, converts + * EMPTY_START_KEY_SENTINEL back to HConstants.EMPTY_BYTE_ARRAY. + */ + private byte[] toHBaseKey(byte[] storedKey, boolean isFirstRegion) { + if (isFirstRegion && Arrays.equals(storedKey, EMPTY_START_KEY_SENTINEL)) { + return HConstants.EMPTY_BYTE_ARRAY; + } + return storedKey; + } + + /** + * Queries for completed mapper regions. Used by PhoenixSyncTableInputFormat to filter out + * already-processed regions. + * @param tableName Source table name + * @param targetCluster Target cluster ZK quorum + * @param fromTime Start timestamp (nullable) + * @param toTime End timestamp (nullable) + * @return List of completed mapper regions + */ + public List getProcessedMapperRegions(String tableName, + String targetCluster, Long fromTime, Long toTime) throws SQLException { + + String query = "SELECT START_ROW_KEY, END_ROW_KEY, IS_FIRST_REGION FROM " + + SYNC_TABLE_CHECKPOINT_TABLE_NAME + " WHERE TABLE_NAME = ? AND TARGET_CLUSTER = ?" + + " AND ENTRY_TYPE = ? AND FROM_TIME = ? AND TO_TIME = ? AND STATUS IN ( ?, ?)" + + " ORDER BY START_ROW_KEY "; + List results = new ArrayList<>(); + + try (PreparedStatement ps = connection.prepareStatement(query)) { + int paramIndex = 1; + ps.setString(paramIndex++, tableName); + ps.setString(paramIndex++, targetCluster); + ps.setString(paramIndex++, Type.MAPPER_REGION.name()); + ps.setLong(paramIndex++, fromTime); + ps.setLong(paramIndex++, toTime); + ps.setString(paramIndex++, Status.VERIFIED.name()); + ps.setString(paramIndex, Status.MISMATCHED.name()); + try (ResultSet rs = ps.executeQuery()) { + while (rs.next()) { + boolean isFirstRegion = rs.getBoolean("IS_FIRST_REGION"); + PhoenixSyncTableOutputRow row = new PhoenixSyncTableOutputRow.Builder() + .setStartRowKey(this.toHBaseKey(rs.getBytes("START_ROW_KEY"), isFirstRegion)) + .setEndRowKey(rs.getBytes("END_ROW_KEY")).build(); + results.add(row); + } + } + } + return results; + } + + /** + * Queries for processed chunks within a mapper region. Used by PhoenixSyncTableMapper to skip + * already-processed chunks. + * @param tableName Source table name + * @param targetCluster Target cluster ZK quorum + * @param fromTime Start timestamp (nullable) + * @param toTime End timestamp (nullable) + * @param mapperRegionStart Mapper region start key + * @param mapperRegionEnd Mapper region end key + * @return List of processed chunks in the region + */ + public List getProcessedChunks(String tableName, String targetCluster, + Long fromTime, Long toTime, byte[] mapperRegionStart, byte[] mapperRegionEnd) + throws SQLException { + String query = "SELECT START_ROW_KEY, END_ROW_KEY, IS_FIRST_REGION FROM " + + SYNC_TABLE_CHECKPOINT_TABLE_NAME + " WHERE TABLE_NAME = ? " + " AND TARGET_CLUSTER = ? " + + " " + " AND ENTRY_TYPE = ? AND FROM_TIME = ? AND TO_TIME = ? AND START_ROW_KEY < ? " + + " AND END_ROW_KEY > ? AND STATUS IN (?, ?)" + " ORDER BY START_ROW_KEY"; + + List results = new ArrayList<>(); + try (PreparedStatement ps = connection.prepareStatement(query)) { + int paramIndex = 1; + ps.setString(paramIndex++, tableName); + ps.setString(paramIndex++, targetCluster); + ps.setString(paramIndex++, Type.CHUNK.name()); + ps.setLong(paramIndex++, fromTime); + ps.setLong(paramIndex++, toTime); + ps.setBytes(paramIndex++, mapperRegionEnd); + ps.setBytes(paramIndex++, mapperRegionStart); + ps.setString(paramIndex++, Status.VERIFIED.name()); + ps.setString(paramIndex, Status.MISMATCHED.name()); + try (ResultSet rs = ps.executeQuery()) { + while (rs.next()) { + PhoenixSyncTableOutputRow row = new PhoenixSyncTableOutputRow.Builder() + .setStartRowKey( + this.toHBaseKey(rs.getBytes("START_ROW_KEY"), rs.getBoolean("IS_FIRST_REGION"))) + .setEndRowKey(rs.getBytes("END_ROW_KEY")).build(); + results.add(row); + } + } + } + return results; + } + + // /** + // * For testing: clears all entries for a given table + // */ + // @VisibleForTesting + // public void deleteEntriesForTable(String tableName) throws SQLException { + // String delete = "DELETE FROM " + OUTPUT_TABLE_NAME + " WHERE TABLE_NAME = ?"; + // try (PreparedStatement ps = connection.prepareStatement(delete)) { + // ps.setString(1, tableName); + // ps.executeUpdate(); + // connection.commit(); + // } + // } +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java new file mode 100644 index 00000000000..ad71d3f910d --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java @@ -0,0 +1,83 @@ +/* + * 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.mapreduce; + +import java.util.Arrays; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Data model class representing a row in the PHOENIX_SYNC_TABLE_OUTPUT table. Used for + * checkpointing sync job progress at both mapper region and chunk levels. + */ +public class PhoenixSyncTableOutputRow { + + public enum Type { + CHUNK, + MAPPER_REGION + } + + public enum Status { + VERIFIED, + MISMATCHED + } + + private byte[] startRowKey; + private byte[] endRowKey; + + @Override + public String toString() { + return String.format("SyncOutputRow[start=%s, end=%s]", Bytes.toStringBinary(startRowKey), + Bytes.toStringBinary(endRowKey)); + } + + public byte[] getStartRowKey() { + return startRowKey; + } + + public byte[] getEndRowKey() { + return endRowKey; + } + + /** + * Builder for PhoenixSyncTableOutputRow + */ + public static class Builder { + private final PhoenixSyncTableOutputRow row; + + public Builder() { + this.row = new PhoenixSyncTableOutputRow(); + } + + public Builder setStartRowKey(byte[] startRowKey) { + row.startRowKey = startRowKey != null ? Arrays.copyOf(startRowKey, startRowKey.length) : null; + return this; + } + + public Builder setEndRowKey(byte[] endRowKey) { + row.endRowKey = endRowKey != null ? Arrays.copyOf(endRowKey, endRowKey.length) : null; + return this; + } + + public PhoenixSyncTableOutputRow build() { + if (row.startRowKey == null) { + throw new IllegalStateException("Start row key is required"); + } + return row; + } + } +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java new file mode 100644 index 00000000000..b581326129a --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java @@ -0,0 +1,439 @@ +/* + * 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.mapreduce; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.SQLException; +import java.util.Arrays; +import java.util.Properties; +import org.apache.commons.lang3.exception.ExceptionUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; +import org.apache.hadoop.mapreduce.Counters; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.hadoop.mapreduce.lib.db.DBInputFormat; +import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; +import org.apache.hadoop.util.Tool; +import org.apache.hadoop.util.ToolRunner; +import org.apache.phoenix.coprocessor.PhoenixSyncTableRegionScanner; +import org.apache.phoenix.mapreduce.util.ConnectionUtil; +import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil; +import org.apache.phoenix.mapreduce.util.PhoenixMapReduceUtil; +import org.apache.phoenix.query.QueryServices; +import org.apache.phoenix.query.QueryServicesOptions; +import org.apache.phoenix.schema.PTable; +import org.apache.phoenix.schema.PTableType; +import org.apache.phoenix.util.EnvironmentEdgeManager; +import org.apache.phoenix.util.SchemaUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.phoenix.thirdparty.org.apache.commons.cli.CommandLine; +import org.apache.phoenix.thirdparty.org.apache.commons.cli.CommandLineParser; +import org.apache.phoenix.thirdparty.org.apache.commons.cli.DefaultParser; +import org.apache.phoenix.thirdparty.org.apache.commons.cli.HelpFormatter; +import org.apache.phoenix.thirdparty.org.apache.commons.cli.Option; +import org.apache.phoenix.thirdparty.org.apache.commons.cli.Options; +import org.apache.phoenix.thirdparty.org.apache.commons.cli.ParseException; + +/** + * A MapReduce tool for verifying and detecting data inconsistencies between Phoenix tables across + * two HBase clusters (source and target). + *

Use Case

This tool is designed for replication/migration verification scenarios where + * data is replicated from a source Phoenix cluster to a target cluster. It efficiently detects + * which data chunks are out of sync without transferring all the data over the network. + *

How It Works

+ *
    + *
  1. Job Setup: The tool creates a MapReduce job that partitions the table into mapper + * regions based on HBase region boundaries or tenant ID ranges (for multi-tenant tables).
  2. + *
  3. Server-Side Chunking: Each mapper triggers a coprocessor scan on both source and + * target clusters. The {@link PhoenixSyncTableRegionScanner} coprocessor accumulates rows into + * chunks (configurable size, default 1GB) and computes a SHA-256 hash of all row data (keys + + * column families + qualifiers + timestamps + values).
  4. + *
  5. Hash Comparison: The {@link PhoenixSyncTableMapper} receives chunk metadata (start + * key, end key, row count, hash) from both clusters and compares the hashes. Matching hashes mean + * the chunk data is identical; mismatched hashes indicate inconsistency.
  6. + *
  7. Result Tracking: Results are checkpointed to the {@code PHOENIX_SYNC_TABLE_OUTPUT} + * table, tracking verified chunks, mismatched chunks, and processing progress for resumable + * operations.
  8. + *
+ *

Usage Example

+ * + *
+ * hbase org.apache.phoenix.mapreduce.PhoenixSyncTableTool \ --table-name MY_TABLE \
+ * --target-cluster target-zk1,target-zk2:2181:/hbase
+ */
+public class PhoenixSyncTableTool extends Configured implements Tool {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(PhoenixSyncTableTool.class);
+
+  private static final Option SCHEMA_NAME_OPTION =
+    new Option("s", "schema", true, "Phoenix schema name (optional)");
+  private static final Option TABLE_NAME_OPTION =
+    new Option("tn", "table-name", true, "Table name (mandatory)");
+  private static final Option TARGET_CLUSTER_OPTION =
+    new Option("tc", "target-cluster", true, "Target cluster ZooKeeper quorum (mandatory)");
+  private static final Option FROM_TIME_OPTION = new Option("ft", "from-time", true,
+    "Start time in milliseconds for sync (optional, defaults to 0)");
+  private static final Option TO_TIME_OPTION = new Option("rt", "to-time", true,
+    "End time in milliseconds for sync (optional, defaults to current time)");
+  private static final Option DRY_RUN_OPTION = new Option("dr", "dry-run", false,
+    "Dry run mode - only checkpoint inconsistencies, do not repair (optional)");
+  private static final Option CHUNK_SIZE_OPTION =
+    new Option("cs", "chunk-size", true, "Chunk size in bytes (optional, defaults to 1GB)");
+  private static final Option RUN_FOREGROUND_OPTION = new Option("runfg", "run-foreground", false,
+    "Run the job in foreground. Default - Runs the job in background.");
+  private static final Option TENANT_ID_OPTION =
+    new Option("tenant", "tenant-id", true, "Tenant ID for tenant-specific table sync (optional)");
+  private static final Option HELP_OPTION = new Option("h", "help", false, "Help");
+
+  private String schemaName;
+  private String tableName;
+  private String targetZkQuorum;
+  private Long startTime;
+  private Long endTime;
+  private boolean isDryRun;
+  private Long chunkSizeBytes;
+  private boolean isForeground;
+  private String tenantId;
+
+  private String qTable;
+  private String qSchemaName;
+
+  private Configuration configuration;
+  private Job job;
+  private PTable pTable;
+
+  /**
+   * Creates a MR job that uses server-side chunking and checksum calculation
+   * @return Configured MapReduce job ready for submission
+   * @throws Exception if job creation fails
+   */
+  private Job configureAndCreatePhoenixSyncTableJob(PTableType tableType) throws Exception {
+    configureTimeoutsAndRetries(configuration);
+    setPhoenixSyncTableToolConfiguration(configuration);
+    Job job = Job.getInstance(configuration, getJobName());
+    Configuration conf = job.getConfiguration();
+    HBaseConfiguration.merge(conf, HBaseConfiguration.create(conf));
+    configureInput(job, tableType);
+    job.setMapperClass(PhoenixSyncTableMapper.class);
+    job.setJarByClass(PhoenixSyncTableTool.class);
+    configureOutput(job);
+    TableMapReduceUtil.initCredentials(job);
+    TableMapReduceUtil.addDependencyJars(job);
+    obtainTargetClusterTokens(job);
+    return job;
+  }
+
+  /**
+   * Obtains HBase delegation tokens from the target cluster and adds them to the job. This is
+   * required for cross-cluster kerberos authentication.
+   * @param job The MapReduce job to add tokens
+   */
+  private void obtainTargetClusterTokens(Job job) throws IOException {
+    Configuration targetConf =
+      PhoenixMapReduceUtil.createConfigurationForZkQuorum(job.getConfiguration(), targetZkQuorum);
+    TableMapReduceUtil.initCredentialsForCluster(job, targetConf);
+  }
+
+  /**
+   * Configures timeouts and retry settings for the sync job
+   */
+  private void configureTimeoutsAndRetries(Configuration configuration) {
+    long syncTableQueryTimeoutMs =
+      configuration.getLong(QueryServices.SYNC_TABLE_QUERY_TIMEOUT_ATTRIB,
+        QueryServicesOptions.DEFAULT_SYNC_TABLE_QUERY_TIMEOUT);
+    long syncTableRPCTimeoutMs = configuration.getLong(QueryServices.SYNC_TABLE_RPC_TIMEOUT_ATTRIB,
+      QueryServicesOptions.DEFAULT_SYNC_TABLE_RPC_TIMEOUT);
+    long syncTableClientScannerTimeoutMs =
+      configuration.getLong(QueryServices.SYNC_TABLE_CLIENT_SCANNER_TIMEOUT_ATTRIB,
+        QueryServicesOptions.DEFAULT_SYNC_TABLE_CLIENT_SCANNER_TIMEOUT);
+    int syncTableRpcRetriesCounter =
+      configuration.getInt(QueryServices.SYNC_TABLE_RPC_RETRIES_COUNTER,
+        QueryServicesOptions.DEFAULT_SYNC_TABLE_RPC_RETRIES_COUNTER);
+
+    configuration.set(QueryServices.THREAD_TIMEOUT_MS_ATTRIB,
+      Long.toString(syncTableQueryTimeoutMs));
+    configuration.set(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD,
+      Long.toString(syncTableClientScannerTimeoutMs));
+    configuration.set(HConstants.HBASE_RPC_TIMEOUT_KEY, Long.toString(syncTableRPCTimeoutMs));
+    configuration.set(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
+      Integer.toString(syncTableRpcRetriesCounter));
+    configuration.set(MRJobConfig.TASK_TIMEOUT, Long.toString(syncTableQueryTimeoutMs));
+  }
+
+  private void setPhoenixSyncTableToolConfiguration(Configuration configuration) {
+    PhoenixConfigurationUtil.setPhoenixSyncTableName(configuration, qTable);
+    PhoenixConfigurationUtil.setPhoenixSyncTableTargetZkQuorum(configuration, targetZkQuorum);
+    PhoenixConfigurationUtil.setPhoenixSyncTableFromTime(configuration, startTime);
+    PhoenixConfigurationUtil.setPhoenixSyncTableToTime(configuration, endTime);
+    PhoenixConfigurationUtil.setPhoenixSyncTableDryRun(configuration, isDryRun);
+    PhoenixConfigurationUtil.setSplitByStats(configuration, false);
+    if (chunkSizeBytes != null) {
+      PhoenixConfigurationUtil.setPhoenixSyncTableChunkSizeBytes(configuration, chunkSizeBytes);
+    }
+    if (tenantId != null) {
+      PhoenixConfigurationUtil.setTenantId(configuration, tenantId);
+    }
+    PhoenixConfigurationUtil.setCurrentScnValue(configuration, endTime);
+    configuration
+      .setBooleanIfUnset(PhoenixConfigurationUtil.MAPREDUCE_RANDOMIZE_MAPPER_EXECUTION_ORDER, true);
+  }
+
+  private void configureInput(Job job, PTableType tableType) throws Exception {
+    // With below query plan, we get Input split based on region boundary
+    String hint = (tableType == PTableType.INDEX) ? "" : "/*+ NO_INDEX */ ";
+    String selectStatement = "SELECT " + hint + "1 FROM " + qTable;
+    PhoenixMapReduceUtil.setInput(job, DBInputFormat.NullDBWritable.class,
+      PhoenixSyncTableInputFormat.class, qTable, selectStatement);
+  }
+
+  private void configureOutput(Job job) {
+    job.setNumReduceTasks(0);
+    job.setOutputFormatClass(NullOutputFormat.class);
+  }
+
+  private String getJobName() {
+    StringBuilder jobName = new StringBuilder("PhoenixSyncTable");
+    if (qSchemaName != null) {
+      jobName.append("-").append(qSchemaName);
+    }
+    jobName.append("-").append(tableName);
+    jobName.append("-").append(System.currentTimeMillis());
+    return jobName.toString();
+  }
+
+  private CommandLine parseOptions(String[] args) throws IllegalStateException {
+    Options options = getOptions();
+    CommandLineParser parser = DefaultParser.builder().setAllowPartialMatching(false)
+      .setStripLeadingAndTrailingQuotes(false).build();
+    CommandLine cmdLine = null;
+    try {
+      cmdLine = parser.parse(options, args);
+    } catch (ParseException e) {
+      LOGGER.error("Failed to parse command line options. Args: {}. Error: {}",
+        Arrays.toString(args), e.getMessage(), e);
+      printHelpAndExit("Error parsing command line options: " + e.getMessage(), options);
+    }
+
+    if (cmdLine.hasOption(HELP_OPTION.getOpt())) {
+      printHelpAndExit(options, 0);
+    }
+    requireOption(cmdLine, TABLE_NAME_OPTION);
+    requireOption(cmdLine, TARGET_CLUSTER_OPTION);
+    return cmdLine;
+  }
+
+  private void requireOption(CommandLine cmdLine, Option option) {
+    if (!cmdLine.hasOption(option.getOpt())) {
+      throw new IllegalStateException(option.getLongOpt() + " is a mandatory parameter");
+    }
+  }
+
+  private Options getOptions() {
+    Options options = new Options();
+    options.addOption(SCHEMA_NAME_OPTION);
+    options.addOption(TABLE_NAME_OPTION);
+    options.addOption(TARGET_CLUSTER_OPTION);
+    options.addOption(FROM_TIME_OPTION);
+    options.addOption(TO_TIME_OPTION);
+    options.addOption(DRY_RUN_OPTION);
+    options.addOption(CHUNK_SIZE_OPTION);
+    options.addOption(RUN_FOREGROUND_OPTION);
+    options.addOption(TENANT_ID_OPTION);
+    options.addOption(HELP_OPTION);
+    return options;
+  }
+
+  private void printHelpAndExit(String errorMessage, Options options) {
+    System.err.println(errorMessage);
+    printHelpAndExit(options, -1);
+  }
+
+  private void printHelpAndExit(Options options, int exitCode) {
+    HelpFormatter formatter = new HelpFormatter();
+    formatter.printHelp("hadoop jar phoenix-server.jar " + PhoenixSyncTableTool.class.getName(),
+      "Synchronize a Phoenix table between source and target clusters", options,
+      "\nExample usage:\n"
+        + "hadoop jar phoenix-server.jar org.apache.phoenix.mapreduce.PhoenixSyncTableTool \\\n"
+        + "  --table-name MY_TABLE \\\n" + "  --target-cluster :2181 \\\n"
+        + "  --dry-run\n",
+      true);
+    System.exit(exitCode);
+  }
+
+  public void populateSyncTableToolAttributes(CommandLine cmdLine) throws Exception {
+    tableName = cmdLine.getOptionValue(TABLE_NAME_OPTION.getOpt());
+    targetZkQuorum = cmdLine.getOptionValue(TARGET_CLUSTER_OPTION.getOpt());
+    schemaName = cmdLine.getOptionValue(SCHEMA_NAME_OPTION.getOpt());
+
+    if (cmdLine.hasOption(FROM_TIME_OPTION.getOpt())) {
+      startTime = Long.valueOf(cmdLine.getOptionValue(FROM_TIME_OPTION.getOpt()));
+    } else {
+      startTime = 0L;
+    }
+
+    if (cmdLine.hasOption(TO_TIME_OPTION.getOpt())) {
+      endTime = Long.valueOf(cmdLine.getOptionValue(TO_TIME_OPTION.getOpt()));
+    } else {
+      endTime = EnvironmentEdgeManager.currentTimeMillis();
+    }
+
+    if (cmdLine.hasOption(CHUNK_SIZE_OPTION.getOpt())) {
+      chunkSizeBytes = Long.valueOf(cmdLine.getOptionValue(CHUNK_SIZE_OPTION.getOpt()));
+    }
+    if (cmdLine.hasOption(TENANT_ID_OPTION.getOpt())) {
+      tenantId = cmdLine.getOptionValue(TENANT_ID_OPTION.getOpt());
+    }
+    isDryRun = cmdLine.hasOption(DRY_RUN_OPTION.getOpt());
+    isForeground = cmdLine.hasOption(RUN_FOREGROUND_OPTION.getOpt());
+    qTable = SchemaUtil.getQualifiedTableName(schemaName, tableName);
+    qSchemaName = SchemaUtil.normalizeIdentifier(schemaName);
+    PhoenixMapReduceUtil.validateTimeRange(startTime, endTime, qTable);
+    PhoenixMapReduceUtil.validateMaxLookbackAge(configuration, endTime, qTable);
+    if (LOGGER.isDebugEnabled()) {
+      LOGGER.debug(
+        "PhoenixSyncTableTool configured - Table: {}, Schema: {}, Target: {}, "
+          + "StartTime: {}, EndTime: {}, DryRun: {}, ChunkSize: {}, Foreground: {}, TenantId: {}",
+        qTable, qSchemaName, targetZkQuorum, startTime, endTime, isDryRun, chunkSizeBytes,
+        isForeground, tenantId);
+    }
+  }
+
+  /**
+   * Creates or verifies the Phoenix sync tool checkpoint output table
+   */
+  private void createSyncOutputTable(Connection connection) throws SQLException {
+    PhoenixSyncTableOutputRepository repository = new PhoenixSyncTableOutputRepository(connection);
+    repository.createOutputTableIfNotExists();
+  }
+
+  /**
+   * Sets up the table reference and validates it exists and is suitable for sync operations.
+   * Validates that the table is not a VIEW
+   */
+  private PTableType validateAndGetTableType() throws SQLException {
+    Properties props = new Properties();
+    if (tenantId != null) {
+      props.setProperty("TenantId", tenantId);
+    }
+    try (Connection connection = ConnectionUtil.getInputConnection(configuration, props)) {
+      pTable = PhoenixMapReduceUtil.validateTableForMRJob(connection, qTable, false, true);
+      return pTable.getType();
+    }
+  }
+
+  private boolean submitPhoenixSyncTableJob() throws Exception {
+    if (!isForeground) {
+      job.submit();
+      LOGGER.info("PhoenixSyncTable Job :{} submitted successfully in background for table {} ",
+        job.getJobName(), qTable);
+      return true;
+    }
+    LOGGER.info("Running PhoenixSyncTable job: {} for table:{}in foreground.", job.getJobName(),
+      qTable);
+    boolean success = job.waitForCompletion(true);
+    if (success) {
+      LOGGER.info("PhoenixSyncTable job: {} completed for table {}", job.getJobName(), qTable);
+    } else {
+      LOGGER.error("PhoenixSyncTable job {} failed for table {} to target cluster {}",
+        job.getJobName(), qTable, targetZkQuorum);
+    }
+    return success;
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+    CommandLine cmdLine;
+    try {
+      cmdLine = parseOptions(args);
+    } catch (IllegalStateException e) {
+      printHelpAndExit(e.getMessage(), getOptions());
+      return -1;
+    }
+    configuration = HBaseConfiguration.addHbaseResources(getConf());
+    try (Connection globalConn = ConnectionUtil.getInputConnection(configuration)) {
+      createSyncOutputTable(globalConn);
+    }
+    populateSyncTableToolAttributes(cmdLine);
+    try {
+      PTableType tableType = validateAndGetTableType();
+      job = configureAndCreatePhoenixSyncTableJob(tableType);
+      boolean result = submitPhoenixSyncTableJob();
+      Counters counters = job.getCounters();
+      LOGGER.info(
+        "PhoenixSyncTable job completed, gathered counters are \n" + "Input Record: {}, \n"
+          + "Ouput Record: {}, \n" + "Failed Record: {}, \n" + "Chunks Verified: {}, \n"
+          + "Chunks Mimatched: {}," + "Rows Processed: {}",
+        counters.findCounter(PhoenixJobCounters.INPUT_RECORDS).getValue(),
+        counters.findCounter(PhoenixJobCounters.OUTPUT_RECORDS).getValue(),
+        counters.findCounter(PhoenixJobCounters.FAILED_RECORDS).getValue(),
+        counters.findCounter(PhoenixSyncTableMapper.SyncCounters.CHUNKS_VERIFIED).getValue(),
+        counters.findCounter(PhoenixSyncTableMapper.SyncCounters.CHUNKS_MISMATCHED).getValue(),
+        counters.findCounter(PhoenixSyncTableMapper.SyncCounters.ROWS_PROCESSED).getValue());
+      return result ? 0 : -1;
+    } catch (Exception ex) {
+      LOGGER.error(
+        "Exception occurred while performing phoenix sync table job for table {} to target {}: {}",
+        qTable, targetZkQuorum, ExceptionUtils.getMessage(ex), ex);
+      return -1;
+    }
+  }
+
+  public static void main(String[] args) throws Exception {
+    int exitCode = ToolRunner.run(new PhoenixSyncTableTool(), args);
+    System.exit(exitCode);
+  }
+
+  // Getters for testing
+  @VisibleForTesting
+  public String getQTable() {
+    return qTable;
+  }
+
+  @VisibleForTesting
+  public String getTargetZkQuorum() {
+    return targetZkQuorum;
+  }
+
+  @VisibleForTesting
+  public boolean isDryRun() {
+    return isDryRun;
+  }
+
+  @VisibleForTesting
+  public Job getJob() {
+    return job;
+  }
+
+  @VisibleForTesting
+  public long getStartTime() {
+    return startTime;
+  }
+
+  @VisibleForTesting
+  public long getEndTime() {
+    return endTime;
+  }
+}
diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyTool.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyTool.java
index 92ea2254a83..625acedc015 100644
--- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyTool.java
+++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyTool.java
@@ -38,7 +38,6 @@
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.phoenix.coprocessorclient.BaseScannerRegionObserverConstants;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.mapreduce.CsvBulkImportUtil;
 import org.apache.phoenix.mapreduce.util.ConnectionUtil;
@@ -402,7 +401,7 @@ public int run(String[] args) throws Exception {
         ? Long.parseLong(cmdLine.getOptionValue(TIMESTAMP.getOpt()))
         : EnvironmentEdgeManager.currentTimeMillis() - 60000;
 
-      validateTimestamp(configuration, ts);
+      PhoenixMapReduceUtil.validateMaxLookbackAge(configuration, ts, qDataTable);
 
       if (indexTable != null) {
         if (!IndexTool.isValidIndexTable(connection, qDataTable, indexTable, tenantId)) {
@@ -495,20 +494,6 @@ public int run(String[] args) throws Exception {
     }
   }
 
-  private void validateTimestamp(Configuration configuration, long ts) {
-    long maxLookBackAge = BaseScannerRegionObserverConstants.getMaxLookbackInMillis(configuration);
-    if (
-      maxLookBackAge != BaseScannerRegionObserverConstants.DEFAULT_PHOENIX_MAX_LOOKBACK_AGE * 1000L
-    ) {
-      long minTimestamp = EnvironmentEdgeManager.currentTimeMillis() - maxLookBackAge;
-      if (ts < minTimestamp) {
-        throw new IllegalArgumentException("Index scrutiny can't look back past the configured"
-          + "max lookback age: " + maxLookBackAge / 1000 + " seconds");
-      }
-    }
-
-  }
-
   @VisibleForTesting
   public List getJobs() {
     return jobs;
diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java
index 92d4d01c559..fca307d4791 100644
--- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java
+++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java
@@ -925,9 +925,6 @@ public int populateIndexToolAttributes(CommandLine cmdLine) throws Exception {
       lastVerifyTime = new Long(cmdLine.getOptionValue(RETRY_VERIFY_OPTION.getOpt()));
       validateLastVerifyTime();
     }
-    if (isTimeRangeSet(startTime, endTime)) {
-      validateTimeRange(startTime, endTime);
-    }
     if (verify) {
       String value = cmdLine.getOptionValue(VERIFY_OPTION.getOpt());
       indexVerifyType = IndexVerifyType.fromValue(value);
@@ -952,6 +949,9 @@ public int populateIndexToolAttributes(CommandLine cmdLine) throws Exception {
     isForeground = cmdLine.hasOption(RUN_FOREGROUND_OPTION.getOpt());
     useSnapshot = cmdLine.hasOption(SNAPSHOT_OPTION.getOpt());
     shouldDeleteBeforeRebuild = cmdLine.hasOption(DELETE_ALL_AND_REBUILD_OPTION.getOpt());
+    if (isTimeRangeSet(startTime, endTime)) {
+      PhoenixMapReduceUtil.validateTimeRange(startTime, endTime, qDataTable);
+    }
     return 0;
   }
 
@@ -982,15 +982,6 @@ public boolean isValidLastVerifyTime(Long lastVerifyTime) throws Exception {
     }
   }
 
-  public static void validateTimeRange(Long sTime, Long eTime) {
-    Long currentTime = EnvironmentEdgeManager.currentTimeMillis();
-    Long st = (sTime == null) ? 0 : sTime;
-    Long et = (eTime == null) ? currentTime : eTime;
-    if (st.compareTo(currentTime) > 0 || et.compareTo(currentTime) > 0 || st.compareTo(et) >= 0) {
-      throw new RuntimeException(INVALID_TIME_RANGE_EXCEPTION_MESSAGE);
-    }
-  }
-
   private Connection getConnection(Configuration configuration) throws SQLException {
     return ConnectionUtil.getInputConnection(configuration);
   }
diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/transform/TransformTool.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/transform/TransformTool.java
index c69857930a1..d2a2dde5236 100644
--- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/transform/TransformTool.java
+++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/transform/TransformTool.java
@@ -21,7 +21,6 @@
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES;
 import static org.apache.phoenix.mapreduce.index.IndexTool.createIndexToolTables;
 import static org.apache.phoenix.mapreduce.index.IndexTool.isTimeRangeSet;
-import static org.apache.phoenix.mapreduce.index.IndexTool.validateTimeRange;
 import static org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil.setCurrentScnValue;
 import static org.apache.phoenix.query.QueryConstants.UNVERIFIED_BYTES;
 import static org.apache.phoenix.util.QueryUtil.getConnection;
@@ -335,10 +334,6 @@ public int populateTransformToolAttributesAndValidate(CommandLine cmdLine) throw
       endTime = new Long(cmdLine.getOptionValue(END_TIME_OPTION.getOpt()));
     }
 
-    if (isTimeRangeSet(startTime, endTime)) {
-      validateTimeRange(startTime, endTime);
-    }
-
     if (
       (isPartialTransform || shouldFixUnverified) && (cmdLine.hasOption(AUTO_SPLIT_OPTION.getOpt()))
     ) {
@@ -363,6 +358,9 @@ public int populateTransformToolAttributesAndValidate(CommandLine cmdLine) throw
     dataTable = cmdLine.getOptionValue(DATA_TABLE_OPTION.getOpt());
     indexTable = cmdLine.getOptionValue(INDEX_TABLE_OPTION.getOpt());
     qDataTable = SchemaUtil.getQualifiedTableName(schemaName, dataTable);
+    if (isTimeRangeSet(startTime, endTime)) {
+      PhoenixMapReduceUtil.validateTimeRange(startTime, endTime, qDataTable);
+    }
     isForeground = cmdLine.hasOption(RUN_FOREGROUND_OPTION.getOpt());
     if (cmdLine.hasOption(SPLIT_SIZE_OPTION.getOpt())) {
       splitSize = Integer.parseInt(cmdLine.getOptionValue(SPLIT_SIZE_OPTION.getOpt()));
diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java
index e396956f7f6..18837b82963 100644
--- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java
+++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java
@@ -224,6 +224,26 @@ public final class PhoenixConfigurationUtil {
   // non-index jobs benefit less from this
   public static final boolean DEFAULT_MAPREDUCE_RANDOMIZE_MAPPER_EXECUTION_ORDER = false;
 
+  public static final String PHOENIX_SYNC_TABLE_NAME = "phoenix.sync.table.table.name";
+
+  public static final String PHOENIX_SYNC_TABLE_TARGET_ZK_QUORUM = "phoenix.sync.table.target.zk";
+
+  public static final String PHOENIX_SYNC_TABLE_FROM_TIME = "phoenix.sync.table.from.time";
+
+  public static final String PHOENIX_SYNC_TABLE_TO_TIME = "phoenix.sync.table.to.time";
+
+  public static final String PHOENIX_SYNC_TABLE_DRY_RUN = "phoenix.sync.table.dry.run";
+
+  public static final String PHOENIX_SYNC_TABLE_CHUNK_SIZE_BYTES =
+    "phoenix.sync.table.chunk.size.bytes";
+
+  public static final String PHOENIX_SYNC_TABLE_CHUNK_MAX_ROWS =
+    "phoenix.sync.table.chunk.max.rows";
+
+  public static final long DEFAULT_PHOENIX_SYNC_TABLE_CHUNK_SIZE_BYTES = 100 * 1024 * 1024; // 100MB
+
+  public static final int DEFAULT_PHOENIX_SYNC_TABLE_CHUNK_MAX_ROWS = 100000;
+
   /**
    * Determines type of Phoenix Map Reduce job. 1. QUERY allows running arbitrary queries without
    * aggregates 2. UPDATE_STATS collects statistics for the table
@@ -941,4 +961,80 @@ public static boolean isMRRandomizeMapperExecutionOrder(final Configuration conf
     return configuration.getBoolean(MAPREDUCE_RANDOMIZE_MAPPER_EXECUTION_ORDER,
       DEFAULT_MAPREDUCE_RANDOMIZE_MAPPER_EXECUTION_ORDER);
   }
+
+  public static void setPhoenixSyncTableName(Configuration conf, String tableName) {
+    Preconditions.checkNotNull(conf);
+    conf.set(PHOENIX_SYNC_TABLE_NAME, tableName);
+  }
+
+  public static String getPhoenixSyncTableName(Configuration conf) {
+    Preconditions.checkNotNull(conf);
+    return conf.get(PHOENIX_SYNC_TABLE_NAME);
+  }
+
+  public static void setPhoenixSyncTableTargetZkQuorum(Configuration conf, String zkQuorum) {
+    Preconditions.checkNotNull(conf);
+    conf.set(PHOENIX_SYNC_TABLE_TARGET_ZK_QUORUM, zkQuorum);
+  }
+
+  public static String getPhoenixSyncTableTargetZkQuorum(Configuration conf) {
+    Preconditions.checkNotNull(conf);
+    return conf.get(PHOENIX_SYNC_TABLE_TARGET_ZK_QUORUM);
+  }
+
+  public static void setPhoenixSyncTableFromTime(Configuration conf, Long fromTime) {
+    Preconditions.checkNotNull(conf);
+    conf.setLong(PHOENIX_SYNC_TABLE_FROM_TIME, fromTime);
+  }
+
+  public static Long getPhoenixSyncTableFromTime(Configuration conf) {
+    Preconditions.checkNotNull(conf);
+    String value = conf.get(PHOENIX_SYNC_TABLE_FROM_TIME);
+    return Long.valueOf(value);
+  }
+
+  public static void setPhoenixSyncTableToTime(Configuration conf, Long toTime) {
+    Preconditions.checkNotNull(conf);
+    conf.setLong(PHOENIX_SYNC_TABLE_TO_TIME, toTime);
+  }
+
+  public static Long getPhoenixSyncTableToTime(Configuration conf) {
+    Preconditions.checkNotNull(conf);
+    String value = conf.get(PHOENIX_SYNC_TABLE_TO_TIME);
+    return Long.valueOf(value);
+  }
+
+  public static void setPhoenixSyncTableDryRun(Configuration conf, boolean dryRun) {
+    Preconditions.checkNotNull(conf);
+    conf.setBoolean(PHOENIX_SYNC_TABLE_DRY_RUN, dryRun);
+  }
+
+  public static boolean getPhoenixSyncTableDryRun(Configuration conf) {
+    Preconditions.checkNotNull(conf);
+    return conf.getBoolean(PHOENIX_SYNC_TABLE_DRY_RUN, false);
+  }
+
+  public static void setPhoenixSyncTableChunkSizeBytes(Configuration conf, Long chunkSizeBytes) {
+    Preconditions.checkNotNull(conf);
+    conf.setLong(PHOENIX_SYNC_TABLE_CHUNK_SIZE_BYTES, chunkSizeBytes);
+  }
+
+  public static long getPhoenixSyncTableChunkSizeBytes(Configuration conf) {
+    Preconditions.checkNotNull(conf);
+    return conf.getLong(PHOENIX_SYNC_TABLE_CHUNK_SIZE_BYTES,
+      DEFAULT_PHOENIX_SYNC_TABLE_CHUNK_SIZE_BYTES);
+  }
+
+  public static void setPhoenixSyncTableChunkMaxRows(Configuration conf, Integer chunkMaxRows) {
+    Preconditions.checkNotNull(conf);
+    if (chunkMaxRows != null) {
+      conf.setInt(PHOENIX_SYNC_TABLE_CHUNK_MAX_ROWS, chunkMaxRows);
+    }
+  }
+
+  public static int getPhoenixSyncTableChunkMaxRows(Configuration conf) {
+    Preconditions.checkNotNull(conf);
+    return conf.getInt(PHOENIX_SYNC_TABLE_CHUNK_MAX_ROWS,
+      DEFAULT_PHOENIX_SYNC_TABLE_CHUNK_MAX_ROWS);
+  }
 }
diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixMapReduceUtil.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixMapReduceUtil.java
index 9c34625d9d7..cad5032d326 100644
--- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixMapReduceUtil.java
+++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixMapReduceUtil.java
@@ -18,16 +18,24 @@
 package org.apache.phoenix.mapreduce.util;
 
 import java.io.IOException;
+import java.sql.Connection;
+import java.sql.SQLException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.lib.db.DBWritable;
+import org.apache.phoenix.coprocessorclient.BaseScannerRegionObserverConstants;
+import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.mapreduce.PhoenixInputFormat;
 import org.apache.phoenix.mapreduce.PhoenixMultiViewInputFormat;
 import org.apache.phoenix.mapreduce.PhoenixOutputFormat;
 import org.apache.phoenix.mapreduce.PhoenixTTLTool;
 import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil.SchemaType;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
 
 /**
  * Utility class for setting Configuration parameters for the Map Reduce job
@@ -239,4 +247,100 @@ public static void setTenantId(final Job job, final String tenantId) {
     PhoenixConfigurationUtil.setTenantId(job.getConfiguration(), tenantId);
   }
 
+  /**
+   * Validates that start and end times are in the past and start < end.
+   * @param startTime Start timestamp in millis (nullable, defaults to 0)
+   * @param endTime   End timestamp in millis (nullable, defaults to current time)
+   * @param tableName Table name for error messages
+   * @throws IllegalArgumentException if time range is invalid
+   */
+  public static void validateTimeRange(Long startTime, Long endTime, String tableName) {
+    Long currentTime = EnvironmentEdgeManager.currentTimeMillis();
+    Long st = (startTime == null) ? 0L : startTime;
+    Long et = (endTime == null) ? currentTime : endTime;
+
+    if (st.compareTo(currentTime) > 0 || et.compareTo(currentTime) > 0 || st.compareTo(et) >= 0) {
+      throw new IllegalArgumentException(String.format(
+        "Invalid time range for table %s: start and end times must be in the past and start < end. "
+          + "Start: %d, End: %d, Current: %d",
+        tableName, st, et, currentTime));
+    }
+  }
+
+  /**
+   * Validates that the end time doesn't exceed the max lookback age configured in Phoenix.
+   * @param configuration Hadoop configuration
+   * @param endTime       End timestamp in millis
+   * @param tableName     Table name for error messages
+   * @throws IllegalArgumentException if endTime is before min allowed timestamp
+   */
+  public static void validateMaxLookbackAge(Configuration configuration, Long endTime,
+    String tableName) {
+    long maxLookBackAge = BaseScannerRegionObserverConstants.getMaxLookbackInMillis(configuration);
+    if (maxLookBackAge > 0) {
+      long minTimestamp = EnvironmentEdgeManager.currentTimeMillis() - maxLookBackAge;
+      if (endTime < minTimestamp) {
+        throw new IllegalArgumentException(String.format(
+          "Table %s sync can't look back past the configured max lookback age: %d ms. "
+            + "End time: %d, Min allowed timestamp: %d",
+          tableName, maxLookBackAge, endTime, minTimestamp));
+      }
+    }
+  }
+
+  /**
+   * Validates that a table is suitable for MR operations. Checks table existence, type, and state.
+   * @param connection         Phoenix connection
+   * @param qualifiedTableName Qualified table name
+   * @param allowViews         Whether to allow VIEW tables
+   * @param allowIndexes       Whether to allow INDEX tables
+   * @return PTable instance
+   * @throws SQLException             if connection fails
+   * @throws IllegalArgumentException if validation fails
+   */
+  public static PTable validateTableForMRJob(Connection connection, String qualifiedTableName,
+    boolean allowViews, boolean allowIndexes) throws SQLException {
+    PTable pTable = connection.unwrap(PhoenixConnection.class).getTableNoCache(qualifiedTableName);
+
+    if (pTable == null) {
+      throw new IllegalArgumentException(
+        String.format("Table %s does not exist", qualifiedTableName));
+    } else if (!allowViews && pTable.getType() == PTableType.VIEW) {
+      throw new IllegalArgumentException(
+        String.format("Cannot run MR job on VIEW table %s", qualifiedTableName));
+    } else if (!allowIndexes && pTable.getType() == PTableType.INDEX) {
+      throw new IllegalArgumentException(
+        String.format("Cannot run MR job on INDEX table %s directly", qualifiedTableName));
+    }
+
+    return pTable;
+  }
+
+  /**
+   * Configures a Configuration object with ZooKeeper settings from a ZK quorum string.
+   * @param baseConf Base configuration to create from (typically job configuration)
+   * @param zkQuorum ZooKeeper quorum string in format: "zk_quorum:port:znode" Example:
+   *                 "zk1,zk2,zk3:2181:/hbase"
+   * @return New Configuration with ZK settings applied
+   * @throws RuntimeException if zkQuorum format is invalid (must have exactly 3 parts)
+   */
+  public static Configuration createConfigurationForZkQuorum(Configuration baseConf,
+    String zkQuorum) {
+    Configuration conf = org.apache.hadoop.hbase.HBaseConfiguration.create(baseConf);
+    String[] parts = zkQuorum.split(":");
+
+    if (!(parts.length == 3 || parts.length == 4)) {
+      throw new RuntimeException(
+        "Invalid ZooKeeper quorum format. Expected: zk_quorum:port:znode OR "
+          + "zk_quorum:port:znode:krb_principal. Got: " + zkQuorum);
+    }
+
+    conf.set(HConstants.ZOOKEEPER_QUORUM, parts[0]);
+    conf.set(HConstants.ZOOKEEPER_CLIENT_PORT, parts[1]);
+    conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, parts[2]);
+    if (parts.length == 4) {
+      conf.set(HConstants.ZK_CLIENT_KERBEROS_PRINCIPAL, parts[3]);
+    }
+    return conf;
+  }
 }

From 833a7cb8cb4c25d4325e67401ee85d208b2a5181 Mon Sep 17 00:00:00 2001
From: Rahul Kumar 
Date: Wed, 11 Feb 2026 18:12:38 +0530
Subject: [PATCH 2/9] Self review checkpointing#1

---
 .../BaseScannerRegionObserverConstants.java   |  14 +--
 .../apache/phoenix/query/QueryServices.java   |   9 +-
 phoenix-core-server/pom.xml                   |   5 +
 .../PhoenixSyncTableRegionScanner.java        |  87 +++++++--------
 .../PhoenixSyncTableInputFormat.java          |  11 +-
 .../mapreduce/PhoenixSyncTableMapper.java     |  95 +++++++++-------
 .../PhoenixSyncTableOutputRepository.java     | 103 ++----------------
 .../mapreduce/PhoenixSyncTableOutputRow.java  |   3 +-
 .../mapreduce/PhoenixSyncTableTool.java       |   2 +-
 .../util/PhoenixConfigurationUtil.java        |  17 +--
 10 files changed, 131 insertions(+), 215 deletions(-)

diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/BaseScannerRegionObserverConstants.java b/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/BaseScannerRegionObserverConstants.java
index 3de94ba9739..51066d98f46 100644
--- a/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/BaseScannerRegionObserverConstants.java
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/BaseScannerRegionObserverConstants.java
@@ -195,27 +195,27 @@ public static long getMaxLookbackInMillis(Configuration conf) {
   public static final String CUSTOM_ANNOTATIONS = "_Annot";
 
   /**
-   * SyncTableTool scan attributes for server-side chunk formation and checksum
+   * PhoenixSyncTableTool scan attributes for server-side chunk formation and checksum
    */
   public static final String SYNC_TABLE_CHUNK_FORMATION = "_SyncTableChunk";
   public static final String SYNC_TABLE_MAPPER_REGION_END_KEY = "_SyncTableMapperRegionEnd";
   public static final String SYNC_TABLE_CHUNK_SIZE_BYTES = "_SyncTableChunkSizeBytes";
-  public static final String SYNC_TABLE_CHUNK_MAX_ROWS = "_SyncTableChunkMaxRows";
   public static final String SYNC_TABLE_FORCE_FULL_RANGE = "_SyncTableForceFullRange";
   public static final String SYNC_TABLE_CONTINUED_DIGEST_STATE = "_SyncTableContinuedDigestState";
 
   /**
-   * SyncTableTool chunk metadata cell qualifiers. These define the wire protocol between
-   * PhoenixSyncTableRegionScanner (server-side coprocessor) and PhoenixSyncTableMapper (client-side
+   * PhoenixSyncTableTool chunk metadata cell qualifiers. These define the wire protocol between
+   * hoenixSyncTableRegionScanner (server-side coprocessor) and PhoenixSyncTableMapper (client-side
    * mapper). The coprocessor returns chunk metadata as HBase cells with these qualifiers, and the
    * mapper parses them to extract chunk information.
    */
-  public static final byte[] SYNC_TABLE_START_KEY_QUALIFIER = Bytes.toBytes("START_KEY");
   public static final byte[] SYNC_TABLE_END_KEY_QUALIFIER = Bytes.toBytes("END_KEY");
   public static final byte[] SYNC_TABLE_HASH_QUALIFIER = Bytes.toBytes("HASH");
   public static final byte[] SYNC_TABLE_ROW_COUNT_QUALIFIER = Bytes.toBytes("ROW_COUNT");
-  public static final byte[] SYNC_TABLE_IS_PARTIAL_QUALIFIER = Bytes.toBytes("IS_PARTIAL");
-  public static final byte[] SYNC_TABLE_HAS_MORE_ROWS_QUALIFIER = Bytes.toBytes("HAS_MORE_ROWS");
+  public static final byte[] SYNC_TABLE_IS_PARTIAL_CHUNK_QUALIFIER =
+    Bytes.toBytes("IS_PARTIAL_CHUNK");
+  public static final byte[] SYNC_TABLE_HAS_MORE_ROWS_IN_REGION_QUALIFIER =
+    Bytes.toBytes("HAS_MORE_ROWS_IN_REGION");
 
   /** Exposed for testing */
   public static final String SCANNER_OPENED_TRACE_INFO = "Scanner opened on server";
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServices.java
index 5f85bd48d0c..26e7b4f73ea 100644
--- a/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -212,12 +212,13 @@ public interface QueryServices extends SQLCloseable {
   public static final String SERVER_SIDE_PRIOIRTY_ATTRIB = "phoenix.serverside.rpc.priority";
   public static final String ALLOW_LOCAL_INDEX_ATTRIB = "phoenix.index.allowLocalIndex";
 
-  public static final String SYNC_TABLE_QUERY_TIMEOUT_ATTRIB = "phoenix.synctable.query.timeout";
-  public static final String SYNC_TABLE_RPC_TIMEOUT_ATTRIB = "phoenix.synctable.rpc.timeout";
+  // Timeout config for PhoenixSyncTableTool
+  public static final String SYNC_TABLE_QUERY_TIMEOUT_ATTRIB = "phoenix.sync.table.query.timeout";
+  public static final String SYNC_TABLE_RPC_TIMEOUT_ATTRIB = "phoenix.sync.table.rpc.timeout";
   public static final String SYNC_TABLE_CLIENT_SCANNER_TIMEOUT_ATTRIB =
-    "phoenix.synctable.client.scanner.timeout";
+    "phoenix.sync.table.client.scanner.timeout";
   public static final String SYNC_TABLE_RPC_RETRIES_COUNTER =
-    "phoenix.synctable.rpc.retries.counter";
+    "phoenix.sync.table.rpc.retries.counter";
 
   // Retries when doing server side writes to SYSTEM.CATALOG
   public static final String METADATA_WRITE_RETRIES_NUMBER = "phoenix.metadata.rpc.retries.number";
diff --git a/phoenix-core-server/pom.xml b/phoenix-core-server/pom.xml
index 93f6e6d4323..e87a3e9eb09 100644
--- a/phoenix-core-server/pom.xml
+++ b/phoenix-core-server/pom.xml
@@ -173,6 +173,11 @@
       com.google.code.findbugs
       jsr305
     
+    
+      org.bouncycastle
+      bcprov-jdk18on
+      1.79
+    
   
 
   
diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java
index 38b677d0a07..cca8cec5f0b 100644
--- a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java
+++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java
@@ -55,19 +55,19 @@
  * (keys, column families, qualifiers, timestamps, cell types, values).
  * 

* Source mode (forceFullRange=false): Returns complete chunks bounded by region boundaries. Sets - * hasMoreRows=false when region is exhausted. + * hasMoreRowsInRegion=false when region is exhausted. *

* Target mode (forceFullRange=true): Returns partial chunks with serialized digest state when * region boundary is reached, allowing cross-region hash continuation. *

- * Returns chunk metadata cells: START_KEY, END_KEY, HASH (or digest state), ROW_COUNT, IS_PARTIAL, - * HAS_MORE_ROWS. + * Returns chunk metadata cells: END_KEY, HASH (or digest state), ROW_COUNT, IS_PARTIAL_CHUNK, + * HAS_MORE_ROWS_IN_REGION. */ 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 static final int MAX_SHA256_STATE_SIZE = 128; + private static final int MAX_SHA256_DIGEST_STATE_SIZE = 128; private final Region region; private final Scan scan; private final RegionCoprocessorEnvironment env; @@ -75,15 +75,15 @@ public class PhoenixSyncTableRegionScanner extends BaseRegionScanner { private final byte[] mapperRegionEndKey; private final long chunkSizeBytes; - private final long chunkMaxRows; private final boolean forceFullRange; private byte[] chunkStartKey = null; private byte[] chunkEndKey = null; private long currentChunkSize = 0L; private long currentChunkRowCount = 0L; private SHA256Digest digest; - private boolean hasMoreRows = true; - private boolean isUsingContinuedDigest = false; + private boolean hasMoreRowsInRegion = true; + private boolean isUsingContinuedDigest; // If target chunk was partial, and we are continuing to + // update digest before calculating checksum /** * @param innerScanner The underlying region scanner @@ -106,18 +106,13 @@ public PhoenixSyncTableRegionScanner(final RegionScanner innerScanner, final Reg scan.getAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_MAPPER_REGION_END_KEY); byte[] chunkSizeAttr = scan.getAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CHUNK_SIZE_BYTES); - byte[] chunkMaxRowsAttr = - scan.getAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CHUNK_MAX_ROWS); this.mapperRegionEndKey = mapperEndAttr != null ? mapperEndAttr : new byte[0]; this.chunkSizeBytes = chunkSizeAttr != null ? Bytes.toLong(chunkSizeAttr) : DEFAULT_PHOENIX_SYNC_TABLE_CHUNK_SIZE_BYTES; - this.chunkMaxRows = chunkMaxRowsAttr != null - ? Bytes.toLong(chunkMaxRowsAttr) - : DEFAULT_PHOENIX_SYNC_TABLE_CHUNK_MAX_ROWS; byte[] forceFullRangeAttr = scan.getAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_FORCE_FULL_RANGE); - this.forceFullRange = Bytes.toBoolean(forceFullRangeAttr); + this.forceFullRange = (forceFullRangeAttr != null && Bytes.toBoolean(forceFullRangeAttr)); // Check if we should continue from a previous digest state (cross-region continuation) byte[] continuedDigestStateAttr = scan.getAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CONTINUED_DIGEST_STATE); @@ -147,11 +142,11 @@ public boolean next(List results) throws IOException { RegionScanner localScanner = delegate; synchronized (localScanner) { List rowCells = new ArrayList<>(); - while (hasMoreRows) { + while (hasMoreRowsInRegion) { // Check region state INSIDE loop for long-running scans ungroupedAggregateRegionObserver.checkForRegionClosingOrSplitting(); rowCells.clear(); - hasMoreRows = localScanner.nextRaw(rowCells); + hasMoreRowsInRegion = localScanner.nextRaw(rowCells); if (rowCells.isEmpty()) { break; } @@ -165,16 +160,23 @@ public boolean next(List results) throws IOException { } } if (chunkStartKey == null) { - // TODO: Add throw exception here and test if it returns null ? + // LOGGER.error( + // "Exception during chunk scanning in region {} table {} at chunk startKey: {}, endkey: + // {})", + // region.getRegionInfo().getRegionNameAsString(), + // region.getRegionInfo().getTable().getNameAsString(), + // chunkStartKey != null ? Bytes.toStringBinary(chunkStartKey) : "null", + // chunkEndKey != null ? Bytes.toStringBinary(chunkEndKey) : "null"); + // throw new RuntimeException("Intentional error throw"); return false; } - boolean isPartial = - forceFullRange && !hasMoreRows && Bytes.compareTo(chunkEndKey, mapperRegionEndKey) < 0; - buildChunkMetadataResult(results, isPartial); + boolean isPartialChunk = forceFullRange && !hasMoreRowsInRegion + && Bytes.compareTo(chunkEndKey, mapperRegionEndKey) < 0; + buildChunkMetadataResult(results, isPartialChunk); LOGGER.info("Chunk metadata being sent with startKey {}, endKey {}, forceFullRange {}", chunkStartKey, chunkEndKey, forceFullRange); - return hasMoreRows; + return hasMoreRowsInRegion; } catch (Throwable t) { LOGGER.error( @@ -219,11 +221,7 @@ private long calculateRowSize(List cells) { } private boolean willExceedChunkLimits(long rowSize) { - // Check if adding this row would exceed size limit - if (currentChunkSize + rowSize > chunkSizeBytes) { - return true; - } - return currentChunkRowCount + 1 > chunkMaxRows; + return currentChunkSize + rowSize > chunkSizeBytes; } /** @@ -292,7 +290,6 @@ private byte[] encodeDigestState(SHA256Digest digest) throws IOException { * @throws IOException if state is invalid, corrupted, or security checks fail */ private SHA256Digest decodeDigestState(byte[] encodedState) throws IOException { - // Why always > 4 ? if (encodedState == null) { String regionName = region.getRegionInfo().getRegionNameAsString(); String tableName = region.getRegionInfo().getTable().getNameAsString(); @@ -303,13 +300,14 @@ private SHA256Digest decodeDigestState(byte[] encodedState) throws IOException { DataInputStream dis = new DataInputStream(new ByteArrayInputStream(encodedState)); int stateLength = dis.readInt(); - // Prevent malicious large allocations, hash digest can never go beyond ~96 bytes - if (stateLength > MAX_SHA256_STATE_SIZE) { + // Prevent malicious large allocations, hash digest can never go beyond ~96 bytes, giving some + // buffer upto 128 Bytes + if (stateLength > MAX_SHA256_DIGEST_STATE_SIZE) { String regionName = region.getRegionInfo().getRegionNameAsString(); String tableName = region.getRegionInfo().getTable().getNameAsString(); throw new IllegalArgumentException( String.format("Invalid SHA256 state length in region %s table %s: %d expected <= %d", - regionName, tableName, stateLength, MAX_SHA256_STATE_SIZE)); + regionName, tableName, stateLength, MAX_SHA256_DIGEST_STATE_SIZE)); } byte[] state = new byte[stateLength]; dis.readFully(state); @@ -317,13 +315,17 @@ private SHA256Digest decodeDigestState(byte[] encodedState) throws IOException { } /** - * Builds chunk metadata result cells and adds them to the results list. Returns a single "row" - * with multiple cells containing chunk metadata. 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 isPartial true if this is a partial chunk (region boundary reached before completion) + * Builds chunk metadata result cells and adds them to the results list. Returns a single + * "row"[rowkey=chunkStartKey] with multiple cells containing chunk metadata[chunkEndKey, + * hash/digest, rowCount, hasMoreRowsInRegion, 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 isPartialChunk true if this is a partial chunk (region boundary reached before + * completion) */ - private void buildChunkMetadataResult(List results, boolean isPartial) throws IOException { + private void buildChunkMetadataResult(List results, boolean isPartialChunk) + throws IOException { byte[] resultRowKey = this.chunkStartKey; results.add(PhoenixKeyValueUtil.newKeyValue(resultRowKey, CHUNK_METADATA_FAMILY, BaseScannerRegionObserverConstants.SYNC_TABLE_END_KEY_QUALIFIER, AGG_TIMESTAMP, chunkEndKey)); @@ -331,14 +333,14 @@ private void buildChunkMetadataResult(List results, boolean isPartial) thr BaseScannerRegionObserverConstants.SYNC_TABLE_ROW_COUNT_QUALIFIER, AGG_TIMESTAMP, Bytes.toBytes(currentChunkRowCount))); results.add(PhoenixKeyValueUtil.newKeyValue(resultRowKey, CHUNK_METADATA_FAMILY, - BaseScannerRegionObserverConstants.SYNC_TABLE_HAS_MORE_ROWS_QUALIFIER, AGG_TIMESTAMP, - Bytes.toBytes(hasMoreRows))); - if (isPartial) { + BaseScannerRegionObserverConstants.SYNC_TABLE_HAS_MORE_ROWS_IN_REGION_QUALIFIER, + AGG_TIMESTAMP, Bytes.toBytes(hasMoreRowsInRegion))); + if (isPartialChunk) { // Partial chunk digest SHA256Digest cloned = new SHA256Digest(digest); byte[] digestState = encodeDigestState(cloned); results.add(PhoenixKeyValueUtil.newKeyValue(resultRowKey, CHUNK_METADATA_FAMILY, - BaseScannerRegionObserverConstants.SYNC_TABLE_IS_PARTIAL_QUALIFIER, AGG_TIMESTAMP, + 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)); @@ -349,7 +351,7 @@ private void buildChunkMetadataResult(List results, boolean isPartial) thr 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_QUALIFIER, AGG_TIMESTAMP, + BaseScannerRegionObserverConstants.SYNC_TABLE_IS_PARTIAL_CHUNK_QUALIFIER, AGG_TIMESTAMP, FALSE_BYTES)); } } @@ -368,9 +370,4 @@ public void close() throws IOException { public long getChunkSizeBytes() { return chunkSizeBytes; } - - @VisibleForTesting - public long getChunkMaxRows() { - return chunkMaxRows; - } } diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java index 5326ffe883f..1e945cd6b68 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java @@ -147,24 +147,17 @@ private List filterCompletedSplits(List allSplits, PhoenixInputSplit split = (PhoenixInputSplit) allSplits.get(splitIdx); KeyRange splitRange = split.getKeyRange(); KeyRange completedRange = completedRegions.get(completedIdx); - - // Normalize boundaries (null becomes empty byte array) byte[] splitStart = normalizeKey(splitRange.getLowerRange()); byte[] splitEnd = normalizeKey(splitRange.getUpperRange()); byte[] completedStart = normalizeKey(completedRange.getLowerRange()); byte[] completedEnd = normalizeKey(completedRange.getUpperRange()); - // Completed region ends before split starts if (Bytes.compareTo(completedEnd, splitStart) <= 0) { completedIdx++; - } - // Completed region starts after split ends - else if (Bytes.compareTo(completedStart, splitEnd) >= 0) { + } else if (Bytes.compareTo(completedStart, splitEnd) >= 0) { unprocessedSplits.add(allSplits.get(splitIdx)); splitIdx++; - } - // Overlap exists - check if split fully contained in completed - else { + } else { // Split is fully contained if: completedStart <= splitStart AND splitEnd <= completedEnd boolean startContained = Bytes.compareTo(completedStart, splitStart) <= 0; boolean endContained = Bytes.compareTo(splitEnd, completedEnd) <= 0; diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java index f5be5dae380..347cf0a2d4a 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java @@ -73,7 +73,6 @@ public enum SyncCounters { private Long toTime; private boolean isDryRun; private long chunkSizeBytes; - private long chunkMaxRows; private Configuration conf; private Connection sourceConnection; private Connection targetConnection; @@ -97,7 +96,6 @@ protected void setup(Context context) throws InterruptedException { toTime = PhoenixConfigurationUtil.getPhoenixSyncTableToTime(conf); isDryRun = PhoenixConfigurationUtil.getPhoenixSyncTableDryRun(conf); chunkSizeBytes = PhoenixConfigurationUtil.getPhoenixSyncTableChunkSizeBytes(conf); - chunkMaxRows = PhoenixConfigurationUtil.getPhoenixSyncTableChunkMaxRows(conf); extractRegionBoundariesFromSplit(context); sourceConnection = ConnectionUtil.getInputConnection(conf); pTable = sourceConnection.unwrap(PhoenixConnection.class).getTable(tableName); @@ -157,8 +155,6 @@ private Connection createGlobalConnection(Configuration conf) throws SQLExceptio @Override protected void map(NullWritable key, DBInputFormat.NullDBWritable value, Context context) throws IOException, InterruptedException { - LOGGER.info("Processing mapper region: {} to {}", Bytes.toStringBinary(mapperRegionStart), - Bytes.toStringBinary(mapperRegionEnd)); context.getCounter(PhoenixJobCounters.INPUT_RECORDS).increment(1); try { List processedChunks = @@ -225,6 +221,33 @@ private void processChunkRange(byte[] rangeStart, byte[] rangeEnd, Context conte byte[] currentStartKey = rangeStart; boolean isEntireMapperRegionChunked = false; boolean isStartKeyInclusive = true; + + // INNER LOOP: Processes all chunks within a single HBase region. + // OUTER LOOP: Processes all chunks across HBase region for given rangeStart, rangeEnd. + // Would be used if source region splits while mapper is running + // + // + // Region boundary detection via null return: + // - When the scanner has more data in the current region: returns valid ChunkInfo and + // hasMoreRowsInRegion=true + // - When the scanner reaches the END of the current region: returns NULL + // + // We cannot predict in advance whether sourceChunk.endKey marks the actual region boundary, + // because, since rangeEnd is exclusive and we can't compare it with actual returned + // sourceChunk.endKey + // + // Therefore, we ALWAYS make one extra scanner call per region. This final call returns null, + // which is our definitive signal that: + // - No more rows exist in the current region + // - We need to move to the next region (outer loop iteration) + // - The region has been fully processed + // + // Example: If an initial Mapper region boundary contained 5 chunks(5GB of data), and it was + // split across + // two source regions, 3 chunk in region-1 and 2 chunk in region-2. + // We would make 4 calls to region-1 scanner and 3 calls to region-2 scanner + // The last to any source region scanner would always returns null + // while (!isEntireMapperRegionChunked) { boolean isRegionChunked = false; try (ChunkScannerContext sourceScanner = createChunkScanner(sourceConnection, currentStartKey, @@ -256,7 +279,7 @@ private void processChunkRange(byte[] rangeStart, byte[] rangeEnd, Context conte context.progress(); currentStartKey = sourceChunk.endKey; isStartKeyInclusive = false; - if (!sourceChunk.hasMoreRows) { + if (!sourceChunk.hasMoreRowsInRegion) { isRegionChunked = true; } } @@ -272,7 +295,9 @@ private void processChunkRange(byte[] rangeStart, byte[] rangeEnd, Context conte /** * Scans target across multiple regions and returns a single combined ChunkInfo. Handles partial * chunks by passing digest state to next scanner via scan attributes, enabling cross-region - * digest continuation. + * digest continuation. Since we are scanning rows based on source chunk boundary, it could be + * distributed across multiple target regions. We keep on creating scanner across target region + * until entire source chunk boundary is processed or chunk is null * @param conn Target connection * @param startKey Source chunk start key * @param endKey Source chunk end key @@ -280,13 +305,12 @@ private void processChunkRange(byte[] rangeStart, byte[] rangeEnd, Context conte */ private ChunkInfo getTargetChunkWithSourceBoundary(Connection conn, byte[] startKey, byte[] endKey) throws IOException, SQLException { - ChunkInfo targetChunk = new ChunkInfo(); - targetChunk.startKey = startKey; - targetChunk.endKey = endKey; - targetChunk.hash = new byte[0]; - targetChunk.rowCount = 0; - targetChunk.isPartial = false; - + ChunkInfo combinedTargetChunk = new ChunkInfo(); + combinedTargetChunk.startKey = startKey; + combinedTargetChunk.endKey = endKey; + combinedTargetChunk.hash = new byte[0]; + combinedTargetChunk.rowCount = 0; + combinedTargetChunk.isPartial = false; byte[] currentStartKey = startKey; byte[] continuedDigestState = null; boolean isStartKeyInclusive = true; @@ -297,18 +321,18 @@ private ChunkInfo getTargetChunkWithSourceBoundary(Connection conn, byte[] start if (chunk == null) { break; } - targetChunk.rowCount += chunk.rowCount; + combinedTargetChunk.rowCount += chunk.rowCount; if (chunk.isPartial) { continuedDigestState = chunk.hash; currentStartKey = chunk.endKey; isStartKeyInclusive = false; } else { - targetChunk.hash = chunk.hash; + combinedTargetChunk.hash = chunk.hash; currentStartKey = null; } } } - return targetChunk; + return combinedTargetChunk; } /** @@ -335,13 +359,11 @@ private ChunkScannerContext createChunkScanner(Connection conn, byte[] startKey, Scan scan = createChunkScan(startKey, endKey, isStartKeyInclusive, isEndKeyInclusive); scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CHUNK_FORMATION, TRUE_BYTES); scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_MAPPER_REGION_END_KEY, endKey); - scan.setAttribute(BaseScannerRegionObserverConstants.SKIP_REGION_BOUNDARY_CHECK, TRUE_BYTES); + // scan.setAttribute(BaseScannerRegionObserverConstants.SKIP_REGION_BOUNDARY_CHECK, TRUE_BYTES); scan.setAttribute(BaseScannerRegionObserverConstants.UNGROUPED_AGG, TRUE_BYTES); if (continuedDigestState != null && continuedDigestState.length > 0) { scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CONTINUED_DIGEST_STATE, continuedDigestState); - LOGGER.info("Creating scanner with continued digest state ({} bytes) for range [{}, {})", - continuedDigestState.length, Bytes.toStringBinary(startKey), Bytes.toStringBinary(endKey)); } if (forceFullRange) { @@ -349,8 +371,6 @@ private ChunkScannerContext createChunkScanner(Connection conn, byte[] startKey, } else { scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CHUNK_SIZE_BYTES, Bytes.toBytes(chunkSizeBytes)); - scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CHUNK_MAX_ROWS, - Bytes.toBytes(chunkMaxRows)); } ResultScanner scanner = hTable.getScanner(scan); return new ChunkScannerContext(hTable, scanner); @@ -363,33 +383,34 @@ private ChunkScannerContext createChunkScanner(Connection conn, byte[] startKey, */ private ChunkInfo parseChunkInfo(Result result) { List cells = Arrays.asList(result.rawCells()); - // Note: startKey is in the row key, not as a separate cell (optimization to avoid duplication) + // RowKey would be chunk start key Cell endKeyCell = MetaDataUtil.getCell(cells, BaseScannerRegionObserverConstants.SYNC_TABLE_END_KEY_QUALIFIER); Cell rowCountCell = MetaDataUtil.getCell(cells, BaseScannerRegionObserverConstants.SYNC_TABLE_ROW_COUNT_QUALIFIER); - Cell isPartialCell = MetaDataUtil.getCell(cells, - BaseScannerRegionObserverConstants.SYNC_TABLE_IS_PARTIAL_QUALIFIER); + Cell isPartialChunkCell = MetaDataUtil.getCell(cells, + BaseScannerRegionObserverConstants.SYNC_TABLE_IS_PARTIAL_CHUNK_QUALIFIER); Cell hashCell = MetaDataUtil.getCell(cells, BaseScannerRegionObserverConstants.SYNC_TABLE_HASH_QUALIFIER); - Cell hasMoreRowsCell = MetaDataUtil.getCell(cells, - BaseScannerRegionObserverConstants.SYNC_TABLE_HAS_MORE_ROWS_QUALIFIER); + Cell hasMoreRowsInRegionCell = MetaDataUtil.getCell(cells, + BaseScannerRegionObserverConstants.SYNC_TABLE_HAS_MORE_ROWS_IN_REGION_QUALIFIER); if ( - endKeyCell == null || rowCountCell == null || isPartialCell == null || hashCell == null - || hasMoreRowsCell == null + endKeyCell == null || rowCountCell == null || isPartialChunkCell == null || hashCell == null + || hasMoreRowsInRegionCell == null ) { throw new RuntimeException("Missing required chunk metadata cells."); } ChunkInfo info = new ChunkInfo(); - info.startKey = result.getRow(); // Read from row key instead of cell value + info.startKey = result.getRow(); info.endKey = CellUtil.cloneValue(endKeyCell); info.rowCount = Bytes.toLong(rowCountCell.getValueArray(), rowCountCell.getValueOffset(), rowCountCell.getValueLength()); - info.isPartial = isPartialCell.getValueArray()[isPartialCell.getValueOffset()] != 0; + info.isPartial = isPartialChunkCell.getValueArray()[isPartialChunkCell.getValueOffset()] != 0; info.hash = CellUtil.cloneValue(hashCell); - info.hasMoreRows = hasMoreRowsCell.getValueArray()[hasMoreRowsCell.getValueOffset()] != 0; + info.hasMoreRowsInRegion = + hasMoreRowsInRegionCell.getValueArray()[hasMoreRowsInRegionCell.getValueOffset()] != 0; return info; } @@ -507,16 +528,12 @@ public List> calculateUnprocessedRanges(byte[] mapperRegion protected void cleanup(Context context) throws IOException, InterruptedException { tryClosingResources(); super.cleanup(context); - LOGGER.info("Mapper cleanup complete - Chunk Verified: {}, Mismatched: {}", - context.getCounter(SyncCounters.CHUNKS_VERIFIED).getValue(), - context.getCounter(SyncCounters.CHUNKS_MISMATCHED).getValue()); } private void tryClosingResources() { if (sourceConnection != null) { try { sourceConnection.close(); - LOGGER.info("Source connection closed"); } catch (SQLException e) { LOGGER.error("Error while closing source connection in PhoenixSyncTableMapper", e); } @@ -524,7 +541,6 @@ private void tryClosingResources() { if (targetConnection != null) { try { targetConnection.close(); - LOGGER.info("Target connection closed"); } catch (SQLException e) { LOGGER.error("Error while closing target connection in PhoenixSyncTableMapper", e); } @@ -532,7 +548,6 @@ private void tryClosingResources() { if (globalConnection != null) { try { globalConnection.close(); - LOGGER.info("Output connection closed"); } catch (SQLException e) { LOGGER.error("Error while closing output connection in PhoenixSyncTableMapper", e); } @@ -548,14 +563,14 @@ private static class ChunkInfo { byte[] hash; long rowCount; boolean isPartial; - boolean hasMoreRows; + boolean hasMoreRowsInRegion; Timestamp executionStartTime; @Override public String toString() { - return String.format("Chunk[start=%s, end=%s, rows=%d, partial=%s, hasMoreRows=%s]", + return String.format("Chunk[start=%s, end=%s, rows=%d, partial=%s, hasMoreRowsInRegion=%s]", Bytes.toStringBinary(startKey), Bytes.toStringBinary(endKey), rowCount, isPartial, - hasMoreRows); + hasMoreRowsInRegion); } } diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java index f1cc51435ec..a2ecb8e537b 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java @@ -33,10 +33,9 @@ import org.slf4j.LoggerFactory; /** - * Repository for managing the PHOENIX_SYNC_TABLE_OUTPUT table. This table stores checkpoint - * information for the PhoenixSyncTableTool, enabling: 1. Job-level checkpointing (skip completed - * mapper regions on restart) 2. Mapper-level checkpointing (skip completed chunks within a region) - * 3. Audit trail of all sync operations + * Repository for managing the PHOENIX_SYNC_TABLE_CHECKPOINT table. This table stores checkpoint + * information for the PhoenixSyncTableTool, enabling: 1. Mapper Level checkpointing (skip completed + * mapper regions on restart) 2. Chunk level checkpointing (skip completed chunks) */ public class PhoenixSyncTableOutputRepository { @@ -48,19 +47,13 @@ public class PhoenixSyncTableOutputRepository { private static final byte[] EMPTY_START_KEY_SENTINEL = new byte[] { 0x00 }; /** - * Constructor * @param connection Phoenix connection */ public PhoenixSyncTableOutputRepository(Connection connection) { this.connection = connection; } - /** - * Creates the PHOENIX_SYNC_TABLE_OUTPUT table if it doesn't exist. Table schema: - Primary key: - * (TABLE_NAME, TARGET_CLUSTER, FROM_TIME, TO_TIME, ENTRY_TYPE, START_ROW_KEY, END_ROW_KEY) - TTL: - * 30 days - Salt buckets: 4 (for better distribution) - */ - public void createOutputTableIfNotExists() throws SQLException { + public void createSyncCheckpointTableIfNotExists() throws SQLException { String ddl = "CREATE TABLE IF NOT EXISTS " + SYNC_TABLE_CHECKPOINT_TABLE_NAME + " (\n" + " TABLE_NAME VARCHAR NOT NULL,\n" + " TARGET_CLUSTER VARCHAR NOT NULL,\n" + " ENTRY_TYPE VARCHAR(20) NOT NULL,\n" + " FROM_TIME BIGINT NOT NULL,\n" @@ -81,55 +74,6 @@ public void createOutputTableIfNotExists() throws SQLException { } } - /** - * Logs a chunk-level sync result to the output table. - * @param tableName Source table name - * @param targetCluster Target cluster ZK quorum - * @param fromTime Start timestamp for sync (nullable) - * @param toTime End timestamp for sync (nullable) - * @param isDryRun Whether this is a dry run - * @param startKey Chunk start row key - * @param endKey Chunk end row key - * @param status Sync status (IN_PROGRESS/VERIFIED) - * @param executionStartTime When chunk processing started - * @param executionEndTime When chunk processing completed - */ - // public void logChunkResult(String tableName, String targetCluster, Long fromTime, Long toTime, - // boolean isDryRun, byte[] startKey, byte[] endKey, Status status, - // Timestamp executionStartTime, Timestamp executionEndTime) - // throws SQLException { - // - // checkpointSyncTableResult(tableName, targetCluster, Type.CHUNK, fromTime, toTime, isDryRun, - // startKey, endKey, status, executionStartTime, executionEndTime); - // } - - /** - * Logs a mapper region completion to the output table. This indicates that all chunks within the - * region have been processed. - * @param tableName Source table name - * @param targetCluster Target cluster ZK quorum - * @param fromTime Start timestamp for sync (nullable) - * @param toTime End timestamp for sync (nullable) - * @param isDryRun Whether this is a dry run - * @param regionStart Mapper region start row key - * @param regionEnd Mapper region end row key - * @param status Overall status for the region - * @param executionStartTime When mapper started processing this region - * @param executionEndTime When mapper finished processing this region - */ - // public void logMapperRegionResult(String tableName, String targetCluster, Long fromTime, - // Long toTime, boolean isDryRun, byte[] regionStart, byte[] regionEnd, Status status, - // Timestamp executionStartTime, Timestamp executionEndTime) - // throws SQLException { - // - // checkpointSyncTableResult(tableName, targetCluster, Type.MAPPER_REGION, fromTime, toTime, - // isDryRun, - // regionStart, regionEnd, status, executionStartTime, executionEndTime); - // } - - /** - * Internal method to log sync results to the output table - */ public void checkpointSyncTableResult(String tableName, String targetCluster, Type type, Long fromTime, Long toTime, boolean isDryRun, byte[] startKey, byte[] endKey, Status status, Timestamp executionStartTime, Timestamp executionEndTime, String counters) throws SQLException { @@ -148,15 +92,13 @@ public void checkpointSyncTableResult(String tableName, String targetCluster, Ty throw new IllegalArgumentException("FromTime and ToTime cannot be null for checkpoint"); } - String upsert = "UPSERT INTO " + SYNC_TABLE_CHECKPOINT_TABLE_NAME + " (" - + "TABLE_NAME, TARGET_CLUSTER, ENTRY_TYPE, FROM_TIME, TO_TIME, IS_DRY_RUN, " - + "START_ROW_KEY, END_ROW_KEY, IS_FIRST_REGION, EXECUTION_START_TIME, EXECUTION_END_TIME, " - + "STATUS, COUNTERS) " + "VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)"; + String upsert = "UPSERT INTO " + SYNC_TABLE_CHECKPOINT_TABLE_NAME + + " (TABLE_NAME, TARGET_CLUSTER, ENTRY_TYPE, FROM_TIME, TO_TIME, IS_DRY_RUN," + + " START_ROW_KEY, END_ROW_KEY, IS_FIRST_REGION, EXECUTION_START_TIME, EXECUTION_END_TIME," + + " STATUS, COUNTERS) " + "VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)"; byte[] effectiveStartKey = (startKey == null || startKey.length == 0) ? EMPTY_START_KEY_SENTINEL : startKey; - - // Determine if this is the first region (empty start key) boolean isFirstRegion = (startKey == null || startKey.length == 0); try (PreparedStatement ps = connection.prepareStatement(upsert)) { @@ -179,15 +121,8 @@ public void checkpointSyncTableResult(String tableName, String targetCluster, Ty } /** - * Helper to check if a key represents an empty boundary - */ - // private boolean isEmptyBoundary(byte[] key) { - // return key != null && key.length == 1 && Bytes.equals(key, HConstants.EMPTY_BYTE_ARRAY); - // } - - /** - * Converts stored key back to HBase empty key if needed. For first region, converts - * EMPTY_START_KEY_SENTINEL back to HConstants.EMPTY_BYTE_ARRAY. + * Converts stored key back to HBase empty key if needed. For first region(empty startKey), + * converts EMPTY_START_KEY_SENTINEL back to HConstants.EMPTY_BYTE_ARRAY. */ private byte[] toHBaseKey(byte[] storedKey, boolean isFirstRegion) { if (isFirstRegion && Arrays.equals(storedKey, EMPTY_START_KEY_SENTINEL)) { @@ -213,7 +148,6 @@ public List getProcessedMapperRegions(String tableNam + " AND ENTRY_TYPE = ? AND FROM_TIME = ? AND TO_TIME = ? AND STATUS IN ( ?, ?)" + " ORDER BY START_ROW_KEY "; List results = new ArrayList<>(); - try (PreparedStatement ps = connection.prepareStatement(query)) { int paramIndex = 1; ps.setString(paramIndex++, tableName); @@ -251,8 +185,8 @@ public List getProcessedChunks(String tableName, Stri Long fromTime, Long toTime, byte[] mapperRegionStart, byte[] mapperRegionEnd) throws SQLException { String query = "SELECT START_ROW_KEY, END_ROW_KEY, IS_FIRST_REGION FROM " - + SYNC_TABLE_CHECKPOINT_TABLE_NAME + " WHERE TABLE_NAME = ? " + " AND TARGET_CLUSTER = ? " - + " " + " AND ENTRY_TYPE = ? AND FROM_TIME = ? AND TO_TIME = ? AND START_ROW_KEY < ? " + + SYNC_TABLE_CHECKPOINT_TABLE_NAME + " WHERE TABLE_NAME = ? AND TARGET_CLUSTER = ? " + + " AND ENTRY_TYPE = ? AND FROM_TIME = ? AND TO_TIME = ? AND START_ROW_KEY < ? " + " AND END_ROW_KEY > ? AND STATUS IN (?, ?)" + " ORDER BY START_ROW_KEY"; List results = new ArrayList<>(); @@ -279,17 +213,4 @@ public List getProcessedChunks(String tableName, Stri } return results; } - - // /** - // * For testing: clears all entries for a given table - // */ - // @VisibleForTesting - // public void deleteEntriesForTable(String tableName) throws SQLException { - // String delete = "DELETE FROM " + OUTPUT_TABLE_NAME + " WHERE TABLE_NAME = ?"; - // try (PreparedStatement ps = connection.prepareStatement(delete)) { - // ps.setString(1, tableName); - // ps.executeUpdate(); - // connection.commit(); - // } - // } } diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java index ad71d3f910d..4af09110ec8 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java @@ -21,8 +21,7 @@ import org.apache.hadoop.hbase.util.Bytes; /** - * Data model class representing a row in the PHOENIX_SYNC_TABLE_OUTPUT table. Used for - * checkpointing sync job progress at both mapper region and chunk levels. + * Data model class representing a row in the PHOENIX_SYNC_TABLE_CHECKPOINT table */ public class PhoenixSyncTableOutputRow { diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java index b581326129a..c284a37ccb0 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java @@ -326,7 +326,7 @@ public void populateSyncTableToolAttributes(CommandLine cmdLine) throws Exceptio */ private void createSyncOutputTable(Connection connection) throws SQLException { PhoenixSyncTableOutputRepository repository = new PhoenixSyncTableOutputRepository(connection); - repository.createOutputTableIfNotExists(); + repository.createSyncCheckpointTableIfNotExists(); } /** diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java index 18837b82963..6e6bda71fff 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java @@ -240,9 +240,7 @@ public final class PhoenixConfigurationUtil { public static final String PHOENIX_SYNC_TABLE_CHUNK_MAX_ROWS = "phoenix.sync.table.chunk.max.rows"; - public static final long DEFAULT_PHOENIX_SYNC_TABLE_CHUNK_SIZE_BYTES = 100 * 1024 * 1024; // 100MB - - public static final int DEFAULT_PHOENIX_SYNC_TABLE_CHUNK_MAX_ROWS = 100000; + public static final long DEFAULT_PHOENIX_SYNC_TABLE_CHUNK_SIZE_BYTES = 1024 * 1024 * 1024; // 1GB /** * Determines type of Phoenix Map Reduce job. 1. QUERY allows running arbitrary queries without @@ -1024,17 +1022,4 @@ public static long getPhoenixSyncTableChunkSizeBytes(Configuration conf) { return conf.getLong(PHOENIX_SYNC_TABLE_CHUNK_SIZE_BYTES, DEFAULT_PHOENIX_SYNC_TABLE_CHUNK_SIZE_BYTES); } - - public static void setPhoenixSyncTableChunkMaxRows(Configuration conf, Integer chunkMaxRows) { - Preconditions.checkNotNull(conf); - if (chunkMaxRows != null) { - conf.setInt(PHOENIX_SYNC_TABLE_CHUNK_MAX_ROWS, chunkMaxRows); - } - } - - public static int getPhoenixSyncTableChunkMaxRows(Configuration conf) { - Preconditions.checkNotNull(conf); - return conf.getInt(PHOENIX_SYNC_TABLE_CHUNK_MAX_ROWS, - DEFAULT_PHOENIX_SYNC_TABLE_CHUNK_MAX_ROWS); - } } From 9fe6d3495137ab978a26c95ec905c5993cef1afb Mon Sep 17 00:00:00 2001 From: Rahul Kumar Date: Sat, 14 Feb 2026 00:17:51 +0530 Subject: [PATCH 3/9] Self review checkpoint#2 --- .../BaseScannerRegionObserverConstants.java | 6 +- .../PhoenixSyncTableRegionScanner.java | 58 ++--- .../PhoenixSyncTableInputFormat.java | 22 +- .../mapreduce/PhoenixSyncTableMapper.java | 227 ++++++++---------- .../PhoenixSyncTableOutputRepository.java | 13 +- .../mapreduce/PhoenixSyncTableOutputRow.java | 2 +- .../mapreduce/PhoenixSyncTableTool.java | 46 ++-- .../util/PhoenixConfigurationUtil.java | 5 +- 8 files changed, 161 insertions(+), 218 deletions(-) diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/BaseScannerRegionObserverConstants.java b/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/BaseScannerRegionObserverConstants.java index 51066d98f46..cf5e10a769d 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/BaseScannerRegionObserverConstants.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/BaseScannerRegionObserverConstants.java @@ -197,10 +197,8 @@ public static long getMaxLookbackInMillis(Configuration conf) { /** * PhoenixSyncTableTool scan attributes for server-side chunk formation and checksum */ - public static final String SYNC_TABLE_CHUNK_FORMATION = "_SyncTableChunk"; - public static final String SYNC_TABLE_MAPPER_REGION_END_KEY = "_SyncTableMapperRegionEnd"; + public static final String SYNC_TABLE_CHUNK_FORMATION = "_SyncTableChunkFormation"; public static final String SYNC_TABLE_CHUNK_SIZE_BYTES = "_SyncTableChunkSizeBytes"; - public static final String SYNC_TABLE_FORCE_FULL_RANGE = "_SyncTableForceFullRange"; public static final String SYNC_TABLE_CONTINUED_DIGEST_STATE = "_SyncTableContinuedDigestState"; /** @@ -214,8 +212,6 @@ public static long getMaxLookbackInMillis(Configuration conf) { public static final byte[] SYNC_TABLE_ROW_COUNT_QUALIFIER = Bytes.toBytes("ROW_COUNT"); public static final byte[] SYNC_TABLE_IS_PARTIAL_CHUNK_QUALIFIER = Bytes.toBytes("IS_PARTIAL_CHUNK"); - public static final byte[] SYNC_TABLE_HAS_MORE_ROWS_IN_REGION_QUALIFIER = - Bytes.toBytes("HAS_MORE_ROWS_IN_REGION"); /** Exposed for testing */ public static final String SCANNER_OPENED_TRACE_INFO = "Scanner opened on server"; diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java index cca8cec5f0b..13202048d3c 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java @@ -51,17 +51,16 @@ * Server-side coprocessor that performs chunk formation and SHA-256 hashing for * PhoenixSyncTableTool. *

- * Accumulates rows into chunks (based on size/row limits) and computes a hash of all row data + * Accumulates rows into chunks (based on size limits) and computes a hash of all row data * (keys, column families, qualifiers, timestamps, cell types, values). *

- * Source mode (forceFullRange=false): Returns complete chunks bounded by region boundaries. Sets - * hasMoreRowsInRegion=false when region is exhausted. + * Source scan (isTargetScan=false): Returns complete chunks bounded by region boundaries. Sets + * hasMoreRows=false when region is exhausted. *

- * Target mode (forceFullRange=true): Returns partial chunks with serialized digest state when + * Target scan (isTargetScan=true): Returns partial chunks with serialized digest state when * region boundary is reached, allowing cross-region hash continuation. *

- * Returns chunk metadata cells: END_KEY, HASH (or digest state), ROW_COUNT, IS_PARTIAL_CHUNK, - * HAS_MORE_ROWS_IN_REGION. + * Returns chunk metadata cells: END_KEY, HASH (or digest state), ROW_COUNT, IS_PARTIAL_CHUNK */ public class PhoenixSyncTableRegionScanner extends BaseRegionScanner { @@ -72,16 +71,14 @@ public class PhoenixSyncTableRegionScanner extends BaseRegionScanner { private final Scan scan; private final RegionCoprocessorEnvironment env; private final UngroupedAggregateRegionObserver ungroupedAggregateRegionObserver; - - private final byte[] mapperRegionEndKey; private final long chunkSizeBytes; - private final boolean forceFullRange; + private boolean isTargetScan = false; private byte[] chunkStartKey = null; private byte[] chunkEndKey = null; private long currentChunkSize = 0L; private long currentChunkRowCount = 0L; private SHA256Digest digest; - private boolean hasMoreRowsInRegion = true; + private boolean hasMoreRows = true; private boolean isUsingContinuedDigest; // If target chunk was partial, and we are continuing to // update digest before calculating checksum @@ -101,18 +98,15 @@ public PhoenixSyncTableRegionScanner(final RegionScanner innerScanner, final Reg this.scan = scan; this.env = env; this.ungroupedAggregateRegionObserver = ungroupedAggregateRegionObserver; - - byte[] mapperEndAttr = - scan.getAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_MAPPER_REGION_END_KEY); byte[] chunkSizeAttr = scan.getAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CHUNK_SIZE_BYTES); - this.mapperRegionEndKey = mapperEndAttr != null ? mapperEndAttr : new byte[0]; + 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; - byte[] forceFullRangeAttr = - scan.getAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_FORCE_FULL_RANGE); - this.forceFullRange = (forceFullRangeAttr != null && Bytes.toBoolean(forceFullRangeAttr)); + // Check if we should continue from a previous digest state (cross-region continuation) byte[] continuedDigestStateAttr = scan.getAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CONTINUED_DIGEST_STATE); @@ -142,11 +136,10 @@ public boolean next(List results) throws IOException { RegionScanner localScanner = delegate; synchronized (localScanner) { List rowCells = new ArrayList<>(); - while (hasMoreRowsInRegion) { - // Check region state INSIDE loop for long-running scans + while (hasMoreRows) { ungroupedAggregateRegionObserver.checkForRegionClosingOrSplitting(); rowCells.clear(); - hasMoreRowsInRegion = localScanner.nextRaw(rowCells); + hasMoreRows = localScanner.nextRaw(rowCells); if (rowCells.isEmpty()) { break; } @@ -154,29 +147,20 @@ public boolean next(List results) throws IOException { byte[] rowKey = CellUtil.cloneRow(rowCells.get(0)); long rowSize = calculateRowSize(rowCells); addRowToChunk(rowKey, rowCells, rowSize); - if (!forceFullRange && willExceedChunkLimits(rowSize)) { + if (!isTargetScan && willExceedChunkLimits(rowSize)) { break; } } } if (chunkStartKey == null) { - // LOGGER.error( - // "Exception during chunk scanning in region {} table {} at chunk startKey: {}, endkey: - // {})", - // region.getRegionInfo().getRegionNameAsString(), - // region.getRegionInfo().getTable().getNameAsString(), - // chunkStartKey != null ? Bytes.toStringBinary(chunkStartKey) : "null", - // chunkEndKey != null ? Bytes.toStringBinary(chunkEndKey) : "null"); - // throw new RuntimeException("Intentional error throw"); return false; } - boolean isPartialChunk = forceFullRange && !hasMoreRowsInRegion - && Bytes.compareTo(chunkEndKey, mapperRegionEndKey) < 0; + // checking if this next() call was Partial chunk. Only needed for target scan. + // Will be partial chunk until chunkEndKey < source chunk endKey + boolean isPartialChunk = isTargetScan && Bytes.compareTo(chunkEndKey, scan.getStopRow()) < 0; buildChunkMetadataResult(results, isPartialChunk); - LOGGER.info("Chunk metadata being sent with startKey {}, endKey {}, forceFullRange {}", - chunkStartKey, chunkEndKey, forceFullRange); - return hasMoreRowsInRegion; + return hasMoreRows; } catch (Throwable t) { LOGGER.error( @@ -250,6 +234,7 @@ private void updateDigestWithRow(byte[] rowKey, List cells) { digest.update(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()); digest.update(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()); long ts = cell.getTimestamp(); + // Big-Endian Byte Serialization digest.update((byte) (ts >>> 56)); digest.update((byte) (ts >>> 48)); digest.update((byte) (ts >>> 40)); @@ -317,7 +302,7 @@ private SHA256Digest decodeDigestState(byte[] encodedState) throws IOException { /** * Builds chunk metadata result cells and adds them to the results list. Returns a single * "row"[rowkey=chunkStartKey] with multiple cells containing chunk metadata[chunkEndKey, - * hash/digest, rowCount, hasMoreRowsInRegion, isPartialChunk]. For complete chunks: includes + * hash/digest, rowCount, hasMoreRows, 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 @@ -332,9 +317,6 @@ private void buildChunkMetadataResult(List results, boolean isPartialChunk results.add(PhoenixKeyValueUtil.newKeyValue(resultRowKey, CHUNK_METADATA_FAMILY, BaseScannerRegionObserverConstants.SYNC_TABLE_ROW_COUNT_QUALIFIER, AGG_TIMESTAMP, Bytes.toBytes(currentChunkRowCount))); - results.add(PhoenixKeyValueUtil.newKeyValue(resultRowKey, CHUNK_METADATA_FAMILY, - BaseScannerRegionObserverConstants.SYNC_TABLE_HAS_MORE_ROWS_IN_REGION_QUALIFIER, - AGG_TIMESTAMP, Bytes.toBytes(hasMoreRowsInRegion))); if (isPartialChunk) { // Partial chunk digest SHA256Digest cloned = new SHA256Digest(digest); diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java index 1e945cd6b68..eeba1e49cce 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java @@ -56,11 +56,12 @@ public PhoenixSyncTableInputFormat() { *

* PhoenixSyncTableMapper doesn't need actual row data from the RecordReader - it extracts region * boundaries from the InputSplit and delegates all scanning to the PhoenixSyncTableRegionScanner - * coprocessor. Using PhoenixNoOpSingleRecordReader ensures that {@code map()} is called exactly + * coprocessor. + * Using PhoenixNoOpSingleRecordReader ensures that {@code map()} is called exactly * once per region no matter what scan looks like, avoiding the overhead of the default * PhoenixRecordReader which would call {@code map()} for every row of scan. * @param split Input Split - * @return A SingleRecordReader instance + * @return A PhoenixNoOpSingleRecordReader instance */ @SuppressWarnings("rawtypes") @Override @@ -69,8 +70,8 @@ public RecordReader createRecordReader(InputSplit split, TaskAttemptContext cont } /** - * Generates InputSplits for the sync job, filtering out already-completed regions using synn - * checkpoint table. + * Generates InputSplits for the Phoenix sync table job, splits are done based on region boundary + * and then filter out already-completed regions using sync table checkpoint table. */ @Override public List getSplits(JobContext context) throws IOException, InterruptedException { @@ -79,7 +80,6 @@ public List getSplits(JobContext context) throws IOException, Interr String targetZkQuorum = PhoenixConfigurationUtil.getPhoenixSyncTableTargetZkQuorum(conf); Long fromTime = PhoenixConfigurationUtil.getPhoenixSyncTableFromTime(conf); Long toTime = PhoenixConfigurationUtil.getPhoenixSyncTableToTime(conf); - List allSplits = super.getSplits(context); if (allSplits == null || allSplits.isEmpty()) { throw new IOException(String.format( @@ -96,10 +96,8 @@ public List getSplits(JobContext context) throws IOException, Interr throw new RuntimeException(e); } if (completedRegions.isEmpty()) { - if (LOGGER.isDebugEnabled()) { - LOGGER.debug("No completed regions for table {} - processing all {} splits", tableName, + LOGGER.info("No completed regions for table {} - processing all {} splits", tableName, allSplits.size()); - } return allSplits; } @@ -143,6 +141,7 @@ private List filterCompletedSplits(List allSplits, List unprocessedSplits = new ArrayList<>(); int splitIdx = 0; int completedIdx = 0; + // Two pointer comparison across splitRange and completedRange while (splitIdx < allSplits.size() && completedIdx < completedRegions.size()) { PhoenixInputSplit split = (PhoenixInputSplit) allSplits.get(splitIdx); KeyRange splitRange = split.getKeyRange(); @@ -152,13 +151,18 @@ private List filterCompletedSplits(List allSplits, byte[] completedStart = normalizeKey(completedRange.getLowerRange()); byte[] completedEnd = normalizeKey(completedRange.getUpperRange()); + // No overlap b/w completedRange/splitRange. + // completedEnd is before splitStart, increment completed pointer to catch up if (Bytes.compareTo(completedEnd, splitStart) <= 0) { completedIdx++; } else if (Bytes.compareTo(completedStart, splitEnd) >= 0) { + // No overlap. completedStart is after splitEnd, splitRange needs to be processed, + // add to unprocessed list and increment unprocessedSplits.add(allSplits.get(splitIdx)); splitIdx++; } else { - // Split is fully contained if: completedStart <= splitStart AND splitEnd <= completedEnd + // Some overlap detected, check if SplitRange is fullyContained within completedRange + // Fully contained if: completedStart <= splitStart AND splitEnd <= completedEnd boolean startContained = Bytes.compareTo(completedStart, splitStart) <= 0; boolean endContained = Bytes.compareTo(splitEnd, completedEnd) <= 0; boolean fullyContained = startContained && endContained; diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java index 347cf0a2d4a..813ef39c0fd 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java @@ -64,7 +64,8 @@ public class PhoenixSyncTableMapper public enum SyncCounters { CHUNKS_VERIFIED, CHUNKS_MISMATCHED, - ROWS_PROCESSED + SOURCE_ROWS_PROCESSED, + TARGET_ROWS_PROCESSED, } private String tableName; @@ -105,7 +106,7 @@ protected void setup(Context context) throws InterruptedException { syncTableOutputRepository = new PhoenixSyncTableOutputRepository(globalConnection); } catch (SQLException | IOException e) { tryClosingResources(); - throw new RuntimeException(String.format("Failed to setup mapper for table: %s", tableName), + throw new RuntimeException(String.format("Failed to setup PhoenixSyncTableMapper for table: %s", tableName), e); } } @@ -126,14 +127,11 @@ private void extractRegionBoundariesFromSplit(Context context) { } /** - * Connects to the target cluster using the target ZK quorum, port, znode + * Connects to the target cluster using the target ZK quorum, port, znode, krb principal */ private void connectToTargetCluster() throws SQLException, IOException { Configuration targetConf = PhoenixMapReduceUtil.createConfigurationForZkQuorum(conf, targetZkQuorum); - if (toTime != null) { - targetConf.set(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(toTime)); - } targetConnection = ConnectionUtil.getInputConnection(targetConf); } @@ -162,19 +160,20 @@ protected void map(NullWritable key, DBInputFormat.NullDBWritable value, Context mapperRegionStart, mapperRegionEnd); List> unprocessedRanges = calculateUnprocessedRanges(mapperRegionStart, mapperRegionEnd, processedChunks); - // TODO: Test with delete where mapper region boundary key is not present in region for (Pair range : unprocessedRanges) { processChunkRange(range.getFirst(), range.getSecond(), context); } long mismatchedChunk = context.getCounter(SyncCounters.CHUNKS_MISMATCHED).getValue(); long verifiedChunk = context.getCounter(SyncCounters.CHUNKS_VERIFIED).getValue(); - long rowsProcessed = context.getCounter(SyncCounters.ROWS_PROCESSED).getValue(); + long sourceRowsProcessed = context.getCounter(SyncCounters.SOURCE_ROWS_PROCESSED).getValue(); + long targetRowsProcessed = context.getCounter(SyncCounters.TARGET_ROWS_PROCESSED).getValue(); Timestamp mapperEndTime = new Timestamp(System.currentTimeMillis()); Map mapperCounters = new LinkedHashMap<>(); mapperCounters.put(SyncCounters.CHUNKS_VERIFIED.name(), verifiedChunk); mapperCounters.put(SyncCounters.CHUNKS_MISMATCHED.name(), mismatchedChunk); - mapperCounters.put(SyncCounters.CHUNKS_MISMATCHED.name(), rowsProcessed); + mapperCounters.put(SyncCounters.SOURCE_ROWS_PROCESSED.name(), sourceRowsProcessed); + mapperCounters.put(SyncCounters.TARGET_ROWS_PROCESSED.name(), targetRowsProcessed); String counters = formatCounters(mapperCounters); if (mismatchedChunk == 0) { @@ -184,12 +183,12 @@ protected void map(NullWritable key, DBInputFormat.NullDBWritable value, Context mapperRegionStart, mapperRegionEnd, PhoenixSyncTableOutputRow.Status.VERIFIED, mapperStartTime, mapperEndTime, counters); LOGGER.info( - "Mapper completed with verified: {} verifiedChunk chunks, {} mismatchedChunk chunks", + "PhoenixSyncTable mapper completed with verified: {} verifiedChunk chunks, {} mismatchedChunk chunks", verifiedChunk, mismatchedChunk); } else { context.getCounter(PhoenixJobCounters.FAILED_RECORDS).increment(1); LOGGER.warn( - "Mapper completed with mismatched: {} mismatchedChunk chunks, {} verifiedChunk chunks", + "PhoenixSyncTable mapper completed with mismatch: {} verifiedChunk chunks, {} mismatchedChunk chunks", verifiedChunk, mismatchedChunk); syncTableOutputRepository.checkpointSyncTableResult(tableName, targetZkQuorum, PhoenixSyncTableOutputRow.Type.MAPPER_REGION, fromTime, toTime, isDryRun, @@ -198,16 +197,15 @@ protected void map(NullWritable key, DBInputFormat.NullDBWritable value, Context } } catch (SQLException e) { tryClosingResources(); - throw new RuntimeException("Error processing mapper", e); + throw new RuntimeException("Error processing PhoenixSyncTableMapper", e); } } /** - * Processes a chunk range by comparing source and target cluster data. Source chunking: Breaks - * data into size-based chunks within each physical region. Creates new scanner and new chunk when - * region boundary is crossed. This can happen if mapper region splits while the mapper job is - * running. Target chunking: Follows source chunk boundaries exactly. Source chunk boundary might - * be split across multiple target region, if so corpoc signal for partial chunk with partial + * Processes a chunk range by comparing source and target cluster data. + * Source chunking: Breaks data into size-based chunks within given mapper region boundary. + * Target chunking: Follows source chunk boundaries exactly. Source chunk boundary might + * be split across multiple target region, if so corpoc signals for partial chunk with partial * digest. Once entire Source chunk is covered by target scanner, we calculate resulting checksum * from combined digest. * @param rangeStart Range start key @@ -217,84 +215,49 @@ protected void map(NullWritable key, DBInputFormat.NullDBWritable value, Context * @throws SQLException if database operations fail */ private void processChunkRange(byte[] rangeStart, byte[] rangeEnd, Context context) - throws IOException, SQLException { - byte[] currentStartKey = rangeStart; - boolean isEntireMapperRegionChunked = false; + throws IOException, SQLException { boolean isStartKeyInclusive = true; - - // INNER LOOP: Processes all chunks within a single HBase region. - // OUTER LOOP: Processes all chunks across HBase region for given rangeStart, rangeEnd. - // Would be used if source region splits while mapper is running - // - // - // Region boundary detection via null return: - // - When the scanner has more data in the current region: returns valid ChunkInfo and - // hasMoreRowsInRegion=true - // - When the scanner reaches the END of the current region: returns NULL - // - // We cannot predict in advance whether sourceChunk.endKey marks the actual region boundary, - // because, since rangeEnd is exclusive and we can't compare it with actual returned - // sourceChunk.endKey - // - // Therefore, we ALWAYS make one extra scanner call per region. This final call returns null, - // which is our definitive signal that: - // - No more rows exist in the current region - // - We need to move to the next region (outer loop iteration) - // - The region has been fully processed - // - // Example: If an initial Mapper region boundary contained 5 chunks(5GB of data), and it was - // split across - // two source regions, 3 chunk in region-1 and 2 chunk in region-2. - // We would make 4 calls to region-1 scanner and 3 calls to region-2 scanner - // The last to any source region scanner would always returns null - // - while (!isEntireMapperRegionChunked) { - boolean isRegionChunked = false; - try (ChunkScannerContext sourceScanner = createChunkScanner(sourceConnection, currentStartKey, - rangeEnd, false, null, isStartKeyInclusive, false)) { - while (!isRegionChunked) { - ChunkInfo sourceChunk = sourceScanner.getNextChunk(); - if (sourceChunk == null) { - isEntireMapperRegionChunked = true; - break; - } - Timestamp chunkStartTime = new Timestamp(System.currentTimeMillis()); - sourceChunk.executionStartTime = chunkStartTime; - ChunkInfo targetChunk = getTargetChunkWithSourceBoundary(targetConnection, - sourceChunk.startKey, sourceChunk.endKey); - context.getCounter(SyncCounters.ROWS_PROCESSED).increment(sourceChunk.rowCount); - boolean matched = MessageDigest.isEqual(sourceChunk.hash, targetChunk.hash); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug("Chunk comparison {}, {}: source={} rows, target={} rows, matched={}", + try (ChunkScannerContext sourceScanner = createChunkScanner(sourceConnection, rangeStart, + rangeEnd, null, isStartKeyInclusive, false, false)) { + while (true) { + // We only try to get one chunked metadata row returned at a time until no more chunk returned(i.e null) + ChunkInfo sourceChunk = sourceScanner.getNextChunk(); + if (sourceChunk == null) { + break; + } + sourceChunk.executionStartTime = new Timestamp(System.currentTimeMillis()); + ChunkInfo targetChunk = + getTargetChunkWithSourceBoundary(targetConnection, sourceChunk.startKey, + sourceChunk.endKey); + + context.getCounter(SyncCounters.SOURCE_ROWS_PROCESSED).increment(sourceChunk.rowCount); + context.getCounter(SyncCounters.TARGET_ROWS_PROCESSED).increment(targetChunk.rowCount); + boolean matched = MessageDigest.isEqual(sourceChunk.hash, targetChunk.hash); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Chunk comparison {}, {}: source={} rows, target={} rows, matched={}", Bytes.toStringBinary(sourceChunk.startKey), Bytes.toStringBinary(sourceChunk.endKey), sourceChunk.rowCount, targetChunk.rowCount, matched); - } - - if (matched) { - handleVerifiedChunk(sourceChunk, context); - } else { - handleMismatchedChunk(sourceChunk, context); - } - - context.progress(); - currentStartKey = sourceChunk.endKey; - isStartKeyInclusive = false; - if (!sourceChunk.hasMoreRowsInRegion) { - isRegionChunked = true; - } } + sourceChunk.executionEndTime = new Timestamp(System.currentTimeMillis()); + Map mapperCounters = new LinkedHashMap<>(); + mapperCounters.put(SyncCounters.SOURCE_ROWS_PROCESSED.name(), sourceChunk.rowCount); + mapperCounters.put(SyncCounters.TARGET_ROWS_PROCESSED.name(), targetChunk.rowCount); + String counters = formatCounters(mapperCounters); + if (matched) { + handleVerifiedChunk(sourceChunk, context, counters); + } else { + handleMismatchedChunk(sourceChunk, context, counters); + } + context.progress(); } } - - if (LOGGER.isDebugEnabled()) { - LOGGER.debug("Completed processing range {}, {}", Bytes.toStringBinary(rangeStart), - Bytes.toStringBinary(rangeEnd)); - } + LOGGER.info("Completed sync table processing of Mapper region boundary {}, {}", + Bytes.toStringBinary(rangeStart), Bytes.toStringBinary(rangeEnd)); } /** - * Scans target across multiple regions and returns a single combined ChunkInfo. Handles partial - * chunks by passing digest state to next scanner via scan attributes, enabling cross-region + * Scans target across multiple regions and returns a single combined ChunkInfo. + * Handles partial chunks by passing digest state to next scanner via scan attributes, enabling cross-region * digest continuation. Since we are scanning rows based on source chunk boundary, it could be * distributed across multiple target regions. We keep on creating scanner across target region * until entire source chunk boundary is processed or chunk is null @@ -314,21 +277,32 @@ private ChunkInfo getTargetChunkWithSourceBoundary(Connection conn, byte[] start byte[] currentStartKey = startKey; byte[] continuedDigestState = null; boolean isStartKeyInclusive = true; - while (currentStartKey != null && Bytes.compareTo(currentStartKey, endKey) < 0) { - try (ChunkScannerContext scanner = createChunkScanner(conn, currentStartKey, endKey, true, - continuedDigestState, isStartKeyInclusive, true)) { + while (true) { + // We are creating a new scanner for every target region chunk. + // This chunk could be partial or full depending on whether the source region boundary is part of one or multiple target region. + // For every target region scanned, we want to have one row processed and returned back + // immediately(that's why we set scan.setLimit(1)), since output from one region partial chunk + // scanner is input to next region scanner. + try (ChunkScannerContext scanner = createChunkScanner(conn, currentStartKey, endKey, + continuedDigestState, isStartKeyInclusive, true, true)) { ChunkInfo chunk = scanner.getNextChunk(); + // In a happy path where source and target rows are matching, target chunk would never be null. + // If chunk returned null, this would mean it couldn't find last source rows in target, + // since we only return isPartial=true until target chunk end key < source chunk endKey. + // Hash would still be digest if chunk returned is null and not a checksum, so would never match(which is expected). + // We could convert the digest to checksum but since it won't match anyhow, we don't need to. if (chunk == null) { break; } combinedTargetChunk.rowCount += chunk.rowCount; + // Updating it with either digest(when isPartial) or checksum(when all rows chunked) + combinedTargetChunk.hash = chunk.hash; if (chunk.isPartial) { continuedDigestState = chunk.hash; currentStartKey = chunk.endKey; isStartKeyInclusive = false; } else { - combinedTargetChunk.hash = chunk.hash; - currentStartKey = null; + break; } } } @@ -340,35 +314,30 @@ private ChunkInfo getTargetChunkWithSourceBoundary(Connection conn, byte[] start * @param conn Connection to cluster (source or target) * @param startKey Range start key (inclusive) * @param endKey Range end key (exclusive) - * @param forceFullRange If true, coprocessor returns entire range without size-based - * breaking (for target cluster) * @param continuedDigestState If not null, coprocessor will continue hashing from this state (for * cross-region continuation on target) * @param isStartKeyInclusive Whether StartKey Inclusive * @param isEndKeyInclusive Whether EndKey Inclusive - * @return ChunkScannerContext that must be closed - * @throws IOException if scanner creation fails - * @throws SQLException if connection fails + * @return ChunkScannerContext + * @throws IOException scanner creation fails + * @throws SQLException hTable connection fails */ private ChunkScannerContext createChunkScanner(Connection conn, byte[] startKey, byte[] endKey, - boolean forceFullRange, byte[] continuedDigestState, boolean isStartKeyInclusive, - boolean isEndKeyInclusive) throws IOException, SQLException { + byte[] continuedDigestState, boolean isStartKeyInclusive, + boolean isEndKeyInclusive, boolean isTargetScan) throws IOException, SQLException { // Not using try-with-resources since ChunkScannerContext owns the table lifecycle Table hTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(physicalTableName); - Scan scan = createChunkScan(startKey, endKey, isStartKeyInclusive, isEndKeyInclusive); + Scan scan = createChunkScan(startKey, endKey, isStartKeyInclusive, isEndKeyInclusive, isTargetScan); scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CHUNK_FORMATION, TRUE_BYTES); - scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_MAPPER_REGION_END_KEY, endKey); - // scan.setAttribute(BaseScannerRegionObserverConstants.SKIP_REGION_BOUNDARY_CHECK, TRUE_BYTES); + scan.setAttribute(BaseScannerRegionObserverConstants.SKIP_REGION_BOUNDARY_CHECK, TRUE_BYTES); scan.setAttribute(BaseScannerRegionObserverConstants.UNGROUPED_AGG, TRUE_BYTES); if (continuedDigestState != null && continuedDigestState.length > 0) { scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CONTINUED_DIGEST_STATE, continuedDigestState); } - if (forceFullRange) { - scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_FORCE_FULL_RANGE, TRUE_BYTES); - } else { + if (!isTargetScan) { scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CHUNK_SIZE_BYTES, Bytes.toBytes(chunkSizeBytes)); } @@ -383,7 +352,6 @@ private ChunkScannerContext createChunkScanner(Connection conn, byte[] startKey, */ private ChunkInfo parseChunkInfo(Result result) { List cells = Arrays.asList(result.rawCells()); - // RowKey would be chunk start key Cell endKeyCell = MetaDataUtil.getCell(cells, BaseScannerRegionObserverConstants.SYNC_TABLE_END_KEY_QUALIFIER); Cell rowCountCell = MetaDataUtil.getCell(cells, @@ -392,13 +360,9 @@ private ChunkInfo parseChunkInfo(Result result) { BaseScannerRegionObserverConstants.SYNC_TABLE_IS_PARTIAL_CHUNK_QUALIFIER); Cell hashCell = MetaDataUtil.getCell(cells, BaseScannerRegionObserverConstants.SYNC_TABLE_HASH_QUALIFIER); - Cell hasMoreRowsInRegionCell = MetaDataUtil.getCell(cells, - BaseScannerRegionObserverConstants.SYNC_TABLE_HAS_MORE_ROWS_IN_REGION_QUALIFIER); - if ( - endKeyCell == null || rowCountCell == null || isPartialChunkCell == null || hashCell == null - || hasMoreRowsInRegionCell == null - ) { + if (endKeyCell == null || rowCountCell == null || isPartialChunkCell == null + || hashCell == null) { throw new RuntimeException("Missing required chunk metadata cells."); } @@ -409,14 +373,12 @@ private ChunkInfo parseChunkInfo(Result result) { rowCountCell.getValueLength()); info.isPartial = isPartialChunkCell.getValueArray()[isPartialChunkCell.getValueOffset()] != 0; info.hash = CellUtil.cloneValue(hashCell); - info.hasMoreRowsInRegion = - hasMoreRowsInRegionCell.getValueArray()[hasMoreRowsInRegionCell.getValueOffset()] != 0; return info; } /** * Formats counters as a comma-separated key=value string. Example: - * "CHUNKS_VERIFIED=10,CHUNKS_MISMATCHED=2,ROWS_PROCESSED=5678" + * "CHUNKS_VERIFIED=10,CHUNKS_MISMATCHED=2,SOURCE_ROWS_PROCESSED=5678..." * @param counters Map of counter names to values * @return Formatted string or null if counters is null/empty */ @@ -439,28 +401,25 @@ private String formatCounters(Map counters) { /*** * */ - private void handleVerifiedChunk(ChunkInfo chunk, Context context) throws SQLException { - Timestamp chunkEndTime = new Timestamp(System.currentTimeMillis()); - String counters = "ROWS_PROCESSED=" + chunk.rowCount; + private void handleVerifiedChunk(ChunkInfo sourceChunk, Context context, String counters) throws SQLException { syncTableOutputRepository.checkpointSyncTableResult(tableName, targetZkQuorum, - PhoenixSyncTableOutputRow.Type.CHUNK, fromTime, toTime, isDryRun, chunk.startKey, - chunk.endKey, PhoenixSyncTableOutputRow.Status.VERIFIED, chunk.executionStartTime, - chunkEndTime, counters); + PhoenixSyncTableOutputRow.Type.CHUNK, fromTime, toTime, isDryRun, sourceChunk.startKey, + sourceChunk.endKey, PhoenixSyncTableOutputRow.Status.VERIFIED, sourceChunk.executionStartTime, + sourceChunk.executionEndTime, counters); context.getCounter(SyncCounters.CHUNKS_VERIFIED).increment(1); } /*** * */ - private void handleMismatchedChunk(ChunkInfo sourceChunk, Context context) throws SQLException { - Timestamp chunkEndTime = new Timestamp(System.currentTimeMillis()); - LOGGER.warn("Chunk mismatch detected: {} to {}", Bytes.toStringBinary(sourceChunk.startKey), - Bytes.toStringBinary(sourceChunk.endKey)); - String counters = "ROWS_PROCESSED=" + sourceChunk.rowCount; + private void handleMismatchedChunk(ChunkInfo sourceChunk, Context context, String counters) throws SQLException { + LOGGER.warn("Chunk mismatch detected for table: {}, with startKey: {}, endKey {}", + tableName, Bytes.toStringBinary(sourceChunk.startKey), + Bytes.toStringBinary(sourceChunk.endKey)); syncTableOutputRepository.checkpointSyncTableResult(tableName, targetZkQuorum, - PhoenixSyncTableOutputRow.Type.CHUNK, fromTime, toTime, isDryRun, sourceChunk.startKey, - sourceChunk.endKey, PhoenixSyncTableOutputRow.Status.MISMATCHED, - sourceChunk.executionStartTime, chunkEndTime, counters); + PhoenixSyncTableOutputRow.Type.CHUNK, fromTime, toTime, isDryRun, sourceChunk.startKey, + sourceChunk.endKey, PhoenixSyncTableOutputRow.Status.MISMATCHED, + sourceChunk.executionStartTime, sourceChunk.executionEndTime, counters); context.getCounter(SyncCounters.CHUNKS_MISMATCHED).increment(1); } @@ -468,15 +427,17 @@ private void handleMismatchedChunk(ChunkInfo sourceChunk, Context context) throw * Creates a Hbase raw scan for a chunk range to capture all cell versions and delete markers. */ private Scan createChunkScan(byte[] startKey, byte[] endKey, boolean isStartKeyInclusive, - boolean isEndKeyInclusive) throws IOException { + boolean isEndKeyInclusive, boolean isTargetScan) throws IOException { Scan scan = new Scan(); scan.withStartRow(startKey, isStartKeyInclusive); scan.withStopRow(endKey, isEndKeyInclusive); scan.setRaw(true); scan.readAllVersions(); scan.setCacheBlocks(false); - if (fromTime != null || toTime != null) { - scan.setTimeRange(fromTime != null ? fromTime : 0L, toTime != null ? toTime : Long.MAX_VALUE); + scan.setTimeRange(fromTime, toTime); + if (isTargetScan) { + scan.setLimit(1); + scan.setCaching(1); } return scan; } @@ -506,8 +467,7 @@ public List> calculateUnprocessedRanges(byte[] mapperRegion // If Mapper region boundary is [20,85) and already processed chunked rows boundaries are // [10,30] and [70,80] // When we start mapper, it would identify already processed ranges as [10,30] and [70,80] - // and calculate unprocessed ranges as [30,70) and [80,85). It would then do further chunking - // on these two new boundaries. + // and return gaps as [30,70) and [80,85). byte[] effectiveStart = Bytes.compareTo(chunkStart, mapperRegionStart) > 0 ? chunkStart : mapperRegionStart; byte[] effectiveEnd = @@ -565,6 +525,7 @@ private static class ChunkInfo { boolean isPartial; boolean hasMoreRowsInRegion; Timestamp executionStartTime; + Timestamp executionEndTime; @Override public String toString() { diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java index a2ecb8e537b..d700f195907 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java @@ -34,8 +34,10 @@ /** * Repository for managing the PHOENIX_SYNC_TABLE_CHECKPOINT table. This table stores checkpoint - * information for the PhoenixSyncTableTool, enabling: 1. Mapper Level checkpointing (skip completed - * mapper regions on restart) 2. Chunk level checkpointing (skip completed chunks) + * information for the PhoenixSyncTableTool, enabling: + * 1. Mapper Level checkpointing (skip completed + * mapper regions on restart) + * 2. Chunk level checkpointing (skip completed chunks) */ public class PhoenixSyncTableOutputRepository { @@ -145,8 +147,7 @@ public List getProcessedMapperRegions(String tableNam String query = "SELECT START_ROW_KEY, END_ROW_KEY, IS_FIRST_REGION FROM " + SYNC_TABLE_CHECKPOINT_TABLE_NAME + " WHERE TABLE_NAME = ? AND TARGET_CLUSTER = ?" - + " AND ENTRY_TYPE = ? AND FROM_TIME = ? AND TO_TIME = ? AND STATUS IN ( ?, ?)" - + " ORDER BY START_ROW_KEY "; + + " AND ENTRY_TYPE = ? AND FROM_TIME = ? AND TO_TIME = ? AND STATUS IN ( ?, ?)"; List results = new ArrayList<>(); try (PreparedStatement ps = connection.prepareStatement(query)) { int paramIndex = 1; @@ -171,7 +172,7 @@ public List getProcessedMapperRegions(String tableNam } /** - * Queries for processed chunks within a mapper region. Used by PhoenixSyncTableMapper to skip + * Queries for processed chunks. Used by PhoenixSyncTableMapper to skip * already-processed chunks. * @param tableName Source table name * @param targetCluster Target cluster ZK quorum @@ -187,7 +188,7 @@ public List getProcessedChunks(String tableName, Stri String query = "SELECT START_ROW_KEY, END_ROW_KEY, IS_FIRST_REGION FROM " + SYNC_TABLE_CHECKPOINT_TABLE_NAME + " WHERE TABLE_NAME = ? AND TARGET_CLUSTER = ? " + " AND ENTRY_TYPE = ? AND FROM_TIME = ? AND TO_TIME = ? AND START_ROW_KEY < ? " - + " AND END_ROW_KEY > ? AND STATUS IN (?, ?)" + " ORDER BY START_ROW_KEY"; + + " AND END_ROW_KEY > ? AND STATUS IN (?, ?) "; List results = new ArrayList<>(); try (PreparedStatement ps = connection.prepareStatement(query)) { diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java index 4af09110ec8..cd3b61ba2c1 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java @@ -21,7 +21,7 @@ import org.apache.hadoop.hbase.util.Bytes; /** - * Data model class representing a row in the PHOENIX_SYNC_TABLE_CHECKPOINT table + * Data model class representing required row in the PHOENIX_SYNC_TABLE_CHECKPOINT table */ public class PhoenixSyncTableOutputRow { diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java index c284a37ccb0..31a5b913de2 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java @@ -66,7 +66,7 @@ *

How It Works

*
    *
  1. Job Setup: The tool creates a MapReduce job that partitions the table into mapper - * regions based on HBase region boundaries or tenant ID ranges (for multi-tenant tables).
  2. + * regions based on HBase region boundaries. *
  3. Server-Side Chunking: Each mapper triggers a coprocessor scan on both source and * target clusters. The {@link PhoenixSyncTableRegionScanner} coprocessor accumulates rows into * chunks (configurable size, default 1GB) and computes a SHA-256 hash of all row data (keys + @@ -96,8 +96,8 @@ public class PhoenixSyncTableTool extends Configured implements Tool { new Option("tc", "target-cluster", true, "Target cluster ZooKeeper quorum (mandatory)"); private static final Option FROM_TIME_OPTION = new Option("ft", "from-time", true, "Start time in milliseconds for sync (optional, defaults to 0)"); - private static final Option TO_TIME_OPTION = new Option("rt", "to-time", true, - "End time in milliseconds for sync (optional, defaults to current time)"); + private static final Option TO_TIME_OPTION = new Option("tt", "to-time", true, + "End time in milliseconds for sync (optional, defaults to current time - 1 hour)"); private static final Option DRY_RUN_OPTION = new Option("dr", "dry-run", false, "Dry run mode - only checkpoint inconsistencies, do not repair (optional)"); private static final Option CHUNK_SIZE_OPTION = @@ -134,14 +134,14 @@ private Job configureAndCreatePhoenixSyncTableJob(PTableType tableType) throws E configureTimeoutsAndRetries(configuration); setPhoenixSyncTableToolConfiguration(configuration); Job job = Job.getInstance(configuration, getJobName()); - Configuration conf = job.getConfiguration(); - HBaseConfiguration.merge(conf, HBaseConfiguration.create(conf)); - configureInput(job, tableType); job.setMapperClass(PhoenixSyncTableMapper.class); job.setJarByClass(PhoenixSyncTableTool.class); - configureOutput(job); TableMapReduceUtil.initCredentials(job); TableMapReduceUtil.addDependencyJars(job); + Configuration conf = job.getConfiguration(); + HBaseConfiguration.merge(conf, HBaseConfiguration.create(conf)); + configureInput(job, tableType); + configureOutput(job); obtainTargetClusterTokens(job); return job; } @@ -215,7 +215,7 @@ private void configureOutput(Job job) { } private String getJobName() { - StringBuilder jobName = new StringBuilder("PhoenixSyncTable"); + StringBuilder jobName = new StringBuilder("PhoenixSyncTableTool"); if (qSchemaName != null) { jobName.append("-").append(qSchemaName); } @@ -297,7 +297,8 @@ public void populateSyncTableToolAttributes(CommandLine cmdLine) throws Exceptio if (cmdLine.hasOption(TO_TIME_OPTION.getOpt())) { endTime = Long.valueOf(cmdLine.getOptionValue(TO_TIME_OPTION.getOpt())); } else { - endTime = EnvironmentEdgeManager.currentTimeMillis(); + // Default endTime, current time - 1 hour + endTime = EnvironmentEdgeManager.currentTimeMillis(); //- (60 * 60 * 1000); } if (cmdLine.hasOption(CHUNK_SIZE_OPTION.getOpt())) { @@ -322,7 +323,7 @@ public void populateSyncTableToolAttributes(CommandLine cmdLine) throws Exceptio } /** - * Creates or verifies the Phoenix sync tool checkpoint output table + * Creates or verifies the Phoenix sync tool checkpoint table */ private void createSyncOutputTable(Connection connection) throws SQLException { PhoenixSyncTableOutputRepository repository = new PhoenixSyncTableOutputRepository(connection); @@ -351,7 +352,7 @@ private boolean submitPhoenixSyncTableJob() throws Exception { job.getJobName(), qTable); return true; } - LOGGER.info("Running PhoenixSyncTable job: {} for table:{}in foreground.", job.getJobName(), + LOGGER.info("Running PhoenixSyncTable job: {} for table:{} in foreground.", job.getJobName(), qTable); boolean success = job.waitForCompletion(true); if (success) { @@ -360,6 +361,18 @@ private boolean submitPhoenixSyncTableJob() throws Exception { LOGGER.error("PhoenixSyncTable job {} failed for table {} to target cluster {}", job.getJobName(), qTable, targetZkQuorum); } + Counters counters = job.getCounters(); + LOGGER.info( + "PhoenixSyncTable job completed, gathered counters are \n Input Record: {}, \n" + + "Ouput Record: {}, \n Failed Record: {}, \n Chunks Verified: {}, \n" + + "Chunks Mimatched: {}, \n Source Rows Processed: {}, \n Target Rows Processed: {}", + counters.findCounter(PhoenixJobCounters.INPUT_RECORDS).getValue(), + counters.findCounter(PhoenixJobCounters.OUTPUT_RECORDS).getValue(), + counters.findCounter(PhoenixJobCounters.FAILED_RECORDS).getValue(), + counters.findCounter(PhoenixSyncTableMapper.SyncCounters.CHUNKS_VERIFIED).getValue(), + counters.findCounter(PhoenixSyncTableMapper.SyncCounters.CHUNKS_MISMATCHED).getValue(), + counters.findCounter(PhoenixSyncTableMapper.SyncCounters.SOURCE_ROWS_PROCESSED).getValue(), + counters.findCounter(PhoenixSyncTableMapper.SyncCounters.TARGET_ROWS_PROCESSED).getValue()); return success; } @@ -381,17 +394,6 @@ public int run(String[] args) throws Exception { PTableType tableType = validateAndGetTableType(); job = configureAndCreatePhoenixSyncTableJob(tableType); boolean result = submitPhoenixSyncTableJob(); - Counters counters = job.getCounters(); - LOGGER.info( - "PhoenixSyncTable job completed, gathered counters are \n" + "Input Record: {}, \n" - + "Ouput Record: {}, \n" + "Failed Record: {}, \n" + "Chunks Verified: {}, \n" - + "Chunks Mimatched: {}," + "Rows Processed: {}", - counters.findCounter(PhoenixJobCounters.INPUT_RECORDS).getValue(), - counters.findCounter(PhoenixJobCounters.OUTPUT_RECORDS).getValue(), - counters.findCounter(PhoenixJobCounters.FAILED_RECORDS).getValue(), - counters.findCounter(PhoenixSyncTableMapper.SyncCounters.CHUNKS_VERIFIED).getValue(), - counters.findCounter(PhoenixSyncTableMapper.SyncCounters.CHUNKS_MISMATCHED).getValue(), - counters.findCounter(PhoenixSyncTableMapper.SyncCounters.ROWS_PROCESSED).getValue()); return result ? 0 : -1; } catch (Exception ex) { LOGGER.error( diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java index 6e6bda71fff..7a3fee6b457 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java @@ -237,9 +237,6 @@ public final class PhoenixConfigurationUtil { public static final String PHOENIX_SYNC_TABLE_CHUNK_SIZE_BYTES = "phoenix.sync.table.chunk.size.bytes"; - public static final String PHOENIX_SYNC_TABLE_CHUNK_MAX_ROWS = - "phoenix.sync.table.chunk.max.rows"; - public static final long DEFAULT_PHOENIX_SYNC_TABLE_CHUNK_SIZE_BYTES = 1024 * 1024 * 1024; // 1GB /** @@ -1009,7 +1006,7 @@ public static void setPhoenixSyncTableDryRun(Configuration conf, boolean dryRun) public static boolean getPhoenixSyncTableDryRun(Configuration conf) { Preconditions.checkNotNull(conf); - return conf.getBoolean(PHOENIX_SYNC_TABLE_DRY_RUN, false); + return conf.getBoolean(PHOENIX_SYNC_TABLE_DRY_RUN, true); } public static void setPhoenixSyncTableChunkSizeBytes(Configuration conf, Long chunkSizeBytes) { From 71d002d7d9ffd23851e4fbbb80d846bf8556eaf6 Mon Sep 17 00:00:00 2001 From: Rahul Kumar Date: Tue, 17 Feb 2026 12:29:27 +0530 Subject: [PATCH 4/9] Fixes after testing --- .../PhoenixSyncTableRegionScanner.java | 12 +-- .../PhoenixSyncTableInputFormat.java | 5 +- .../mapreduce/PhoenixSyncTableMapper.java | 81 ++++++++++--------- .../PhoenixSyncTableOutputRepository.java | 9 +-- .../mapreduce/PhoenixSyncTableTool.java | 22 ++--- 5 files changed, 65 insertions(+), 64 deletions(-) diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java index 13202048d3c..acc4e2e2342 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java @@ -51,14 +51,14 @@ * Server-side coprocessor that performs chunk formation and SHA-256 hashing for * PhoenixSyncTableTool. *

    - * Accumulates rows into chunks (based on size limits) and computes a hash of all row data - * (keys, column families, qualifiers, timestamps, cell types, values). + * Accumulates rows into chunks (based on size limits) and computes a hash of all row data (keys, + * column families, qualifiers, timestamps, cell types, values). *

    * Source scan (isTargetScan=false): Returns complete chunks bounded by region boundaries. Sets * hasMoreRows=false when region is exhausted. *

    - * Target scan (isTargetScan=true): Returns partial chunks with serialized digest state when - * region boundary is reached, allowing cross-region hash continuation. + * Target scan (isTargetScan=true): Returns partial chunks with serialized digest state when region + * boundary is reached, allowing cross-region hash continuation. *

    * Returns chunk metadata cells: END_KEY, HASH (or digest state), ROW_COUNT, IS_PARTIAL_CHUNK */ @@ -302,8 +302,8 @@ private SHA256Digest decodeDigestState(byte[] encodedState) throws IOException { /** * Builds chunk metadata result cells and adds them to the results list. Returns a single * "row"[rowkey=chunkStartKey] with multiple cells containing chunk metadata[chunkEndKey, - * hash/digest, rowCount, hasMoreRows, isPartialChunk]. For complete chunks: includes - * final SHA-256 hash (32 bytes) For partial chunks: includes serialized MessageDigest state for + * hash/digest, rowCount, hasMoreRows, 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 isPartialChunk true if this is a partial chunk (region boundary reached before diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java index eeba1e49cce..ccc0a1a7b4d 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java @@ -56,8 +56,7 @@ public PhoenixSyncTableInputFormat() { *

    * PhoenixSyncTableMapper doesn't need actual row data from the RecordReader - it extracts region * boundaries from the InputSplit and delegates all scanning to the PhoenixSyncTableRegionScanner - * coprocessor. - * Using PhoenixNoOpSingleRecordReader ensures that {@code map()} is called exactly + * coprocessor. Using PhoenixNoOpSingleRecordReader ensures that {@code map()} is called exactly * once per region no matter what scan looks like, avoiding the overhead of the default * PhoenixRecordReader which would call {@code map()} for every row of scan. * @param split Input Split @@ -97,7 +96,7 @@ public List getSplits(JobContext context) throws IOException, Interr } if (completedRegions.isEmpty()) { LOGGER.info("No completed regions for table {} - processing all {} splits", tableName, - allSplits.size()); + allSplits.size()); return allSplits; } diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java index 813ef39c0fd..527da9108d0 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java @@ -106,8 +106,8 @@ protected void setup(Context context) throws InterruptedException { syncTableOutputRepository = new PhoenixSyncTableOutputRepository(globalConnection); } catch (SQLException | IOException e) { tryClosingResources(); - throw new RuntimeException(String.format("Failed to setup PhoenixSyncTableMapper for table: %s", tableName), - e); + throw new RuntimeException( + String.format("Failed to setup PhoenixSyncTableMapper for table: %s", tableName), e); } } @@ -202,12 +202,11 @@ protected void map(NullWritable key, DBInputFormat.NullDBWritable value, Context } /** - * Processes a chunk range by comparing source and target cluster data. - * Source chunking: Breaks data into size-based chunks within given mapper region boundary. - * Target chunking: Follows source chunk boundaries exactly. Source chunk boundary might - * be split across multiple target region, if so corpoc signals for partial chunk with partial - * digest. Once entire Source chunk is covered by target scanner, we calculate resulting checksum - * from combined digest. + * Processes a chunk range by comparing source and target cluster data. Source chunking: Breaks + * data into size-based chunks within given mapper region boundary. Target chunking: Follows + * source chunk boundaries exactly. Source chunk boundary might be split across multiple target + * region, if so corpoc signals for partial chunk with partial digest. Once entire Source chunk is + * covered by target scanner, we calculate resulting checksum from combined digest. * @param rangeStart Range start key * @param rangeEnd Range end key * @param context Mapper context for progress and counters @@ -215,28 +214,28 @@ protected void map(NullWritable key, DBInputFormat.NullDBWritable value, Context * @throws SQLException if database operations fail */ private void processChunkRange(byte[] rangeStart, byte[] rangeEnd, Context context) - throws IOException, SQLException { + throws IOException, SQLException { boolean isStartKeyInclusive = true; try (ChunkScannerContext sourceScanner = createChunkScanner(sourceConnection, rangeStart, - rangeEnd, null, isStartKeyInclusive, false, false)) { + rangeEnd, null, isStartKeyInclusive, false, false)) { while (true) { - // We only try to get one chunked metadata row returned at a time until no more chunk returned(i.e null) + // We only try to get one chunked metadata row returned at a time until no more chunk + // returned(i.e null) ChunkInfo sourceChunk = sourceScanner.getNextChunk(); if (sourceChunk == null) { break; } sourceChunk.executionStartTime = new Timestamp(System.currentTimeMillis()); - ChunkInfo targetChunk = - getTargetChunkWithSourceBoundary(targetConnection, sourceChunk.startKey, - sourceChunk.endKey); + ChunkInfo targetChunk = getTargetChunkWithSourceBoundary(targetConnection, + sourceChunk.startKey, sourceChunk.endKey); context.getCounter(SyncCounters.SOURCE_ROWS_PROCESSED).increment(sourceChunk.rowCount); context.getCounter(SyncCounters.TARGET_ROWS_PROCESSED).increment(targetChunk.rowCount); boolean matched = MessageDigest.isEqual(sourceChunk.hash, targetChunk.hash); if (LOGGER.isDebugEnabled()) { LOGGER.debug("Chunk comparison {}, {}: source={} rows, target={} rows, matched={}", - Bytes.toStringBinary(sourceChunk.startKey), Bytes.toStringBinary(sourceChunk.endKey), - sourceChunk.rowCount, targetChunk.rowCount, matched); + Bytes.toStringBinary(sourceChunk.startKey), Bytes.toStringBinary(sourceChunk.endKey), + sourceChunk.rowCount, targetChunk.rowCount, matched); } sourceChunk.executionEndTime = new Timestamp(System.currentTimeMillis()); Map mapperCounters = new LinkedHashMap<>(); @@ -252,12 +251,12 @@ private void processChunkRange(byte[] rangeStart, byte[] rangeEnd, Context conte } } LOGGER.info("Completed sync table processing of Mapper region boundary {}, {}", - Bytes.toStringBinary(rangeStart), Bytes.toStringBinary(rangeEnd)); + Bytes.toStringBinary(rangeStart), Bytes.toStringBinary(rangeEnd)); } /** - * Scans target across multiple regions and returns a single combined ChunkInfo. - * Handles partial chunks by passing digest state to next scanner via scan attributes, enabling cross-region + * Scans target across multiple regions and returns a single combined ChunkInfo. Handles partial + * chunks by passing digest state to next scanner via scan attributes, enabling cross-region * digest continuation. Since we are scanning rows based on source chunk boundary, it could be * distributed across multiple target regions. We keep on creating scanner across target region * until entire source chunk boundary is processed or chunk is null @@ -279,18 +278,22 @@ private ChunkInfo getTargetChunkWithSourceBoundary(Connection conn, byte[] start boolean isStartKeyInclusive = true; while (true) { // We are creating a new scanner for every target region chunk. - // This chunk could be partial or full depending on whether the source region boundary is part of one or multiple target region. + // This chunk could be partial or full depending on whether the source region boundary is part + // of one or multiple target region. // For every target region scanned, we want to have one row processed and returned back // immediately(that's why we set scan.setLimit(1)), since output from one region partial chunk // scanner is input to next region scanner. try (ChunkScannerContext scanner = createChunkScanner(conn, currentStartKey, endKey, continuedDigestState, isStartKeyInclusive, true, true)) { ChunkInfo chunk = scanner.getNextChunk(); - // In a happy path where source and target rows are matching, target chunk would never be null. + // In a happy path where source and target rows are matching, target chunk would never be + // null. // If chunk returned null, this would mean it couldn't find last source rows in target, // since we only return isPartial=true until target chunk end key < source chunk endKey. - // Hash would still be digest if chunk returned is null and not a checksum, so would never match(which is expected). - // We could convert the digest to checksum but since it won't match anyhow, we don't need to. + // Hash would still be digest if chunk returned is null and not a checksum, so would never + // match(which is expected). + // We could convert the digest to checksum but since it won't match anyhow, we don't need + // to. if (chunk == null) { break; } @@ -318,17 +321,17 @@ private ChunkInfo getTargetChunkWithSourceBoundary(Connection conn, byte[] start * cross-region continuation on target) * @param isStartKeyInclusive Whether StartKey Inclusive * @param isEndKeyInclusive Whether EndKey Inclusive - * @return ChunkScannerContext * @throws IOException scanner creation fails * @throws SQLException hTable connection fails */ private ChunkScannerContext createChunkScanner(Connection conn, byte[] startKey, byte[] endKey, - byte[] continuedDigestState, boolean isStartKeyInclusive, - boolean isEndKeyInclusive, boolean isTargetScan) throws IOException, SQLException { + byte[] continuedDigestState, boolean isStartKeyInclusive, boolean isEndKeyInclusive, + boolean isTargetScan) throws IOException, SQLException { // Not using try-with-resources since ChunkScannerContext owns the table lifecycle Table hTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(physicalTableName); - Scan scan = createChunkScan(startKey, endKey, isStartKeyInclusive, isEndKeyInclusive, isTargetScan); + Scan scan = + createChunkScan(startKey, endKey, isStartKeyInclusive, isEndKeyInclusive, isTargetScan); scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CHUNK_FORMATION, TRUE_BYTES); scan.setAttribute(BaseScannerRegionObserverConstants.SKIP_REGION_BOUNDARY_CHECK, TRUE_BYTES); scan.setAttribute(BaseScannerRegionObserverConstants.UNGROUPED_AGG, TRUE_BYTES); @@ -361,8 +364,9 @@ private ChunkInfo parseChunkInfo(Result result) { Cell hashCell = MetaDataUtil.getCell(cells, BaseScannerRegionObserverConstants.SYNC_TABLE_HASH_QUALIFIER); - if (endKeyCell == null || rowCountCell == null || isPartialChunkCell == null - || hashCell == null) { + if ( + endKeyCell == null || rowCountCell == null || isPartialChunkCell == null || hashCell == null + ) { throw new RuntimeException("Missing required chunk metadata cells."); } @@ -401,10 +405,11 @@ private String formatCounters(Map counters) { /*** * */ - private void handleVerifiedChunk(ChunkInfo sourceChunk, Context context, String counters) throws SQLException { + private void handleVerifiedChunk(ChunkInfo sourceChunk, Context context, String counters) + throws SQLException { syncTableOutputRepository.checkpointSyncTableResult(tableName, targetZkQuorum, PhoenixSyncTableOutputRow.Type.CHUNK, fromTime, toTime, isDryRun, sourceChunk.startKey, - sourceChunk.endKey, PhoenixSyncTableOutputRow.Status.VERIFIED, sourceChunk.executionStartTime, + sourceChunk.endKey, PhoenixSyncTableOutputRow.Status.VERIFIED, sourceChunk.executionStartTime, sourceChunk.executionEndTime, counters); context.getCounter(SyncCounters.CHUNKS_VERIFIED).increment(1); } @@ -412,14 +417,14 @@ private void handleVerifiedChunk(ChunkInfo sourceChunk, Context context, String /*** * */ - private void handleMismatchedChunk(ChunkInfo sourceChunk, Context context, String counters) throws SQLException { - LOGGER.warn("Chunk mismatch detected for table: {}, with startKey: {}, endKey {}", - tableName, Bytes.toStringBinary(sourceChunk.startKey), - Bytes.toStringBinary(sourceChunk.endKey)); + private void handleMismatchedChunk(ChunkInfo sourceChunk, Context context, String counters) + throws SQLException { + LOGGER.warn("Chunk mismatch detected for table: {}, with startKey: {}, endKey {}", tableName, + Bytes.toStringBinary(sourceChunk.startKey), Bytes.toStringBinary(sourceChunk.endKey)); syncTableOutputRepository.checkpointSyncTableResult(tableName, targetZkQuorum, - PhoenixSyncTableOutputRow.Type.CHUNK, fromTime, toTime, isDryRun, sourceChunk.startKey, - sourceChunk.endKey, PhoenixSyncTableOutputRow.Status.MISMATCHED, - sourceChunk.executionStartTime, sourceChunk.executionEndTime, counters); + PhoenixSyncTableOutputRow.Type.CHUNK, fromTime, toTime, isDryRun, sourceChunk.startKey, + sourceChunk.endKey, PhoenixSyncTableOutputRow.Status.MISMATCHED, + sourceChunk.executionStartTime, sourceChunk.executionEndTime, counters); context.getCounter(SyncCounters.CHUNKS_MISMATCHED).increment(1); } diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java index d700f195907..0d6f7c0a1cf 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java @@ -34,10 +34,8 @@ /** * Repository for managing the PHOENIX_SYNC_TABLE_CHECKPOINT table. This table stores checkpoint - * information for the PhoenixSyncTableTool, enabling: - * 1. Mapper Level checkpointing (skip completed - * mapper regions on restart) - * 2. Chunk level checkpointing (skip completed chunks) + * information for the PhoenixSyncTableTool, enabling: 1. Mapper Level checkpointing (skip completed + * mapper regions on restart) 2. Chunk level checkpointing (skip completed chunks) */ public class PhoenixSyncTableOutputRepository { @@ -172,8 +170,7 @@ public List getProcessedMapperRegions(String tableNam } /** - * Queries for processed chunks. Used by PhoenixSyncTableMapper to skip - * already-processed chunks. + * Queries for processed chunks. Used by PhoenixSyncTableMapper to skip already-processed chunks. * @param tableName Source table name * @param targetCluster Target cluster ZK quorum * @param fromTime Start timestamp (nullable) diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java index 31a5b913de2..85cb68a620f 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java @@ -298,7 +298,7 @@ public void populateSyncTableToolAttributes(CommandLine cmdLine) throws Exceptio endTime = Long.valueOf(cmdLine.getOptionValue(TO_TIME_OPTION.getOpt())); } else { // Default endTime, current time - 1 hour - endTime = EnvironmentEdgeManager.currentTimeMillis(); //- (60 * 60 * 1000); + endTime = EnvironmentEdgeManager.currentTimeMillis(); // - (60 * 60 * 1000); } if (cmdLine.hasOption(CHUNK_SIZE_OPTION.getOpt())) { @@ -363,16 +363,16 @@ private boolean submitPhoenixSyncTableJob() throws Exception { } Counters counters = job.getCounters(); LOGGER.info( - "PhoenixSyncTable job completed, gathered counters are \n Input Record: {}, \n" - + "Ouput Record: {}, \n Failed Record: {}, \n Chunks Verified: {}, \n" - + "Chunks Mimatched: {}, \n Source Rows Processed: {}, \n Target Rows Processed: {}", - counters.findCounter(PhoenixJobCounters.INPUT_RECORDS).getValue(), - counters.findCounter(PhoenixJobCounters.OUTPUT_RECORDS).getValue(), - counters.findCounter(PhoenixJobCounters.FAILED_RECORDS).getValue(), - counters.findCounter(PhoenixSyncTableMapper.SyncCounters.CHUNKS_VERIFIED).getValue(), - counters.findCounter(PhoenixSyncTableMapper.SyncCounters.CHUNKS_MISMATCHED).getValue(), - counters.findCounter(PhoenixSyncTableMapper.SyncCounters.SOURCE_ROWS_PROCESSED).getValue(), - counters.findCounter(PhoenixSyncTableMapper.SyncCounters.TARGET_ROWS_PROCESSED).getValue()); + "PhoenixSyncTable job completed, gathered counters are \n Input Record: {}, \n" + + "Ouput Record: {}, \n Failed Record: {}, \n Chunks Verified: {}, \n" + + "Chunks Mimatched: {}, \n Source Rows Processed: {}, \n Target Rows Processed: {}", + counters.findCounter(PhoenixJobCounters.INPUT_RECORDS).getValue(), + counters.findCounter(PhoenixJobCounters.OUTPUT_RECORDS).getValue(), + counters.findCounter(PhoenixJobCounters.FAILED_RECORDS).getValue(), + counters.findCounter(PhoenixSyncTableMapper.SyncCounters.CHUNKS_VERIFIED).getValue(), + counters.findCounter(PhoenixSyncTableMapper.SyncCounters.CHUNKS_MISMATCHED).getValue(), + counters.findCounter(PhoenixSyncTableMapper.SyncCounters.SOURCE_ROWS_PROCESSED).getValue(), + counters.findCounter(PhoenixSyncTableMapper.SyncCounters.TARGET_ROWS_PROCESSED).getValue()); return success; } From 348c722d78925d03a3fe3f0ebfcdd59a3310ef64 Mon Sep 17 00:00:00 2001 From: Rahul Kumar Date: Tue, 17 Feb 2026 12:32:13 +0530 Subject: [PATCH 5/9] Handling edge cases --- .../PhoenixSyncTableInputFormat.java | 33 ++- .../mapreduce/PhoenixSyncTableMapper.java | 245 ++++++++++++------ .../PhoenixSyncTableOutputRepository.java | 78 ++++-- .../mapreduce/PhoenixSyncTableOutputRow.java | 8 +- 4 files changed, 238 insertions(+), 126 deletions(-) diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java index ccc0a1a7b4d..87ceba090ca 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java @@ -23,6 +23,7 @@ import java.util.ArrayList; import java.util.List; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.JobContext; @@ -140,21 +141,25 @@ private List filterCompletedSplits(List allSplits, List unprocessedSplits = new ArrayList<>(); int splitIdx = 0; int completedIdx = 0; + // Two pointer comparison across splitRange and completedRange while (splitIdx < allSplits.size() && completedIdx < completedRegions.size()) { PhoenixInputSplit split = (PhoenixInputSplit) allSplits.get(splitIdx); KeyRange splitRange = split.getKeyRange(); KeyRange completedRange = completedRegions.get(completedIdx); - byte[] splitStart = normalizeKey(splitRange.getLowerRange()); - byte[] splitEnd = normalizeKey(splitRange.getUpperRange()); - byte[] completedStart = normalizeKey(completedRange.getLowerRange()); - byte[] completedEnd = normalizeKey(completedRange.getUpperRange()); + byte[] splitStart = splitRange.getLowerRange(); + byte[] splitEnd = splitRange.getUpperRange(); + byte[] completedStart = completedRange.getLowerRange(); + byte[] completedEnd = completedRange.getUpperRange(); + // No overlap b/w completedRange/splitRange. // completedEnd is before splitStart, increment completed pointer to catch up - if (Bytes.compareTo(completedEnd, splitStart) <= 0) { + if (!Bytes.equals(completedEnd, HConstants.EMPTY_END_ROW) + && Bytes.compareTo(completedEnd, splitStart) <= 0) { completedIdx++; - } else if (Bytes.compareTo(completedStart, splitEnd) >= 0) { + } else if (!Bytes.equals(splitEnd, HConstants.EMPTY_END_ROW) + && Bytes.compareTo(completedStart, splitEnd) >= 0) { // No overlap. completedStart is after splitEnd, splitRange needs to be processed, // add to unprocessed list and increment unprocessedSplits.add(allSplits.get(splitIdx)); @@ -162,11 +167,14 @@ private List filterCompletedSplits(List allSplits, } else { // Some overlap detected, check if SplitRange is fullyContained within completedRange // Fully contained if: completedStart <= splitStart AND splitEnd <= completedEnd + boolean startContained = Bytes.compareTo(completedStart, splitStart) <= 0; - boolean endContained = Bytes.compareTo(splitEnd, completedEnd) <= 0; + boolean endContained = Bytes.equals(completedEnd, HConstants.EMPTY_END_ROW) + || Bytes.compareTo(splitEnd, completedEnd) <= 0; + boolean fullyContained = startContained && endContained; if (!fullyContained) { - // Not fully contained - keep the split + // Not fully contained, keep the split unprocessedSplits.add(allSplits.get(splitIdx)); } splitIdx++; @@ -181,13 +189,4 @@ private List filterCompletedSplits(List allSplits, } return unprocessedSplits; } - - /** - * Normalizes a key boundary for comparison - * @param key The key to normalize - * @return Empty byte array if key is null, otherwise the key unchanged - */ - private byte[] normalizeKey(byte[] key) { - return key == null ? new byte[0] : key; - } } diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java index 527da9108d0..595908d48f3 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java @@ -25,6 +25,7 @@ import java.sql.SQLException; import java.sql.Timestamp; import java.util.*; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; @@ -57,7 +58,7 @@ * chunk hashes from both clusters, compares them and write to checkpoint table. */ public class PhoenixSyncTableMapper - extends Mapper { + extends Mapper { private static final Logger LOGGER = LoggerFactory.getLogger(PhoenixSyncTableMapper.class); @@ -107,7 +108,7 @@ protected void setup(Context context) throws InterruptedException { } catch (SQLException | IOException e) { tryClosingResources(); throw new RuntimeException( - String.format("Failed to setup PhoenixSyncTableMapper for table: %s", tableName), e); + String.format("Failed to setup PhoenixSyncTableMapper for table: %s", tableName), e); } } @@ -119,8 +120,8 @@ private void extractRegionBoundariesFromSplit(Context context) { KeyRange keyRange = split.getKeyRange(); if (keyRange == null) { throw new IllegalStateException(String.format( - "PhoenixInputSplit has no KeyRange for table: %s . Cannot determine region boundaries for sync operation.", - tableName)); + "PhoenixInputSplit has no KeyRange for table: %s . Cannot determine region boundaries for sync operation.", + tableName)); } mapperRegionStart = keyRange.getLowerRange(); mapperRegionEnd = keyRange.getUpperRange(); @@ -131,7 +132,7 @@ private void extractRegionBoundariesFromSplit(Context context) { */ private void connectToTargetCluster() throws SQLException, IOException { Configuration targetConf = - PhoenixMapReduceUtil.createConfigurationForZkQuorum(conf, targetZkQuorum); + PhoenixMapReduceUtil.createConfigurationForZkQuorum(conf, targetZkQuorum); targetConnection = ConnectionUtil.getInputConnection(targetConf); } @@ -152,16 +153,24 @@ private Connection createGlobalConnection(Configuration conf) throws SQLExceptio */ @Override protected void map(NullWritable key, DBInputFormat.NullDBWritable value, Context context) - throws IOException, InterruptedException { + throws IOException, InterruptedException { context.getCounter(PhoenixJobCounters.INPUT_RECORDS).increment(1); try { List processedChunks = - syncTableOutputRepository.getProcessedChunks(tableName, targetZkQuorum, fromTime, toTime, - mapperRegionStart, mapperRegionEnd); + syncTableOutputRepository.getProcessedChunks(tableName, targetZkQuorum, fromTime, toTime, + mapperRegionStart, mapperRegionEnd); List> unprocessedRanges = - calculateUnprocessedRanges(mapperRegionStart, mapperRegionEnd, processedChunks); + calculateUnprocessedRanges(mapperRegionStart, mapperRegionEnd, processedChunks); + // Checking if start key should be inclusive, this is specific to scenario when there are processed + // chunks within this Mapper region boundary. + boolean isStartKeyInclusive = shouldStartKeyBeInclusive(mapperRegionStart, processedChunks); for (Pair range : unprocessedRanges) { - processChunkRange(range.getFirst(), range.getSecond(), context); + // From a Mapper region boundary, if we get multiple fragments of ranges it means there were some processed chunks in this region boundary + // And since chunks are inclusive of start and endKey, we just needed to confirm whether first fragment of range should have start key inclusive. + // Other fragment will not have start key inclusive since these ranges are carved out of chunk boundary, + // both start and key would already have been processed as part of chunk itself + processMapperRanges(range.getFirst(), range.getSecond(), isStartKeyInclusive, context); + isStartKeyInclusive = false; } long mismatchedChunk = context.getCounter(SyncCounters.CHUNKS_MISMATCHED).getValue(); @@ -176,24 +185,30 @@ protected void map(NullWritable key, DBInputFormat.NullDBWritable value, Context mapperCounters.put(SyncCounters.TARGET_ROWS_PROCESSED.name(), targetRowsProcessed); String counters = formatCounters(mapperCounters); - if (mismatchedChunk == 0) { - context.getCounter(PhoenixJobCounters.OUTPUT_RECORDS).increment(1); - syncTableOutputRepository.checkpointSyncTableResult(tableName, targetZkQuorum, - PhoenixSyncTableOutputRow.Type.MAPPER_REGION, fromTime, toTime, isDryRun, - mapperRegionStart, mapperRegionEnd, PhoenixSyncTableOutputRow.Status.VERIFIED, - mapperStartTime, mapperEndTime, counters); - LOGGER.info( - "PhoenixSyncTable mapper completed with verified: {} verifiedChunk chunks, {} mismatchedChunk chunks", - verifiedChunk, mismatchedChunk); + if (sourceRowsProcessed > 0) { + if (mismatchedChunk == 0) { + context.getCounter(PhoenixJobCounters.OUTPUT_RECORDS).increment(1); + syncTableOutputRepository.checkpointSyncTableResult(tableName, targetZkQuorum, + PhoenixSyncTableOutputRow.Type.MAPPER_REGION, fromTime, toTime, isDryRun, + mapperRegionStart, mapperRegionEnd, PhoenixSyncTableOutputRow.Status.VERIFIED, + mapperStartTime, mapperEndTime, counters); + LOGGER.info( + "PhoenixSyncTable mapper completed with verified: {} verifiedChunk chunks, {} mismatchedChunk chunks", + verifiedChunk, mismatchedChunk); + } else { + context.getCounter(PhoenixJobCounters.FAILED_RECORDS).increment(1); + LOGGER.warn( + "PhoenixSyncTable mapper completed with mismatch: {} verifiedChunk chunks, {} mismatchedChunk chunks", + verifiedChunk, mismatchedChunk); + syncTableOutputRepository.checkpointSyncTableResult(tableName, targetZkQuorum, + PhoenixSyncTableOutputRow.Type.MAPPER_REGION, fromTime, toTime, isDryRun, + mapperRegionStart, mapperRegionEnd, PhoenixSyncTableOutputRow.Status.MISMATCHED, + mapperStartTime, mapperEndTime, counters); + } } else { - context.getCounter(PhoenixJobCounters.FAILED_RECORDS).increment(1); - LOGGER.warn( - "PhoenixSyncTable mapper completed with mismatch: {} verifiedChunk chunks, {} mismatchedChunk chunks", - verifiedChunk, mismatchedChunk); - syncTableOutputRepository.checkpointSyncTableResult(tableName, targetZkQuorum, - PhoenixSyncTableOutputRow.Type.MAPPER_REGION, fromTime, toTime, isDryRun, - mapperRegionStart, mapperRegionEnd, PhoenixSyncTableOutputRow.Status.MISMATCHED, - mapperStartTime, mapperEndTime, counters); + LOGGER.info( + "No rows pending to process. All mapper region boundaries are covered for startKey:{}, endKey: {}", + mapperRegionStart, mapperRegionEnd); } } catch (SQLException e) { tryClosingResources(); @@ -213,11 +228,10 @@ protected void map(NullWritable key, DBInputFormat.NullDBWritable value, Context * @throws IOException if scan fails * @throws SQLException if database operations fail */ - private void processChunkRange(byte[] rangeStart, byte[] rangeEnd, Context context) - throws IOException, SQLException { - boolean isStartKeyInclusive = true; + private void processMapperRanges(byte[] rangeStart, byte[] rangeEnd, boolean isStartKeyInclusive, Context context) + throws IOException, SQLException { try (ChunkScannerContext sourceScanner = createChunkScanner(sourceConnection, rangeStart, - rangeEnd, null, isStartKeyInclusive, false, false)) { + rangeEnd, null, isStartKeyInclusive, false, false)) { while (true) { // We only try to get one chunked metadata row returned at a time until no more chunk // returned(i.e null) @@ -226,16 +240,17 @@ private void processChunkRange(byte[] rangeStart, byte[] rangeEnd, Context conte break; } sourceChunk.executionStartTime = new Timestamp(System.currentTimeMillis()); - ChunkInfo targetChunk = getTargetChunkWithSourceBoundary(targetConnection, - sourceChunk.startKey, sourceChunk.endKey); + ChunkInfo targetChunk = + getTargetChunkWithSourceBoundary(targetConnection, sourceChunk.startKey, + sourceChunk.endKey); context.getCounter(SyncCounters.SOURCE_ROWS_PROCESSED).increment(sourceChunk.rowCount); context.getCounter(SyncCounters.TARGET_ROWS_PROCESSED).increment(targetChunk.rowCount); boolean matched = MessageDigest.isEqual(sourceChunk.hash, targetChunk.hash); if (LOGGER.isDebugEnabled()) { LOGGER.debug("Chunk comparison {}, {}: source={} rows, target={} rows, matched={}", - Bytes.toStringBinary(sourceChunk.startKey), Bytes.toStringBinary(sourceChunk.endKey), - sourceChunk.rowCount, targetChunk.rowCount, matched); + Bytes.toStringBinary(sourceChunk.startKey), Bytes.toStringBinary(sourceChunk.endKey), + sourceChunk.rowCount, targetChunk.rowCount, matched); } sourceChunk.executionEndTime = new Timestamp(System.currentTimeMillis()); Map mapperCounters = new LinkedHashMap<>(); @@ -251,7 +266,7 @@ private void processChunkRange(byte[] rangeStart, byte[] rangeEnd, Context conte } } LOGGER.info("Completed sync table processing of Mapper region boundary {}, {}", - Bytes.toStringBinary(rangeStart), Bytes.toStringBinary(rangeEnd)); + Bytes.toStringBinary(rangeStart), Bytes.toStringBinary(rangeEnd)); } /** @@ -266,7 +281,7 @@ private void processChunkRange(byte[] rangeStart, byte[] rangeEnd, Context conte * @return Single ChunkInfo with final hash from all target regions */ private ChunkInfo getTargetChunkWithSourceBoundary(Connection conn, byte[] startKey, - byte[] endKey) throws IOException, SQLException { + byte[] endKey) throws IOException, SQLException { ChunkInfo combinedTargetChunk = new ChunkInfo(); combinedTargetChunk.startKey = startKey; combinedTargetChunk.endKey = endKey; @@ -284,7 +299,7 @@ private ChunkInfo getTargetChunkWithSourceBoundary(Connection conn, byte[] start // immediately(that's why we set scan.setLimit(1)), since output from one region partial chunk // scanner is input to next region scanner. try (ChunkScannerContext scanner = createChunkScanner(conn, currentStartKey, endKey, - continuedDigestState, isStartKeyInclusive, true, true)) { + continuedDigestState, isStartKeyInclusive, true, true)) { ChunkInfo chunk = scanner.getNextChunk(); // In a happy path where source and target rows are matching, target chunk would never be // null. @@ -325,24 +340,24 @@ private ChunkInfo getTargetChunkWithSourceBoundary(Connection conn, byte[] start * @throws SQLException hTable connection fails */ private ChunkScannerContext createChunkScanner(Connection conn, byte[] startKey, byte[] endKey, - byte[] continuedDigestState, boolean isStartKeyInclusive, boolean isEndKeyInclusive, - boolean isTargetScan) throws IOException, SQLException { + byte[] continuedDigestState, boolean isStartKeyInclusive, boolean isEndKeyInclusive, + boolean isTargetScan) throws IOException, SQLException { // Not using try-with-resources since ChunkScannerContext owns the table lifecycle Table hTable = - conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(physicalTableName); + conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(physicalTableName); Scan scan = - createChunkScan(startKey, endKey, isStartKeyInclusive, isEndKeyInclusive, isTargetScan); + createChunkScan(startKey, endKey, isStartKeyInclusive, isEndKeyInclusive, isTargetScan); scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CHUNK_FORMATION, TRUE_BYTES); scan.setAttribute(BaseScannerRegionObserverConstants.SKIP_REGION_BOUNDARY_CHECK, TRUE_BYTES); scan.setAttribute(BaseScannerRegionObserverConstants.UNGROUPED_AGG, TRUE_BYTES); if (continuedDigestState != null && continuedDigestState.length > 0) { scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CONTINUED_DIGEST_STATE, - continuedDigestState); + continuedDigestState); } if (!isTargetScan) { scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CHUNK_SIZE_BYTES, - Bytes.toBytes(chunkSizeBytes)); + Bytes.toBytes(chunkSizeBytes)); } ResultScanner scanner = hTable.getScanner(scan); return new ChunkScannerContext(hTable, scanner); @@ -355,18 +370,17 @@ private ChunkScannerContext createChunkScanner(Connection conn, byte[] startKey, */ private ChunkInfo parseChunkInfo(Result result) { List cells = Arrays.asList(result.rawCells()); - Cell endKeyCell = - MetaDataUtil.getCell(cells, BaseScannerRegionObserverConstants.SYNC_TABLE_END_KEY_QUALIFIER); + Cell endKeyCell = MetaDataUtil.getCell(cells, + BaseScannerRegionObserverConstants.SYNC_TABLE_END_KEY_QUALIFIER); Cell rowCountCell = MetaDataUtil.getCell(cells, - BaseScannerRegionObserverConstants.SYNC_TABLE_ROW_COUNT_QUALIFIER); + BaseScannerRegionObserverConstants.SYNC_TABLE_ROW_COUNT_QUALIFIER); Cell isPartialChunkCell = MetaDataUtil.getCell(cells, - BaseScannerRegionObserverConstants.SYNC_TABLE_IS_PARTIAL_CHUNK_QUALIFIER); + BaseScannerRegionObserverConstants.SYNC_TABLE_IS_PARTIAL_CHUNK_QUALIFIER); Cell hashCell = - MetaDataUtil.getCell(cells, BaseScannerRegionObserverConstants.SYNC_TABLE_HASH_QUALIFIER); + MetaDataUtil.getCell(cells, BaseScannerRegionObserverConstants.SYNC_TABLE_HASH_QUALIFIER); - if ( - endKeyCell == null || rowCountCell == null || isPartialChunkCell == null || hashCell == null - ) { + if (endKeyCell == null || rowCountCell == null || isPartialChunkCell == null + || hashCell == null) { throw new RuntimeException("Missing required chunk metadata cells."); } @@ -374,7 +388,7 @@ private ChunkInfo parseChunkInfo(Result result) { info.startKey = result.getRow(); info.endKey = CellUtil.cloneValue(endKeyCell); info.rowCount = Bytes.toLong(rowCountCell.getValueArray(), rowCountCell.getValueOffset(), - rowCountCell.getValueLength()); + rowCountCell.getValueLength()); info.isPartial = isPartialChunkCell.getValueArray()[isPartialChunkCell.getValueOffset()] != 0; info.hash = CellUtil.cloneValue(hashCell); return info; @@ -406,11 +420,11 @@ private String formatCounters(Map counters) { * */ private void handleVerifiedChunk(ChunkInfo sourceChunk, Context context, String counters) - throws SQLException { + throws SQLException { syncTableOutputRepository.checkpointSyncTableResult(tableName, targetZkQuorum, - PhoenixSyncTableOutputRow.Type.CHUNK, fromTime, toTime, isDryRun, sourceChunk.startKey, - sourceChunk.endKey, PhoenixSyncTableOutputRow.Status.VERIFIED, sourceChunk.executionStartTime, - sourceChunk.executionEndTime, counters); + PhoenixSyncTableOutputRow.Type.CHUNK, fromTime, toTime, isDryRun, sourceChunk.startKey, + sourceChunk.endKey, PhoenixSyncTableOutputRow.Status.VERIFIED, + sourceChunk.executionStartTime, sourceChunk.executionEndTime, counters); context.getCounter(SyncCounters.CHUNKS_VERIFIED).increment(1); } @@ -418,13 +432,14 @@ private void handleVerifiedChunk(ChunkInfo sourceChunk, Context context, String * */ private void handleMismatchedChunk(ChunkInfo sourceChunk, Context context, String counters) - throws SQLException { + throws SQLException { LOGGER.warn("Chunk mismatch detected for table: {}, with startKey: {}, endKey {}", tableName, - Bytes.toStringBinary(sourceChunk.startKey), Bytes.toStringBinary(sourceChunk.endKey)); + Bytes.toStringBinary(sourceChunk.startKey), Bytes.toStringBinary(sourceChunk.endKey)); syncTableOutputRepository.checkpointSyncTableResult(tableName, targetZkQuorum, - PhoenixSyncTableOutputRow.Type.CHUNK, fromTime, toTime, isDryRun, sourceChunk.startKey, - sourceChunk.endKey, PhoenixSyncTableOutputRow.Status.MISMATCHED, - sourceChunk.executionStartTime, sourceChunk.executionEndTime, counters); + PhoenixSyncTableOutputRow.Type.CHUNK, fromTime, toTime, isDryRun, sourceChunk.startKey, + sourceChunk.endKey, PhoenixSyncTableOutputRow.Status.MISMATCHED, + sourceChunk.executionStartTime, sourceChunk.executionEndTime, counters); + context.getCounter(SyncCounters.CHUNKS_MISMATCHED).increment(1); } @@ -432,7 +447,7 @@ private void handleMismatchedChunk(ChunkInfo sourceChunk, Context context, Strin * Creates a Hbase raw scan for a chunk range to capture all cell versions and delete markers. */ private Scan createChunkScan(byte[] startKey, byte[] endKey, boolean isStartKeyInclusive, - boolean isEndKeyInclusive, boolean isTargetScan) throws IOException { + boolean isEndKeyInclusive, boolean isTargetScan) throws IOException { Scan scan = new Scan(); scan.withStartRow(startKey, isStartKeyInclusive); scan.withStopRow(endKey, isEndKeyInclusive); @@ -448,8 +463,13 @@ private Scan createChunkScan(byte[] startKey, byte[] endKey, boolean isStartKeyI } /** - * Calculates unprocessed chunk ranges (gaps) within a mapper region. Given a list of processed - * chunks, returns the ranges that haven't been processed yet. + * Calculates unprocessed gaps within a mapper region. Since a mapper region is divided into + * multiple chunks and we store mapper region boundary and its chunked boundary. + * Once we have all the processedChunks which falls in this Mapper region boundary, + * we look for holes/gaps in mapper region boundary which haven't been processed as chunks. + * Given a list of processed chunks, returns the ranges that haven't been processed yet. + * This will be useful on retries if Region boundary has changed and we some chunks in the + * new region boundary has been processed which can be skipped * @param mapperRegionStart Start of mapper region * @param mapperRegionEnd End of mapper region * @param processedChunks List of already-processed chunks from getProcessedChunks() @@ -457,38 +477,93 @@ private Scan createChunkScan(byte[] startKey, byte[] endKey, boolean isStartKeyI */ @VisibleForTesting public List> calculateUnprocessedRanges(byte[] mapperRegionStart, - byte[] mapperRegionEnd, List processedChunks) { + byte[] mapperRegionEnd, List processedChunks) { List> gaps = new ArrayList<>(); + // If processedChunks is null or empty, the entire mapper region needs processing if (processedChunks == null || processedChunks.isEmpty()) { gaps.add(new Pair<>(mapperRegionStart, mapperRegionEnd)); return gaps; } - byte[] currentPos = mapperRegionStart; - for (PhoenixSyncTableOutputRow chunk : processedChunks) { + // Since chunk keys are always inclusive(start/endKey) it would never be null/empty. + // But Mapper region boundary can be empty i.e [] for start/end region of table. + // We would be doing byte comparison as part of identifying gaps and empty bytes + // needs to be considered as special case as comparison won't work on them. + boolean isStartRegionOfTable = mapperRegionStart == null || mapperRegionStart.length == 0; + boolean isEndRegionOfTable = mapperRegionEnd == null || mapperRegionEnd.length == 0; + + // Track our scanning position through the mapper region as we iterate through chunks + byte[] scanPos = mapperRegionStart; + + // With entire Mapper region boundary, we iterate over each chunk and if any gap/hole identified + // in Mapper region range which is not covered by processed chunk, we add it to gaps list. + // Since chunks are sorted and non-overlapping, only first/last chunks + // need boundary clipping. All middle chunks are guaranteed to be within region boundaries. + for (int i = 0; i < processedChunks.size(); i++) { + PhoenixSyncTableOutputRow chunk = processedChunks.get(i); byte[] chunkStart = chunk.getStartRowKey(); byte[] chunkEnd = chunk.getEndRowKey(); - // Clip chunk boundaries to mapper region boundaries - // If Mapper region boundary is [20,85) and already processed chunked rows boundaries are - // [10,30] and [70,80] - // When we start mapper, it would identify already processed ranges as [10,30] and [70,80] - // and return gaps as [30,70) and [80,85). - byte[] effectiveStart = - Bytes.compareTo(chunkStart, mapperRegionStart) > 0 ? chunkStart : mapperRegionStart; - byte[] effectiveEnd = - Bytes.compareTo(chunkEnd, mapperRegionEnd) < 0 ? chunkEnd : mapperRegionEnd; - // Check for gap before this chunk's effective start - if (Bytes.compareTo(currentPos, effectiveStart) < 0) { - gaps.add(new Pair<>(currentPos, effectiveStart)); + boolean initialChunk = (i == 0); + boolean lastChunk = (i == processedChunks.size() - 1); + + // Determine effective start boundary for this chunk + // Only the first chunk might start before mapperRegionStart and need clipping + byte[] effectiveStart; + if (initialChunk && !isStartRegionOfTable) { + // initialChunk chunk, clip boundary outside of Mapper region. + // Example: Mapper region [20, 85), first chunk [10, 30] + // effectiveStart = max(10, 20) = 20 + effectiveStart = Bytes.compareTo(chunkStart, mapperRegionStart) > 0 + ? chunkStart + : mapperRegionStart; + } else { + // isFirstRegionOfTable -> Mapper region [,80) effectiveStart = chunkStart + // Not an initial chunks: chunk start guaranteed to be within region boundaries, no clipping needed + effectiveStart = chunkStart; } - currentPos = effectiveEnd; + + // Determine effective end boundary for this chunk + // Only the last chunk might extend beyond mapperRegionEnd and need clipping + byte[] effectiveEnd; + if (lastChunk && !isEndRegionOfTable) { + // last Chunk, clip boundary outside of Mapper region. + // Example: Mapper region [20, 85), last chunk [70, 90] + // → effectiveEnd = min(90, 85) = 85 + effectiveEnd = Bytes.compareTo(chunkEnd, mapperRegionEnd) < 0 + ? chunkEnd + : mapperRegionEnd; + } else { + // isLastRegionOfTable -> Mapper region [80,) effectiveEnd = chunkEnd + // Not last chunk: chunk end is guaranteed to be within region boundaries, no clipping needed + effectiveEnd = chunkEnd; + } + + // Check for gap BEFORE this chunk + // If there's space between our current position and where this chunk starts, that's a gap that needs processing + // Example: scanPos=30 (processed till this key), effectiveStart=70 (chunk start key) + // Gap detected: [30, 70) needs processing + if (Bytes.compareTo(scanPos, effectiveStart) < 0) { + gaps.add(new Pair<>(scanPos, effectiveStart)); + } + // We've now "processed" up to this key + scanPos = effectiveEnd; } - if (Bytes.compareTo(currentPos, mapperRegionEnd) < 0) { - gaps.add(new Pair<>(currentPos, mapperRegionEnd)); + + // Since Mapper region end is exclusive, we want to add any remaining key boundary as gaps + // except when scanPos == mapperRegionEnd (i.e end of Mapper region boundary got covered by chunk) + if (isEndRegionOfTable || Bytes.compareTo(scanPos, mapperRegionEnd) < 0) { + gaps.add(new Pair<>(scanPos, mapperRegionEnd)); } return gaps; } + boolean shouldStartKeyBeInclusive(byte[] mapperRegionStart, List processedChunks) { + if (mapperRegionStart == null || mapperRegionStart.length == 0 || processedChunks == null || processedChunks.isEmpty()) { + return true; + } + return Bytes.compareTo(processedChunks.get(0).getStartRowKey(), mapperRegionStart) > 0; + } + @Override protected void cleanup(Context context) throws IOException, InterruptedException { tryClosingResources(); @@ -535,8 +610,8 @@ private static class ChunkInfo { @Override public String toString() { return String.format("Chunk[start=%s, end=%s, rows=%d, partial=%s, hasMoreRowsInRegion=%s]", - Bytes.toStringBinary(startKey), Bytes.toStringBinary(endKey), rowCount, isPartial, - hasMoreRowsInRegion); + Bytes.toStringBinary(startKey), Bytes.toStringBinary(endKey), rowCount, isPartial, + hasMoreRowsInRegion); } } diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java index 0d6f7c0a1cf..6ded4225147 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java @@ -54,17 +54,30 @@ public PhoenixSyncTableOutputRepository(Connection connection) { } public void createSyncCheckpointTableIfNotExists() throws SQLException { - String ddl = "CREATE TABLE IF NOT EXISTS " + SYNC_TABLE_CHECKPOINT_TABLE_NAME + " (\n" - + " TABLE_NAME VARCHAR NOT NULL,\n" + " TARGET_CLUSTER VARCHAR NOT NULL,\n" - + " ENTRY_TYPE VARCHAR(20) NOT NULL,\n" + " FROM_TIME BIGINT NOT NULL,\n" - + " TO_TIME BIGINT NOT NULL,\n" + " IS_DRY_RUN BOOLEAN NOT NULL,\n" - + " START_ROW_KEY VARBINARY NOT NULL,\n" + " END_ROW_KEY VARBINARY,\n" - + " IS_FIRST_REGION BOOLEAN, \n" + " EXECUTION_START_TIME TIMESTAMP,\n" - + " EXECUTION_END_TIME TIMESTAMP,\n" + " STATUS VARCHAR(20),\n" - + " COUNTERS VARCHAR(255), \n" + " CONSTRAINT PK PRIMARY KEY (\n" - + " TABLE_NAME,\n" + " TARGET_CLUSTER,\n" + " ENTRY_TYPE ,\n" - + " FROM_TIME,\n" + " TO_TIME,\n" + " IS_DRY_RUN,\n" - + " START_ROW_KEY )" + ") TTL=" + OUTPUT_TABLE_TTL_SECONDS; + String ddl = "CREATE TABLE IF NOT EXISTS " + + SYNC_TABLE_CHECKPOINT_TABLE_NAME+ " (\n" + + " TABLE_NAME VARCHAR NOT NULL,\n" + + " TARGET_CLUSTER VARCHAR NOT NULL,\n" + + " TYPE VARCHAR(20) NOT NULL,\n" + + " FROM_TIME BIGINT NOT NULL,\n" + + " TO_TIME BIGINT NOT NULL,\n" + + " IS_DRY_RUN BOOLEAN NOT NULL,\n" + + " START_ROW_KEY VARBINARY NOT NULL,\n" + + " END_ROW_KEY VARBINARY,\n" + + " IS_FIRST_REGION BOOLEAN, \n" + + " EXECUTION_START_TIME TIMESTAMP,\n" + + " EXECUTION_END_TIME TIMESTAMP,\n" + + " STATUS VARCHAR(20),\n" + + " COUNTERS VARCHAR(255), \n" + + " CONSTRAINT PK PRIMARY KEY (\n" + + " TABLE_NAME,\n" + + " TARGET_CLUSTER,\n" + + " TYPE ,\n" + + " FROM_TIME,\n" + + " TO_TIME,\n" + + " IS_DRY_RUN,\n" + + " START_ROW_KEY )" + + ") TTL=" + OUTPUT_TABLE_TTL_SECONDS; try (Statement stmt = connection.createStatement()) { stmt.execute(ddl); @@ -93,7 +106,7 @@ public void checkpointSyncTableResult(String tableName, String targetCluster, Ty } String upsert = "UPSERT INTO " + SYNC_TABLE_CHECKPOINT_TABLE_NAME - + " (TABLE_NAME, TARGET_CLUSTER, ENTRY_TYPE, FROM_TIME, TO_TIME, IS_DRY_RUN," + + " (TABLE_NAME, TARGET_CLUSTER, TYPE, FROM_TIME, TO_TIME, IS_DRY_RUN," + " START_ROW_KEY, END_ROW_KEY, IS_FIRST_REGION, EXECUTION_START_TIME, EXECUTION_END_TIME," + " STATUS, COUNTERS) " + "VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)"; @@ -105,8 +118,8 @@ public void checkpointSyncTableResult(String tableName, String targetCluster, Ty ps.setString(1, tableName); ps.setString(2, targetCluster); ps.setString(3, type.name()); - ps.setObject(4, fromTime); - ps.setObject(5, toTime); + ps.setLong(4, fromTime); + ps.setLong(5, toTime); ps.setBoolean(6, isDryRun); ps.setBytes(7, effectiveStartKey); ps.setBytes(8, endKey); @@ -145,7 +158,7 @@ public List getProcessedMapperRegions(String tableNam String query = "SELECT START_ROW_KEY, END_ROW_KEY, IS_FIRST_REGION FROM " + SYNC_TABLE_CHECKPOINT_TABLE_NAME + " WHERE TABLE_NAME = ? AND TARGET_CLUSTER = ?" - + " AND ENTRY_TYPE = ? AND FROM_TIME = ? AND TO_TIME = ? AND STATUS IN ( ?, ?)"; + + " AND TYPE = ? AND FROM_TIME = ? AND TO_TIME = ? AND STATUS IN ( ?, ?)"; List results = new ArrayList<>(); try (PreparedStatement ps = connection.prepareStatement(query)) { int paramIndex = 1; @@ -182,21 +195,42 @@ public List getProcessedMapperRegions(String tableNam public List getProcessedChunks(String tableName, String targetCluster, Long fromTime, Long toTime, byte[] mapperRegionStart, byte[] mapperRegionEnd) throws SQLException { - String query = "SELECT START_ROW_KEY, END_ROW_KEY, IS_FIRST_REGION FROM " - + SYNC_TABLE_CHECKPOINT_TABLE_NAME + " WHERE TABLE_NAME = ? AND TARGET_CLUSTER = ? " - + " AND ENTRY_TYPE = ? AND FROM_TIME = ? AND TO_TIME = ? AND START_ROW_KEY < ? " - + " AND END_ROW_KEY > ? AND STATUS IN (?, ?) "; + StringBuilder queryBuilder = new StringBuilder(); + queryBuilder.append("SELECT START_ROW_KEY, END_ROW_KEY, IS_FIRST_REGION FROM " + + SYNC_TABLE_CHECKPOINT_TABLE_NAME + + " WHERE TABLE_NAME = ? AND TARGET_CLUSTER = ? " + + " AND TYPE = ? AND FROM_TIME = ? AND TO_TIME = ?"); + + // Check if mapper region boundaries are non-empty (i.e., NOT first/last regions) + // Only add boundary conditions for non-empty boundaries + boolean hasEndBoundary = mapperRegionEnd != null && mapperRegionEnd.length > 0; + boolean hasStartBoundary = mapperRegionStart != null && mapperRegionStart.length > 0; + + // Filter chunks that overlap with this mapper region: + // - Chunk overlaps if: chunkStart < mapperRegionEnd (when end boundary exists) + // - Chunk overlaps if: chunkEnd > mapperRegionStart (when start boundary exists) + if (hasEndBoundary) { + queryBuilder.append(" AND START_ROW_KEY <= ?"); + } + if (hasStartBoundary) { + queryBuilder.append(" AND END_ROW_KEY >= ?"); + } + queryBuilder.append(" AND STATUS IN (?, ?)"); List results = new ArrayList<>(); - try (PreparedStatement ps = connection.prepareStatement(query)) { + try (PreparedStatement ps = connection.prepareStatement(queryBuilder.toString())) { int paramIndex = 1; ps.setString(paramIndex++, tableName); ps.setString(paramIndex++, targetCluster); ps.setString(paramIndex++, Type.CHUNK.name()); ps.setLong(paramIndex++, fromTime); ps.setLong(paramIndex++, toTime); - ps.setBytes(paramIndex++, mapperRegionEnd); - ps.setBytes(paramIndex++, mapperRegionStart); + if (hasEndBoundary) { + ps.setBytes(paramIndex++, mapperRegionEnd); + } + if (hasStartBoundary) { + ps.setBytes(paramIndex++, mapperRegionStart); + } ps.setString(paramIndex++, Status.VERIFIED.name()); ps.setString(paramIndex, Status.MISMATCHED.name()); try (ResultSet rs = ps.executeQuery()) { diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java index cd3b61ba2c1..80ed0e5e0e6 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java @@ -18,6 +18,8 @@ package org.apache.phoenix.mapreduce; import java.util.Arrays; + +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.util.Bytes; /** @@ -63,12 +65,14 @@ public Builder() { } public Builder setStartRowKey(byte[] startRowKey) { - row.startRowKey = startRowKey != null ? Arrays.copyOf(startRowKey, startRowKey.length) : null; + row.startRowKey = Arrays.copyOf(startRowKey, startRowKey.length); return this; } public Builder setEndRowKey(byte[] endRowKey) { - row.endRowKey = endRowKey != null ? Arrays.copyOf(endRowKey, endRowKey.length) : null; + row.endRowKey = (endRowKey == null || endRowKey.length == 0) + ? HConstants.EMPTY_END_ROW + : Arrays.copyOf(endRowKey, endRowKey.length); return this; } From 3f8633858413ad9a86950d8a4bd994efdccd2f1b Mon Sep 17 00:00:00 2001 From: Rahul Kumar Date: Tue, 17 Feb 2026 12:59:56 +0530 Subject: [PATCH 6/9] mvn spotless apply --- .../PhoenixSyncTableInputFormat.java | 15 +- .../mapreduce/PhoenixSyncTableMapper.java | 174 +++++++++--------- .../PhoenixSyncTableOutputRepository.java | 40 ++-- .../mapreduce/PhoenixSyncTableOutputRow.java | 5 +- .../mapreduce/util/PhoenixMapReduceUtil.java | 2 +- 5 files changed, 116 insertions(+), 120 deletions(-) diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java index 87ceba090ca..2e6452bee9f 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java @@ -152,14 +152,17 @@ private List filterCompletedSplits(List allSplits, byte[] completedStart = completedRange.getLowerRange(); byte[] completedEnd = completedRange.getUpperRange(); - // No overlap b/w completedRange/splitRange. // completedEnd is before splitStart, increment completed pointer to catch up - if (!Bytes.equals(completedEnd, HConstants.EMPTY_END_ROW) - && Bytes.compareTo(completedEnd, splitStart) <= 0) { + if ( + !Bytes.equals(completedEnd, HConstants.EMPTY_END_ROW) + && Bytes.compareTo(completedEnd, splitStart) <= 0 + ) { completedIdx++; - } else if (!Bytes.equals(splitEnd, HConstants.EMPTY_END_ROW) - && Bytes.compareTo(completedStart, splitEnd) >= 0) { + } else if ( + !Bytes.equals(splitEnd, HConstants.EMPTY_END_ROW) + && Bytes.compareTo(completedStart, splitEnd) >= 0 + ) { // No overlap. completedStart is after splitEnd, splitRange needs to be processed, // add to unprocessed list and increment unprocessedSplits.add(allSplits.get(splitIdx)); @@ -170,7 +173,7 @@ private List filterCompletedSplits(List allSplits, boolean startContained = Bytes.compareTo(completedStart, splitStart) <= 0; boolean endContained = Bytes.equals(completedEnd, HConstants.EMPTY_END_ROW) - || Bytes.compareTo(splitEnd, completedEnd) <= 0; + || Bytes.compareTo(splitEnd, completedEnd) <= 0; boolean fullyContained = startContained && endContained; if (!fullyContained) { diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java index 595908d48f3..e0ede2e9166 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java @@ -25,7 +25,6 @@ import java.sql.SQLException; import java.sql.Timestamp; import java.util.*; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; @@ -58,7 +57,7 @@ * chunk hashes from both clusters, compares them and write to checkpoint table. */ public class PhoenixSyncTableMapper - extends Mapper { + extends Mapper { private static final Logger LOGGER = LoggerFactory.getLogger(PhoenixSyncTableMapper.class); @@ -108,7 +107,7 @@ protected void setup(Context context) throws InterruptedException { } catch (SQLException | IOException e) { tryClosingResources(); throw new RuntimeException( - String.format("Failed to setup PhoenixSyncTableMapper for table: %s", tableName), e); + String.format("Failed to setup PhoenixSyncTableMapper for table: %s", tableName), e); } } @@ -120,8 +119,8 @@ private void extractRegionBoundariesFromSplit(Context context) { KeyRange keyRange = split.getKeyRange(); if (keyRange == null) { throw new IllegalStateException(String.format( - "PhoenixInputSplit has no KeyRange for table: %s . Cannot determine region boundaries for sync operation.", - tableName)); + "PhoenixInputSplit has no KeyRange for table: %s . Cannot determine region boundaries for sync operation.", + tableName)); } mapperRegionStart = keyRange.getLowerRange(); mapperRegionEnd = keyRange.getUpperRange(); @@ -132,7 +131,7 @@ private void extractRegionBoundariesFromSplit(Context context) { */ private void connectToTargetCluster() throws SQLException, IOException { Configuration targetConf = - PhoenixMapReduceUtil.createConfigurationForZkQuorum(conf, targetZkQuorum); + PhoenixMapReduceUtil.createConfigurationForZkQuorum(conf, targetZkQuorum); targetConnection = ConnectionUtil.getInputConnection(targetConf); } @@ -153,21 +152,25 @@ private Connection createGlobalConnection(Configuration conf) throws SQLExceptio */ @Override protected void map(NullWritable key, DBInputFormat.NullDBWritable value, Context context) - throws IOException, InterruptedException { + throws IOException, InterruptedException { context.getCounter(PhoenixJobCounters.INPUT_RECORDS).increment(1); try { List processedChunks = - syncTableOutputRepository.getProcessedChunks(tableName, targetZkQuorum, fromTime, toTime, - mapperRegionStart, mapperRegionEnd); + syncTableOutputRepository.getProcessedChunks(tableName, targetZkQuorum, fromTime, toTime, + mapperRegionStart, mapperRegionEnd); List> unprocessedRanges = - calculateUnprocessedRanges(mapperRegionStart, mapperRegionEnd, processedChunks); - // Checking if start key should be inclusive, this is specific to scenario when there are processed + calculateUnprocessedRanges(mapperRegionStart, mapperRegionEnd, processedChunks); + // Checking if start key should be inclusive, this is specific to scenario when there are + // processed // chunks within this Mapper region boundary. boolean isStartKeyInclusive = shouldStartKeyBeInclusive(mapperRegionStart, processedChunks); for (Pair range : unprocessedRanges) { - // From a Mapper region boundary, if we get multiple fragments of ranges it means there were some processed chunks in this region boundary - // And since chunks are inclusive of start and endKey, we just needed to confirm whether first fragment of range should have start key inclusive. - // Other fragment will not have start key inclusive since these ranges are carved out of chunk boundary, + // From a Mapper region boundary, if we get multiple fragments of ranges it means there were + // some processed chunks in this region boundary + // And since chunks are inclusive of start and endKey, we just needed to confirm whether + // first fragment of range should have start key inclusive. + // Other fragment will not have start key inclusive since these ranges are carved out of + // chunk boundary, // both start and key would already have been processed as part of chunk itself processMapperRanges(range.getFirst(), range.getSecond(), isStartKeyInclusive, context); isStartKeyInclusive = false; @@ -189,26 +192,26 @@ protected void map(NullWritable key, DBInputFormat.NullDBWritable value, Context if (mismatchedChunk == 0) { context.getCounter(PhoenixJobCounters.OUTPUT_RECORDS).increment(1); syncTableOutputRepository.checkpointSyncTableResult(tableName, targetZkQuorum, - PhoenixSyncTableOutputRow.Type.MAPPER_REGION, fromTime, toTime, isDryRun, - mapperRegionStart, mapperRegionEnd, PhoenixSyncTableOutputRow.Status.VERIFIED, - mapperStartTime, mapperEndTime, counters); + PhoenixSyncTableOutputRow.Type.MAPPER_REGION, fromTime, toTime, isDryRun, + mapperRegionStart, mapperRegionEnd, PhoenixSyncTableOutputRow.Status.VERIFIED, + mapperStartTime, mapperEndTime, counters); LOGGER.info( - "PhoenixSyncTable mapper completed with verified: {} verifiedChunk chunks, {} mismatchedChunk chunks", - verifiedChunk, mismatchedChunk); + "PhoenixSyncTable mapper completed with verified: {} verifiedChunk chunks, {} mismatchedChunk chunks", + verifiedChunk, mismatchedChunk); } else { context.getCounter(PhoenixJobCounters.FAILED_RECORDS).increment(1); LOGGER.warn( - "PhoenixSyncTable mapper completed with mismatch: {} verifiedChunk chunks, {} mismatchedChunk chunks", - verifiedChunk, mismatchedChunk); + "PhoenixSyncTable mapper completed with mismatch: {} verifiedChunk chunks, {} mismatchedChunk chunks", + verifiedChunk, mismatchedChunk); syncTableOutputRepository.checkpointSyncTableResult(tableName, targetZkQuorum, - PhoenixSyncTableOutputRow.Type.MAPPER_REGION, fromTime, toTime, isDryRun, - mapperRegionStart, mapperRegionEnd, PhoenixSyncTableOutputRow.Status.MISMATCHED, - mapperStartTime, mapperEndTime, counters); + PhoenixSyncTableOutputRow.Type.MAPPER_REGION, fromTime, toTime, isDryRun, + mapperRegionStart, mapperRegionEnd, PhoenixSyncTableOutputRow.Status.MISMATCHED, + mapperStartTime, mapperEndTime, counters); } } else { LOGGER.info( - "No rows pending to process. All mapper region boundaries are covered for startKey:{}, endKey: {}", - mapperRegionStart, mapperRegionEnd); + "No rows pending to process. All mapper region boundaries are covered for startKey:{}, endKey: {}", + mapperRegionStart, mapperRegionEnd); } } catch (SQLException e) { tryClosingResources(); @@ -228,10 +231,10 @@ protected void map(NullWritable key, DBInputFormat.NullDBWritable value, Context * @throws IOException if scan fails * @throws SQLException if database operations fail */ - private void processMapperRanges(byte[] rangeStart, byte[] rangeEnd, boolean isStartKeyInclusive, Context context) - throws IOException, SQLException { + private void processMapperRanges(byte[] rangeStart, byte[] rangeEnd, boolean isStartKeyInclusive, + Context context) throws IOException, SQLException { try (ChunkScannerContext sourceScanner = createChunkScanner(sourceConnection, rangeStart, - rangeEnd, null, isStartKeyInclusive, false, false)) { + rangeEnd, null, isStartKeyInclusive, false, false)) { while (true) { // We only try to get one chunked metadata row returned at a time until no more chunk // returned(i.e null) @@ -240,17 +243,16 @@ private void processMapperRanges(byte[] rangeStart, byte[] rangeEnd, boolean isS break; } sourceChunk.executionStartTime = new Timestamp(System.currentTimeMillis()); - ChunkInfo targetChunk = - getTargetChunkWithSourceBoundary(targetConnection, sourceChunk.startKey, - sourceChunk.endKey); + ChunkInfo targetChunk = getTargetChunkWithSourceBoundary(targetConnection, + sourceChunk.startKey, sourceChunk.endKey); context.getCounter(SyncCounters.SOURCE_ROWS_PROCESSED).increment(sourceChunk.rowCount); context.getCounter(SyncCounters.TARGET_ROWS_PROCESSED).increment(targetChunk.rowCount); boolean matched = MessageDigest.isEqual(sourceChunk.hash, targetChunk.hash); if (LOGGER.isDebugEnabled()) { LOGGER.debug("Chunk comparison {}, {}: source={} rows, target={} rows, matched={}", - Bytes.toStringBinary(sourceChunk.startKey), Bytes.toStringBinary(sourceChunk.endKey), - sourceChunk.rowCount, targetChunk.rowCount, matched); + Bytes.toStringBinary(sourceChunk.startKey), Bytes.toStringBinary(sourceChunk.endKey), + sourceChunk.rowCount, targetChunk.rowCount, matched); } sourceChunk.executionEndTime = new Timestamp(System.currentTimeMillis()); Map mapperCounters = new LinkedHashMap<>(); @@ -266,7 +268,7 @@ private void processMapperRanges(byte[] rangeStart, byte[] rangeEnd, boolean isS } } LOGGER.info("Completed sync table processing of Mapper region boundary {}, {}", - Bytes.toStringBinary(rangeStart), Bytes.toStringBinary(rangeEnd)); + Bytes.toStringBinary(rangeStart), Bytes.toStringBinary(rangeEnd)); } /** @@ -281,7 +283,7 @@ private void processMapperRanges(byte[] rangeStart, byte[] rangeEnd, boolean isS * @return Single ChunkInfo with final hash from all target regions */ private ChunkInfo getTargetChunkWithSourceBoundary(Connection conn, byte[] startKey, - byte[] endKey) throws IOException, SQLException { + byte[] endKey) throws IOException, SQLException { ChunkInfo combinedTargetChunk = new ChunkInfo(); combinedTargetChunk.startKey = startKey; combinedTargetChunk.endKey = endKey; @@ -299,7 +301,7 @@ private ChunkInfo getTargetChunkWithSourceBoundary(Connection conn, byte[] start // immediately(that's why we set scan.setLimit(1)), since output from one region partial chunk // scanner is input to next region scanner. try (ChunkScannerContext scanner = createChunkScanner(conn, currentStartKey, endKey, - continuedDigestState, isStartKeyInclusive, true, true)) { + continuedDigestState, isStartKeyInclusive, true, true)) { ChunkInfo chunk = scanner.getNextChunk(); // In a happy path where source and target rows are matching, target chunk would never be // null. @@ -340,24 +342,24 @@ private ChunkInfo getTargetChunkWithSourceBoundary(Connection conn, byte[] start * @throws SQLException hTable connection fails */ private ChunkScannerContext createChunkScanner(Connection conn, byte[] startKey, byte[] endKey, - byte[] continuedDigestState, boolean isStartKeyInclusive, boolean isEndKeyInclusive, - boolean isTargetScan) throws IOException, SQLException { + byte[] continuedDigestState, boolean isStartKeyInclusive, boolean isEndKeyInclusive, + boolean isTargetScan) throws IOException, SQLException { // Not using try-with-resources since ChunkScannerContext owns the table lifecycle Table hTable = - conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(physicalTableName); + conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(physicalTableName); Scan scan = - createChunkScan(startKey, endKey, isStartKeyInclusive, isEndKeyInclusive, isTargetScan); + createChunkScan(startKey, endKey, isStartKeyInclusive, isEndKeyInclusive, isTargetScan); scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CHUNK_FORMATION, TRUE_BYTES); scan.setAttribute(BaseScannerRegionObserverConstants.SKIP_REGION_BOUNDARY_CHECK, TRUE_BYTES); scan.setAttribute(BaseScannerRegionObserverConstants.UNGROUPED_AGG, TRUE_BYTES); if (continuedDigestState != null && continuedDigestState.length > 0) { scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CONTINUED_DIGEST_STATE, - continuedDigestState); + continuedDigestState); } if (!isTargetScan) { scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CHUNK_SIZE_BYTES, - Bytes.toBytes(chunkSizeBytes)); + Bytes.toBytes(chunkSizeBytes)); } ResultScanner scanner = hTable.getScanner(scan); return new ChunkScannerContext(hTable, scanner); @@ -370,17 +372,18 @@ private ChunkScannerContext createChunkScanner(Connection conn, byte[] startKey, */ private ChunkInfo parseChunkInfo(Result result) { List cells = Arrays.asList(result.rawCells()); - Cell endKeyCell = MetaDataUtil.getCell(cells, - BaseScannerRegionObserverConstants.SYNC_TABLE_END_KEY_QUALIFIER); + Cell endKeyCell = + MetaDataUtil.getCell(cells, BaseScannerRegionObserverConstants.SYNC_TABLE_END_KEY_QUALIFIER); Cell rowCountCell = MetaDataUtil.getCell(cells, - BaseScannerRegionObserverConstants.SYNC_TABLE_ROW_COUNT_QUALIFIER); + BaseScannerRegionObserverConstants.SYNC_TABLE_ROW_COUNT_QUALIFIER); Cell isPartialChunkCell = MetaDataUtil.getCell(cells, - BaseScannerRegionObserverConstants.SYNC_TABLE_IS_PARTIAL_CHUNK_QUALIFIER); + BaseScannerRegionObserverConstants.SYNC_TABLE_IS_PARTIAL_CHUNK_QUALIFIER); Cell hashCell = - MetaDataUtil.getCell(cells, BaseScannerRegionObserverConstants.SYNC_TABLE_HASH_QUALIFIER); + MetaDataUtil.getCell(cells, BaseScannerRegionObserverConstants.SYNC_TABLE_HASH_QUALIFIER); - if (endKeyCell == null || rowCountCell == null || isPartialChunkCell == null - || hashCell == null) { + if ( + endKeyCell == null || rowCountCell == null || isPartialChunkCell == null || hashCell == null + ) { throw new RuntimeException("Missing required chunk metadata cells."); } @@ -388,7 +391,7 @@ private ChunkInfo parseChunkInfo(Result result) { info.startKey = result.getRow(); info.endKey = CellUtil.cloneValue(endKeyCell); info.rowCount = Bytes.toLong(rowCountCell.getValueArray(), rowCountCell.getValueOffset(), - rowCountCell.getValueLength()); + rowCountCell.getValueLength()); info.isPartial = isPartialChunkCell.getValueArray()[isPartialChunkCell.getValueOffset()] != 0; info.hash = CellUtil.cloneValue(hashCell); return info; @@ -420,11 +423,11 @@ private String formatCounters(Map counters) { * */ private void handleVerifiedChunk(ChunkInfo sourceChunk, Context context, String counters) - throws SQLException { + throws SQLException { syncTableOutputRepository.checkpointSyncTableResult(tableName, targetZkQuorum, - PhoenixSyncTableOutputRow.Type.CHUNK, fromTime, toTime, isDryRun, sourceChunk.startKey, - sourceChunk.endKey, PhoenixSyncTableOutputRow.Status.VERIFIED, - sourceChunk.executionStartTime, sourceChunk.executionEndTime, counters); + PhoenixSyncTableOutputRow.Type.CHUNK, fromTime, toTime, isDryRun, sourceChunk.startKey, + sourceChunk.endKey, PhoenixSyncTableOutputRow.Status.VERIFIED, sourceChunk.executionStartTime, + sourceChunk.executionEndTime, counters); context.getCounter(SyncCounters.CHUNKS_VERIFIED).increment(1); } @@ -432,13 +435,13 @@ private void handleVerifiedChunk(ChunkInfo sourceChunk, Context context, String * */ private void handleMismatchedChunk(ChunkInfo sourceChunk, Context context, String counters) - throws SQLException { + throws SQLException { LOGGER.warn("Chunk mismatch detected for table: {}, with startKey: {}, endKey {}", tableName, - Bytes.toStringBinary(sourceChunk.startKey), Bytes.toStringBinary(sourceChunk.endKey)); + Bytes.toStringBinary(sourceChunk.startKey), Bytes.toStringBinary(sourceChunk.endKey)); syncTableOutputRepository.checkpointSyncTableResult(tableName, targetZkQuorum, - PhoenixSyncTableOutputRow.Type.CHUNK, fromTime, toTime, isDryRun, sourceChunk.startKey, - sourceChunk.endKey, PhoenixSyncTableOutputRow.Status.MISMATCHED, - sourceChunk.executionStartTime, sourceChunk.executionEndTime, counters); + PhoenixSyncTableOutputRow.Type.CHUNK, fromTime, toTime, isDryRun, sourceChunk.startKey, + sourceChunk.endKey, PhoenixSyncTableOutputRow.Status.MISMATCHED, + sourceChunk.executionStartTime, sourceChunk.executionEndTime, counters); context.getCounter(SyncCounters.CHUNKS_MISMATCHED).increment(1); } @@ -447,7 +450,7 @@ private void handleMismatchedChunk(ChunkInfo sourceChunk, Context context, Strin * Creates a Hbase raw scan for a chunk range to capture all cell versions and delete markers. */ private Scan createChunkScan(byte[] startKey, byte[] endKey, boolean isStartKeyInclusive, - boolean isEndKeyInclusive, boolean isTargetScan) throws IOException { + boolean isEndKeyInclusive, boolean isTargetScan) throws IOException { Scan scan = new Scan(); scan.withStartRow(startKey, isStartKeyInclusive); scan.withStopRow(endKey, isEndKeyInclusive); @@ -464,12 +467,12 @@ private Scan createChunkScan(byte[] startKey, byte[] endKey, boolean isStartKeyI /** * Calculates unprocessed gaps within a mapper region. Since a mapper region is divided into - * multiple chunks and we store mapper region boundary and its chunked boundary. - * Once we have all the processedChunks which falls in this Mapper region boundary, - * we look for holes/gaps in mapper region boundary which haven't been processed as chunks. - * Given a list of processed chunks, returns the ranges that haven't been processed yet. - * This will be useful on retries if Region boundary has changed and we some chunks in the - * new region boundary has been processed which can be skipped + * multiple chunks and we store mapper region boundary and its chunked boundary. Once we have all + * the processedChunks which falls in this Mapper region boundary, we look for holes/gaps in + * mapper region boundary which haven't been processed as chunks. Given a list of processed + * chunks, returns the ranges that haven't been processed yet. This will be useful on retries if + * Region boundary has changed and we some chunks in the new region boundary has been processed + * which can be skipped * @param mapperRegionStart Start of mapper region * @param mapperRegionEnd End of mapper region * @param processedChunks List of already-processed chunks from getProcessedChunks() @@ -477,7 +480,7 @@ private Scan createChunkScan(byte[] startKey, byte[] endKey, boolean isStartKeyI */ @VisibleForTesting public List> calculateUnprocessedRanges(byte[] mapperRegionStart, - byte[] mapperRegionEnd, List processedChunks) { + byte[] mapperRegionEnd, List processedChunks) { List> gaps = new ArrayList<>(); // If processedChunks is null or empty, the entire mapper region needs processing if (processedChunks == null || processedChunks.isEmpty()) { @@ -513,12 +516,12 @@ public List> calculateUnprocessedRanges(byte[] mapperRegion // initialChunk chunk, clip boundary outside of Mapper region. // Example: Mapper region [20, 85), first chunk [10, 30] // effectiveStart = max(10, 20) = 20 - effectiveStart = Bytes.compareTo(chunkStart, mapperRegionStart) > 0 - ? chunkStart - : mapperRegionStart; + effectiveStart = + Bytes.compareTo(chunkStart, mapperRegionStart) > 0 ? chunkStart : mapperRegionStart; } else { // isFirstRegionOfTable -> Mapper region [,80) effectiveStart = chunkStart - // Not an initial chunks: chunk start guaranteed to be within region boundaries, no clipping needed + // Not an initial chunks: chunk start guaranteed to be within region boundaries, no clipping + // needed effectiveStart = chunkStart; } @@ -528,18 +531,18 @@ public List> calculateUnprocessedRanges(byte[] mapperRegion if (lastChunk && !isEndRegionOfTable) { // last Chunk, clip boundary outside of Mapper region. // Example: Mapper region [20, 85), last chunk [70, 90] - // → effectiveEnd = min(90, 85) = 85 - effectiveEnd = Bytes.compareTo(chunkEnd, mapperRegionEnd) < 0 - ? chunkEnd - : mapperRegionEnd; + // → effectiveEnd = min(90, 85) = 85 + effectiveEnd = Bytes.compareTo(chunkEnd, mapperRegionEnd) < 0 ? chunkEnd : mapperRegionEnd; } else { // isLastRegionOfTable -> Mapper region [80,) effectiveEnd = chunkEnd - // Not last chunk: chunk end is guaranteed to be within region boundaries, no clipping needed + // Not last chunk: chunk end is guaranteed to be within region boundaries, no clipping + // needed effectiveEnd = chunkEnd; } // Check for gap BEFORE this chunk - // If there's space between our current position and where this chunk starts, that's a gap that needs processing + // If there's space between our current position and where this chunk starts, that's a gap + // that needs processing // Example: scanPos=30 (processed till this key), effectiveStart=70 (chunk start key) // Gap detected: [30, 70) needs processing if (Bytes.compareTo(scanPos, effectiveStart) < 0) { @@ -550,15 +553,20 @@ public List> calculateUnprocessedRanges(byte[] mapperRegion } // Since Mapper region end is exclusive, we want to add any remaining key boundary as gaps - // except when scanPos == mapperRegionEnd (i.e end of Mapper region boundary got covered by chunk) + // except when scanPos == mapperRegionEnd (i.e end of Mapper region boundary got covered by + // chunk) if (isEndRegionOfTable || Bytes.compareTo(scanPos, mapperRegionEnd) < 0) { gaps.add(new Pair<>(scanPos, mapperRegionEnd)); } return gaps; } - boolean shouldStartKeyBeInclusive(byte[] mapperRegionStart, List processedChunks) { - if (mapperRegionStart == null || mapperRegionStart.length == 0 || processedChunks == null || processedChunks.isEmpty()) { + boolean shouldStartKeyBeInclusive(byte[] mapperRegionStart, + List processedChunks) { + if ( + mapperRegionStart == null || mapperRegionStart.length == 0 || processedChunks == null + || processedChunks.isEmpty() + ) { return true; } return Bytes.compareTo(processedChunks.get(0).getStartRowKey(), mapperRegionStart) > 0; @@ -610,8 +618,8 @@ private static class ChunkInfo { @Override public String toString() { return String.format("Chunk[start=%s, end=%s, rows=%d, partial=%s, hasMoreRowsInRegion=%s]", - Bytes.toStringBinary(startKey), Bytes.toStringBinary(endKey), rowCount, isPartial, - hasMoreRowsInRegion); + Bytes.toStringBinary(startKey), Bytes.toStringBinary(endKey), rowCount, isPartial, + hasMoreRowsInRegion); } } diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java index 6ded4225147..722c33c9f10 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java @@ -54,30 +54,17 @@ public PhoenixSyncTableOutputRepository(Connection connection) { } public void createSyncCheckpointTableIfNotExists() throws SQLException { - String ddl = "CREATE TABLE IF NOT EXISTS " - + SYNC_TABLE_CHECKPOINT_TABLE_NAME+ " (\n" - + " TABLE_NAME VARCHAR NOT NULL,\n" - + " TARGET_CLUSTER VARCHAR NOT NULL,\n" - + " TYPE VARCHAR(20) NOT NULL,\n" - + " FROM_TIME BIGINT NOT NULL,\n" - + " TO_TIME BIGINT NOT NULL,\n" - + " IS_DRY_RUN BOOLEAN NOT NULL,\n" - + " START_ROW_KEY VARBINARY NOT NULL,\n" - + " END_ROW_KEY VARBINARY,\n" - + " IS_FIRST_REGION BOOLEAN, \n" - + " EXECUTION_START_TIME TIMESTAMP,\n" - + " EXECUTION_END_TIME TIMESTAMP,\n" - + " STATUS VARCHAR(20),\n" - + " COUNTERS VARCHAR(255), \n" - + " CONSTRAINT PK PRIMARY KEY (\n" - + " TABLE_NAME,\n" - + " TARGET_CLUSTER,\n" - + " TYPE ,\n" - + " FROM_TIME,\n" - + " TO_TIME,\n" - + " IS_DRY_RUN,\n" - + " START_ROW_KEY )" - + ") TTL=" + OUTPUT_TABLE_TTL_SECONDS; + String ddl = "CREATE TABLE IF NOT EXISTS " + SYNC_TABLE_CHECKPOINT_TABLE_NAME + " (\n" + + " TABLE_NAME VARCHAR NOT NULL,\n" + " TARGET_CLUSTER VARCHAR NOT NULL,\n" + + " TYPE VARCHAR(20) NOT NULL,\n" + " FROM_TIME BIGINT NOT NULL,\n" + + " TO_TIME BIGINT NOT NULL,\n" + " IS_DRY_RUN BOOLEAN NOT NULL,\n" + + " START_ROW_KEY VARBINARY NOT NULL,\n" + " END_ROW_KEY VARBINARY,\n" + + " IS_FIRST_REGION BOOLEAN, \n" + " EXECUTION_START_TIME TIMESTAMP,\n" + + " EXECUTION_END_TIME TIMESTAMP,\n" + " STATUS VARCHAR(20),\n" + + " COUNTERS VARCHAR(255), \n" + " CONSTRAINT PK PRIMARY KEY (\n" + + " TABLE_NAME,\n" + " TARGET_CLUSTER,\n" + " TYPE ,\n" + + " FROM_TIME,\n" + " TO_TIME,\n" + " IS_DRY_RUN,\n" + + " START_ROW_KEY )" + ") TTL=" + OUTPUT_TABLE_TTL_SECONDS; try (Statement stmt = connection.createStatement()) { stmt.execute(ddl); @@ -197,9 +184,8 @@ public List getProcessedChunks(String tableName, Stri throws SQLException { StringBuilder queryBuilder = new StringBuilder(); queryBuilder.append("SELECT START_ROW_KEY, END_ROW_KEY, IS_FIRST_REGION FROM " - + SYNC_TABLE_CHECKPOINT_TABLE_NAME - + " WHERE TABLE_NAME = ? AND TARGET_CLUSTER = ? " - + " AND TYPE = ? AND FROM_TIME = ? AND TO_TIME = ?"); + + SYNC_TABLE_CHECKPOINT_TABLE_NAME + " WHERE TABLE_NAME = ? AND TARGET_CLUSTER = ? " + + " AND TYPE = ? AND FROM_TIME = ? AND TO_TIME = ?"); // Check if mapper region boundaries are non-empty (i.e., NOT first/last regions) // Only add boundary conditions for non-empty boundaries diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java index 80ed0e5e0e6..b9a284aae97 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java @@ -18,7 +18,6 @@ package org.apache.phoenix.mapreduce; import java.util.Arrays; - import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.util.Bytes; @@ -71,8 +70,8 @@ public Builder setStartRowKey(byte[] startRowKey) { public Builder setEndRowKey(byte[] endRowKey) { row.endRowKey = (endRowKey == null || endRowKey.length == 0) - ? HConstants.EMPTY_END_ROW - : Arrays.copyOf(endRowKey, endRowKey.length); + ? HConstants.EMPTY_END_ROW + : Arrays.copyOf(endRowKey, endRowKey.length); return this; } diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixMapReduceUtil.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixMapReduceUtil.java index cad5032d326..c7c74601f60 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixMapReduceUtil.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixMapReduceUtil.java @@ -281,7 +281,7 @@ public static void validateMaxLookbackAge(Configuration configuration, Long endT long minTimestamp = EnvironmentEdgeManager.currentTimeMillis() - maxLookBackAge; if (endTime < minTimestamp) { throw new IllegalArgumentException(String.format( - "Table %s sync can't look back past the configured max lookback age: %d ms. " + "Table %s can't look back past the configured max lookback age: %d ms. " + "End time: %d, Min allowed timestamp: %d", tableName, maxLookBackAge, endTime, minTimestamp)); } From b75cc140d8af5147db0e1dbbbd5583715b2d7b8e Mon Sep 17 00:00:00 2001 From: Rahul Kumar Date: Tue, 17 Feb 2026 19:24:17 +0530 Subject: [PATCH 7/9] adding verbose comments --- .../phoenix/query/QueryServicesOptions.java | 1 + .../PhoenixSyncTableRegionScanner.java | 61 ++++++------- .../PhoenixNoOpSingleRecordReader.java | 3 +- .../PhoenixSyncTableInputFormat.java | 35 +++++-- .../mapreduce/PhoenixSyncTableMapper.java | 91 ++++++++++--------- .../PhoenixSyncTableOutputRepository.java | 12 +-- .../mapreduce/PhoenixSyncTableTool.java | 83 ++++++----------- 7 files changed, 138 insertions(+), 148 deletions(-) diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java index e3854ae4f49..b9fd587dbd6 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java @@ -248,6 +248,7 @@ public class QueryServicesOptions { // hrs public static final long DEFAULT_INDEX_PENDING_DISABLE_THRESHOLD = 30000; // 30 secs + // 30 min scan timeout * 5 tries, with 2100ms total pause time between retries public static final long DEFAULT_SYNC_TABLE_QUERY_TIMEOUT = (5 * 30000 * 60) + 2100; public static final long DEFAULT_SYNC_TABLE_RPC_TIMEOUT = 30000 * 60; // 30 mins public static final long DEFAULT_SYNC_TABLE_CLIENT_SCANNER_TIMEOUT = 30000 * 60; // 30 mins diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java index acc4e2e2342..8ac9755d4b1 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java @@ -24,10 +24,9 @@ import static org.apache.phoenix.schema.types.PDataType.TRUE_BYTES; import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; import java.io.DataInputStream; -import java.io.DataOutputStream; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; import org.apache.hadoop.hbase.Cell; @@ -77,10 +76,14 @@ public class PhoenixSyncTableRegionScanner extends BaseRegionScanner { private byte[] chunkEndKey = null; private long currentChunkSize = 0L; private long currentChunkRowCount = 0L; + // We are not using jdk bundled SHA, since their digest can't be serialization/deserialization + // which is needed for passing around partial chunk private SHA256Digest digest; private boolean hasMoreRows = true; - private boolean isUsingContinuedDigest; // If target chunk was partial, and we are continuing to - // update digest before calculating checksum + // If target chunk was partial, and we are continuing to + // update digest before calculating checksum + private boolean isUsingContinuedDigest; + private final byte[] timestampBuffer = new byte[8]; /** * @param innerScanner The underlying region scanner @@ -235,14 +238,15 @@ private void updateDigestWithRow(byte[] rowKey, List cells) { digest.update(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()); long ts = cell.getTimestamp(); // Big-Endian Byte Serialization - digest.update((byte) (ts >>> 56)); - digest.update((byte) (ts >>> 48)); - digest.update((byte) (ts >>> 40)); - digest.update((byte) (ts >>> 32)); - digest.update((byte) (ts >>> 24)); - digest.update((byte) (ts >>> 16)); - digest.update((byte) (ts >>> 8)); - digest.update((byte) (ts)); + timestampBuffer[0] = (byte) (ts >>> 56); + timestampBuffer[1] = (byte) (ts >>> 48); + timestampBuffer[2] = (byte) (ts >>> 40); + timestampBuffer[3] = (byte) (ts >>> 32); + timestampBuffer[4] = (byte) (ts >>> 24); + timestampBuffer[5] = (byte) (ts >>> 16); + timestampBuffer[6] = (byte) (ts >>> 8); + timestampBuffer[7] = (byte) (ts); + digest.update(timestampBuffer, 0, 8); digest.update(cell.getType().getCode()); digest.update(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()); @@ -256,16 +260,13 @@ private void updateDigestWithRow(byte[] rowKey, List cells) { * corrupted serialization * @param digest The digest whose state should be encoded * @return Byte array containing 4-byte length prefix + encoded state - * @throws IOException if encoding fails */ - private byte[] encodeDigestState(SHA256Digest digest) throws IOException { + private byte[] encodeDigestState(SHA256Digest digest) { byte[] encoded = digest.getEncodedState(); - ByteArrayOutputStream bos = new ByteArrayOutputStream(); - DataOutputStream dos = new DataOutputStream(bos); - dos.writeInt(encoded.length); - dos.write(encoded); - dos.flush(); - return bos.toByteArray(); + ByteBuffer buffer = ByteBuffer.allocate(4 + encoded.length); + buffer.putInt(encoded.length); + buffer.put(encoded); + return buffer.array(); } /** @@ -276,11 +277,10 @@ private byte[] encodeDigestState(SHA256Digest digest) throws IOException { */ private SHA256Digest decodeDigestState(byte[] encodedState) throws IOException { if (encodedState == null) { - String regionName = region.getRegionInfo().getRegionNameAsString(); - String tableName = region.getRegionInfo().getTable().getNameAsString(); throw new IllegalArgumentException( String.format("Invalid encoded digest state in region %s table %s: encodedState is null", - regionName, tableName)); + region.getRegionInfo().getRegionNameAsString(), + region.getRegionInfo().getTable().getNameAsString())); } DataInputStream dis = new DataInputStream(new ByteArrayInputStream(encodedState)); @@ -288,11 +288,11 @@ private SHA256Digest decodeDigestState(byte[] encodedState) throws IOException { // Prevent malicious large allocations, hash digest can never go beyond ~96 bytes, giving some // buffer upto 128 Bytes if (stateLength > MAX_SHA256_DIGEST_STATE_SIZE) { - String regionName = region.getRegionInfo().getRegionNameAsString(); - String tableName = region.getRegionInfo().getTable().getNameAsString(); throw new IllegalArgumentException( String.format("Invalid SHA256 state length in region %s table %s: %d expected <= %d", - regionName, tableName, stateLength, MAX_SHA256_DIGEST_STATE_SIZE)); + region.getRegionInfo().getRegionNameAsString(), + region.getRegionInfo().getTable().getNameAsString(), stateLength, + MAX_SHA256_DIGEST_STATE_SIZE)); } byte[] state = new byte[stateLength]; dis.readFully(state); @@ -319,8 +319,7 @@ private void buildChunkMetadataResult(List results, boolean isPartialChunk Bytes.toBytes(currentChunkRowCount))); if (isPartialChunk) { // Partial chunk digest - SHA256Digest cloned = new SHA256Digest(digest); - byte[] digestState = encodeDigestState(cloned); + byte[] digestState = encodeDigestState(digest); results.add(PhoenixKeyValueUtil.newKeyValue(resultRowKey, CHUNK_METADATA_FAMILY, BaseScannerRegionObserverConstants.SYNC_TABLE_IS_PARTIAL_CHUNK_QUALIFIER, AGG_TIMESTAMP, TRUE_BYTES)); @@ -346,10 +345,4 @@ public void close() throws IOException { LOGGER.error("Error closing PhoenixSyncTableRegionScanner", e); } } - - // Getters for testing - @VisibleForTesting - public long getChunkSizeBytes() { - return chunkSizeBytes; - } } diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixNoOpSingleRecordReader.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixNoOpSingleRecordReader.java index 8dc8860f426..28ec1ce4404 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixNoOpSingleRecordReader.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixNoOpSingleRecordReader.java @@ -64,7 +64,8 @@ public void initialize(InputSplit split, TaskAttemptContext context) { /** * Returns true exactly once to trigger a single map() call per split. - * @return true on first call, false on all subsequent calls + * @return true on first call, false on subsequent calls which makes Mapper task to exit calling + * map method */ @Override public boolean nextKeyValue() { diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java index 2e6452bee9f..76b01a9d14d 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java @@ -45,9 +45,6 @@ public class PhoenixSyncTableInputFormat extends PhoenixInputFormat { private static final Logger LOGGER = LoggerFactory.getLogger(PhoenixSyncTableInputFormat.class); - /** - * Instantiated by MapReduce framework - */ public PhoenixSyncTableInputFormat() { super(); } @@ -88,7 +85,7 @@ public List getSplits(JobContext context) throws IOException, Interr } LOGGER.info("Total splits generated {} of table {} for PhoenixSyncTable ", allSplits.size(), tableName); - List completedRegions = null; + List completedRegions; try { completedRegions = queryCompletedMapperRegions(conf, tableName, targetZkQuorum, fromTime, toTime); @@ -153,7 +150,11 @@ private List filterCompletedSplits(List allSplits, byte[] completedEnd = completedRange.getUpperRange(); // No overlap b/w completedRange/splitRange. - // completedEnd is before splitStart, increment completed pointer to catch up + // completedEnd is before splitStart, increment completed pointer to catch up. For scenario + // like below + // [----splitRange-----) + // [----completed----) + // If completedEnd is [], it means this is for last region, this check has no meaning. if ( !Bytes.equals(completedEnd, HConstants.EMPTY_END_ROW) && Bytes.compareTo(completedEnd, splitStart) <= 0 @@ -163,21 +164,35 @@ private List filterCompletedSplits(List allSplits, !Bytes.equals(splitEnd, HConstants.EMPTY_END_ROW) && Bytes.compareTo(completedStart, splitEnd) >= 0 ) { - // No overlap. completedStart is after splitEnd, splitRange needs to be processed, - // add to unprocessed list and increment + // No overlap b/w completedRange/splitRange. + // splitEnd is before completedStart, add this splitRange to unprocessed. For scenario like + // below + // [----splitRange-----) + // [----completed----) + // If splitEnd is [], it means this is for last region, this check has no meaning. unprocessedSplits.add(allSplits.get(splitIdx)); splitIdx++; } else { // Some overlap detected, check if SplitRange is fullyContained within completedRange - // Fully contained if: completedStart <= splitStart AND splitEnd <= completedEnd - + // [----splitRange-----) + // [----completed----) // partialContained -- unprocessedSplits + // OR + // [----splitRange-----) + // [----completed----) // partialContained -- unprocessedSplits + // OR + // [----splitRange-----------) + // [----completed--) // partialContained -- unprocessedSplits + // OR + // [----splitRange-----) + // [----completed----------) // fullyContained -- nothing to process boolean startContained = Bytes.compareTo(completedStart, splitStart) <= 0; + // If we are at end of completedRange region, we can assume end boundary is always contained + // wrt splitRange boolean endContained = Bytes.equals(completedEnd, HConstants.EMPTY_END_ROW) || Bytes.compareTo(splitEnd, completedEnd) <= 0; boolean fullyContained = startContained && endContained; if (!fullyContained) { - // Not fully contained, keep the split unprocessedSplits.add(allSplits.get(splitIdx)); } splitIdx++; diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java index e0ede2e9166..3ea306df730 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java @@ -24,7 +24,9 @@ import java.sql.Connection; import java.sql.SQLException; import java.sql.Timestamp; -import java.util.*; +import java.util.List; +import java.util.Arrays; +import java.util.ArrayList; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; @@ -147,7 +149,7 @@ private Connection createGlobalConnection(Configuration conf) throws SQLExceptio /** * Processes a mapper region by comparing chunks between source and target clusters. Gets already - * processed chunks from checkpoint table, resumes from checkpointed progress and records final + * processed chunks from checkpoint table, resumes from check pointed progress and records final * status for chunks & mapper (VERIFIED/MISMATCHED). */ @Override @@ -160,18 +162,9 @@ protected void map(NullWritable key, DBInputFormat.NullDBWritable value, Context mapperRegionStart, mapperRegionEnd); List> unprocessedRanges = calculateUnprocessedRanges(mapperRegionStart, mapperRegionEnd, processedChunks); - // Checking if start key should be inclusive, this is specific to scenario when there are - // processed - // chunks within this Mapper region boundary. + boolean isStartKeyInclusive = shouldStartKeyBeInclusive(mapperRegionStart, processedChunks); for (Pair range : unprocessedRanges) { - // From a Mapper region boundary, if we get multiple fragments of ranges it means there were - // some processed chunks in this region boundary - // And since chunks are inclusive of start and endKey, we just needed to confirm whether - // first fragment of range should have start key inclusive. - // Other fragment will not have start key inclusive since these ranges are carved out of - // chunk boundary, - // both start and key would already have been processed as part of chunk itself processMapperRanges(range.getFirst(), range.getSecond(), isStartKeyInclusive, context); isStartKeyInclusive = false; } @@ -181,12 +174,8 @@ protected void map(NullWritable key, DBInputFormat.NullDBWritable value, Context long sourceRowsProcessed = context.getCounter(SyncCounters.SOURCE_ROWS_PROCESSED).getValue(); long targetRowsProcessed = context.getCounter(SyncCounters.TARGET_ROWS_PROCESSED).getValue(); Timestamp mapperEndTime = new Timestamp(System.currentTimeMillis()); - Map mapperCounters = new LinkedHashMap<>(); - mapperCounters.put(SyncCounters.CHUNKS_VERIFIED.name(), verifiedChunk); - mapperCounters.put(SyncCounters.CHUNKS_MISMATCHED.name(), mismatchedChunk); - mapperCounters.put(SyncCounters.SOURCE_ROWS_PROCESSED.name(), sourceRowsProcessed); - mapperCounters.put(SyncCounters.TARGET_ROWS_PROCESSED.name(), targetRowsProcessed); - String counters = formatCounters(mapperCounters); + String counters = formatMapperCounters(verifiedChunk, mismatchedChunk, sourceRowsProcessed, + targetRowsProcessed); if (sourceRowsProcessed > 0) { if (mismatchedChunk == 0) { @@ -196,12 +185,12 @@ protected void map(NullWritable key, DBInputFormat.NullDBWritable value, Context mapperRegionStart, mapperRegionEnd, PhoenixSyncTableOutputRow.Status.VERIFIED, mapperStartTime, mapperEndTime, counters); LOGGER.info( - "PhoenixSyncTable mapper completed with verified: {} verifiedChunk chunks, {} mismatchedChunk chunks", + "PhoenixSyncTable mapper completed with verified: {} verified chunks, {} mismatched chunks", verifiedChunk, mismatchedChunk); } else { context.getCounter(PhoenixJobCounters.FAILED_RECORDS).increment(1); LOGGER.warn( - "PhoenixSyncTable mapper completed with mismatch: {} verifiedChunk chunks, {} mismatchedChunk chunks", + "PhoenixSyncTable mapper completed with mismatch: {} verified chunks, {} mismatched chunks", verifiedChunk, mismatchedChunk); syncTableOutputRepository.checkpointSyncTableResult(tableName, targetZkQuorum, PhoenixSyncTableOutputRow.Type.MAPPER_REGION, fromTime, toTime, isDryRun, @@ -255,10 +244,7 @@ private void processMapperRanges(byte[] rangeStart, byte[] rangeEnd, boolean isS sourceChunk.rowCount, targetChunk.rowCount, matched); } sourceChunk.executionEndTime = new Timestamp(System.currentTimeMillis()); - Map mapperCounters = new LinkedHashMap<>(); - mapperCounters.put(SyncCounters.SOURCE_ROWS_PROCESSED.name(), sourceChunk.rowCount); - mapperCounters.put(SyncCounters.TARGET_ROWS_PROCESSED.name(), targetChunk.rowCount); - String counters = formatCounters(mapperCounters); + String counters = formatChunkCounters(sourceChunk.rowCount, targetChunk.rowCount); if (matched) { handleVerifiedChunk(sourceChunk, context, counters); } else { @@ -287,7 +273,7 @@ private ChunkInfo getTargetChunkWithSourceBoundary(Connection conn, byte[] start ChunkInfo combinedTargetChunk = new ChunkInfo(); combinedTargetChunk.startKey = startKey; combinedTargetChunk.endKey = endKey; - combinedTargetChunk.hash = new byte[0]; + combinedTargetChunk.hash = null; combinedTargetChunk.rowCount = 0; combinedTargetChunk.isPartial = false; byte[] currentStartKey = startKey; @@ -298,7 +284,8 @@ private ChunkInfo getTargetChunkWithSourceBoundary(Connection conn, byte[] start // This chunk could be partial or full depending on whether the source region boundary is part // of one or multiple target region. // For every target region scanned, we want to have one row processed and returned back - // immediately(that's why we set scan.setLimit(1)), since output from one region partial chunk + // immediately(that's why we set scan.setLimit(1)/scan.setCaching(1)), since output from one + // region partial chunk // scanner is input to next region scanner. try (ChunkScannerContext scanner = createChunkScanner(conn, currentStartKey, endKey, continuedDigestState, isStartKeyInclusive, true, true)) { @@ -398,25 +385,32 @@ private ChunkInfo parseChunkInfo(Result result) { } /** - * Formats counters as a comma-separated key=value string. Example: - * "CHUNKS_VERIFIED=10,CHUNKS_MISMATCHED=2,SOURCE_ROWS_PROCESSED=5678..." - * @param counters Map of counter names to values - * @return Formatted string or null if counters is null/empty + * Formats chunk counters as a comma-separated string (optimized for hot path). Avoids + * LinkedHashMap allocation by building string directly. + * @param sourceRows Source rows processed + * @param targetRows Target rows processed + * @return Formatted string: "SOURCE_ROWS_PROCESSED=123,TARGET_ROWS_PROCESSED=456" */ - private String formatCounters(Map counters) { - if (counters == null || counters.isEmpty()) { - return null; - } - StringBuilder sb = new StringBuilder(); - boolean first = true; - for (Map.Entry entry : counters.entrySet()) { - if (!first) { - sb.append(","); - } - sb.append(entry.getKey()).append("=").append(entry.getValue()); - first = false; - } - return sb.toString(); + private String formatChunkCounters(long sourceRows, long targetRows) { + return String.format("%s=%d,%s=%d", SyncCounters.SOURCE_ROWS_PROCESSED.name(), sourceRows, + SyncCounters.TARGET_ROWS_PROCESSED.name(), targetRows); + } + + /** + * Formats mapper counters as a comma-separated string. Avoids LinkedHashMap allocation by + * building string directly. + * @param chunksVerified Chunks verified count + * @param chunksMismatched Chunks mismatched count + * @param sourceRows Source rows processed + * @param targetRows Target rows processed + * @return Formatted string with all mapper counters + */ + private String formatMapperCounters(long chunksVerified, long chunksMismatched, long sourceRows, + long targetRows) { + return String.format("%s=%d,%s=%d,%s=%d,%s=%d", SyncCounters.CHUNKS_VERIFIED.name(), + chunksVerified, SyncCounters.CHUNKS_MISMATCHED.name(), chunksMismatched, + SyncCounters.SOURCE_ROWS_PROCESSED.name(), sourceRows, + SyncCounters.TARGET_ROWS_PROCESSED.name(), targetRows); } /*** @@ -561,6 +555,15 @@ public List> calculateUnprocessedRanges(byte[] mapperRegion return gaps; } + /*** + * Checking if start key should be inclusive, this is specific to scenario when there are + * processed chunks within this Mapper region boundary. [---MapperRegion---------------) + * [--chunk1--] [--chunk2--] // With processed chunk, for this specific scenario, only we need to + * have first unprocessedRanges startKeyInclusive = true, for unprocessedRanges, their startkey + * would be false, since it would have been already covered by processed chunk + * [---MapperRegion---------------) [--chunk1--] [--chunk2--] // In such scenario, we don't want + * startKeyInclusive for any unprocessedRanges + */ boolean shouldStartKeyBeInclusive(byte[] mapperRegionStart, List processedChunks) { if ( diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java index 722c33c9f10..429f3c7e0ef 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java @@ -45,6 +45,11 @@ public class PhoenixSyncTableOutputRepository { private static final int OUTPUT_TABLE_TTL_SECONDS = 30 * 24 * 60 * 60; private final Connection connection; private static final byte[] EMPTY_START_KEY_SENTINEL = new byte[] { 0x00 }; + private static final String UPSERT_CHECKPOINT_SQL = + "UPSERT INTO " + SYNC_TABLE_CHECKPOINT_TABLE_NAME + + " (TABLE_NAME, TARGET_CLUSTER, TYPE, FROM_TIME, TO_TIME, IS_DRY_RUN," + + " START_ROW_KEY, END_ROW_KEY, IS_FIRST_REGION, EXECUTION_START_TIME, EXECUTION_END_TIME," + + " STATUS, COUNTERS) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)"; /** * @param connection Phoenix connection @@ -92,16 +97,11 @@ public void checkpointSyncTableResult(String tableName, String targetCluster, Ty throw new IllegalArgumentException("FromTime and ToTime cannot be null for checkpoint"); } - String upsert = "UPSERT INTO " + SYNC_TABLE_CHECKPOINT_TABLE_NAME - + " (TABLE_NAME, TARGET_CLUSTER, TYPE, FROM_TIME, TO_TIME, IS_DRY_RUN," - + " START_ROW_KEY, END_ROW_KEY, IS_FIRST_REGION, EXECUTION_START_TIME, EXECUTION_END_TIME," - + " STATUS, COUNTERS) " + "VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)"; - byte[] effectiveStartKey = (startKey == null || startKey.length == 0) ? EMPTY_START_KEY_SENTINEL : startKey; boolean isFirstRegion = (startKey == null || startKey.length == 0); - try (PreparedStatement ps = connection.prepareStatement(upsert)) { + try (PreparedStatement ps = connection.prepareStatement(UPSERT_CHECKPOINT_SQL)) { ps.setString(1, tableName); ps.setString(2, targetCluster); ps.setString(3, type.name()); diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java index 85cb68a620f..f1da979bef0 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java @@ -48,7 +48,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.phoenix.thirdparty.org.apache.commons.cli.CommandLine; import org.apache.phoenix.thirdparty.org.apache.commons.cli.CommandLineParser; import org.apache.phoenix.thirdparty.org.apache.commons.cli.DefaultParser; @@ -69,12 +68,12 @@ * regions based on HBase region boundaries.

  4. *
  5. Server-Side Chunking: Each mapper triggers a coprocessor scan on both source and * target clusters. The {@link PhoenixSyncTableRegionScanner} coprocessor accumulates rows into - * chunks (configurable size, default 1GB) and computes a SHA-256 hash of all row data (keys + + * chunks (configurable size, default 1GB) and computes an SHA-256 hash of all row data (keys + * column families + qualifiers + timestamps + values).
  6. *
  7. Hash Comparison: The {@link PhoenixSyncTableMapper} receives chunk metadata (start * key, end key, row count, hash) from both clusters and compares the hashes. Matching hashes mean * the chunk data is identical; mismatched hashes indicate inconsistency.
  8. - *
  9. Result Tracking: Results are checkpointed to the {@code PHOENIX_SYNC_TABLE_OUTPUT} + *
  10. Result Tracking: Results are check pointed to the {@code PHOENIX_SYNC_TABLE_OUTPUT} * table, tracking verified chunks, mismatched chunks, and processing progress for resumable * operations.
  11. *
@@ -126,7 +125,7 @@ public class PhoenixSyncTableTool extends Configured implements Tool { private PTable pTable; /** - * Creates a MR job that uses server-side chunking and checksum calculation + * Creates an MR job that uses server-side chunking and checksum calculation * @return Configured MapReduce job ready for submission * @throws Exception if job creation fails */ @@ -173,8 +172,6 @@ private void configureTimeoutsAndRetries(Configuration configuration) { configuration.getInt(QueryServices.SYNC_TABLE_RPC_RETRIES_COUNTER, QueryServicesOptions.DEFAULT_SYNC_TABLE_RPC_RETRIES_COUNTER); - configuration.set(QueryServices.THREAD_TIMEOUT_MS_ATTRIB, - Long.toString(syncTableQueryTimeoutMs)); configuration.set(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, Long.toString(syncTableClientScannerTimeoutMs)); configuration.set(HConstants.HBASE_RPC_TIMEOUT_KEY, Long.toString(syncTableRPCTimeoutMs)); @@ -201,7 +198,7 @@ private void setPhoenixSyncTableToolConfiguration(Configuration configuration) { .setBooleanIfUnset(PhoenixConfigurationUtil.MAPREDUCE_RANDOMIZE_MAPPER_EXECUTION_ORDER, true); } - private void configureInput(Job job, PTableType tableType) throws Exception { + private void configureInput(Job job, PTableType tableType) { // With below query plan, we get Input split based on region boundary String hint = (tableType == PTableType.INDEX) ? "" : "/*+ NO_INDEX */ "; String selectStatement = "SELECT " + hint + "1 FROM " + qTable; @@ -283,26 +280,26 @@ private void printHelpAndExit(Options options, int exitCode) { System.exit(exitCode); } - public void populateSyncTableToolAttributes(CommandLine cmdLine) throws Exception { + public void populateSyncTableToolAttributes(CommandLine cmdLine) { tableName = cmdLine.getOptionValue(TABLE_NAME_OPTION.getOpt()); targetZkQuorum = cmdLine.getOptionValue(TARGET_CLUSTER_OPTION.getOpt()); schemaName = cmdLine.getOptionValue(SCHEMA_NAME_OPTION.getOpt()); if (cmdLine.hasOption(FROM_TIME_OPTION.getOpt())) { - startTime = Long.valueOf(cmdLine.getOptionValue(FROM_TIME_OPTION.getOpt())); + startTime = Long.parseLong(cmdLine.getOptionValue(FROM_TIME_OPTION)); } else { startTime = 0L; } if (cmdLine.hasOption(TO_TIME_OPTION.getOpt())) { - endTime = Long.valueOf(cmdLine.getOptionValue(TO_TIME_OPTION.getOpt())); + endTime = Long.parseLong(cmdLine.getOptionValue(TO_TIME_OPTION)); } else { // Default endTime, current time - 1 hour endTime = EnvironmentEdgeManager.currentTimeMillis(); // - (60 * 60 * 1000); } if (cmdLine.hasOption(CHUNK_SIZE_OPTION.getOpt())) { - chunkSizeBytes = Long.valueOf(cmdLine.getOptionValue(CHUNK_SIZE_OPTION.getOpt())); + chunkSizeBytes = Long.parseLong(cmdLine.getOptionValue(CHUNK_SIZE_OPTION)); } if (cmdLine.hasOption(TENANT_ID_OPTION.getOpt())) { tenantId = cmdLine.getOptionValue(TENANT_ID_OPTION.getOpt()); @@ -362,17 +359,28 @@ private boolean submitPhoenixSyncTableJob() throws Exception { job.getJobName(), qTable, targetZkQuorum); } Counters counters = job.getCounters(); - LOGGER.info( - "PhoenixSyncTable job completed, gathered counters are \n Input Record: {}, \n" - + "Ouput Record: {}, \n Failed Record: {}, \n Chunks Verified: {}, \n" - + "Chunks Mimatched: {}, \n Source Rows Processed: {}, \n Target Rows Processed: {}", - counters.findCounter(PhoenixJobCounters.INPUT_RECORDS).getValue(), - counters.findCounter(PhoenixJobCounters.OUTPUT_RECORDS).getValue(), - counters.findCounter(PhoenixJobCounters.FAILED_RECORDS).getValue(), - counters.findCounter(PhoenixSyncTableMapper.SyncCounters.CHUNKS_VERIFIED).getValue(), - counters.findCounter(PhoenixSyncTableMapper.SyncCounters.CHUNKS_MISMATCHED).getValue(), - counters.findCounter(PhoenixSyncTableMapper.SyncCounters.SOURCE_ROWS_PROCESSED).getValue(), - counters.findCounter(PhoenixSyncTableMapper.SyncCounters.TARGET_ROWS_PROCESSED).getValue()); + if (counters != null) { + long inputRecords = counters.findCounter(PhoenixJobCounters.INPUT_RECORDS).getValue(); + long outputRecords = counters.findCounter(PhoenixJobCounters.OUTPUT_RECORDS).getValue(); + long failedRecords = counters.findCounter(PhoenixJobCounters.FAILED_RECORDS).getValue(); + long chunksVerified = + counters.findCounter(PhoenixSyncTableMapper.SyncCounters.CHUNKS_VERIFIED).getValue(); + long chunksMismatched = + counters.findCounter(PhoenixSyncTableMapper.SyncCounters.CHUNKS_MISMATCHED).getValue(); + long sourceRowsProcessed = + counters.findCounter(PhoenixSyncTableMapper.SyncCounters.SOURCE_ROWS_PROCESSED).getValue(); + long targetRowsProcessed = + counters.findCounter(PhoenixSyncTableMapper.SyncCounters.TARGET_ROWS_PROCESSED).getValue(); + LOGGER.info( + "PhoenixSyncTable job completed, gathered counters are \n Input Record: {}, \n" + + "Output Record: {}, \n Failed Record: {}, \n Chunks Verified: {}, \n" + + "Chunks Mismatched: {}, \n Source Rows Processed: {}, \n Target Rows Processed: {}", + inputRecords, outputRecords, failedRecords, chunksVerified, chunksMismatched, + sourceRowsProcessed, targetRowsProcessed); + } else { + LOGGER.warn("Unable to retrieve job counters for table {} - job may have failed " + + "during initialization", qTable); + } return success; } @@ -407,35 +415,4 @@ public static void main(String[] args) throws Exception { int exitCode = ToolRunner.run(new PhoenixSyncTableTool(), args); System.exit(exitCode); } - - // Getters for testing - @VisibleForTesting - public String getQTable() { - return qTable; - } - - @VisibleForTesting - public String getTargetZkQuorum() { - return targetZkQuorum; - } - - @VisibleForTesting - public boolean isDryRun() { - return isDryRun; - } - - @VisibleForTesting - public Job getJob() { - return job; - } - - @VisibleForTesting - public long getStartTime() { - return startTime; - } - - @VisibleForTesting - public long getEndTime() { - return endTime; - } } From 8a1e5c61ef305cc48ec0e26c64d90f69d9b33e94 Mon Sep 17 00:00:00 2001 From: Rahul Kumar Date: Tue, 17 Feb 2026 22:41:01 +0530 Subject: [PATCH 8/9] fix checkstlye issues --- .../PhoenixSyncTableRegionScanner.java | 10 ++++++---- .../mapreduce/PhoenixSyncTableMapper.java | 8 ++++---- .../PhoenixSyncTableOutputRepository.java | 7 +++++-- .../mapreduce/PhoenixSyncTableOutputRow.java | 4 ++-- .../phoenix/mapreduce/PhoenixSyncTableTool.java | 11 ++++++++--- .../phoenix/mapreduce/index/IndexTool.java | 3 --- .../mapreduce/util/PhoenixMapReduceUtil.java | 16 +++++++++------- .../org/apache/phoenix/index/IndexToolTest.java | 2 +- 8 files changed, 35 insertions(+), 26 deletions(-) diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java index 8ac9755d4b1..13108a58c81 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java @@ -17,7 +17,7 @@ */ package org.apache.phoenix.coprocessor; -import static org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil.*; +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; @@ -86,11 +86,13 @@ public class PhoenixSyncTableRegionScanner extends BaseRegionScanner { private final byte[] timestampBuffer = new byte[8]; /** + * 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, @@ -118,7 +120,7 @@ public PhoenixSyncTableRegionScanner(final RegionScanner innerScanner, final Reg this.digest = decodeDigestState(continuedDigestStateAttr); this.isUsingContinuedDigest = true; } catch (IOException e) { - throw new RuntimeException("Failed to restore continued digest state", e); + throw new IllegalStateException("Failed to restore continued digest state", e); } } else { this.digest = new SHA256Digest(); @@ -245,7 +247,7 @@ private void updateDigestWithRow(byte[] rowKey, List cells) { timestampBuffer[4] = (byte) (ts >>> 24); timestampBuffer[5] = (byte) (ts >>> 16); timestampBuffer[6] = (byte) (ts >>> 8); - timestampBuffer[7] = (byte) (ts); + timestampBuffer[7] = (byte) ts; digest.update(timestampBuffer, 0, 8); digest.update(cell.getType().getCode()); @@ -286,7 +288,7 @@ private SHA256Digest decodeDigestState(byte[] encodedState) throws IOException { DataInputStream dis = new DataInputStream(new ByteArrayInputStream(encodedState)); int stateLength = dis.readInt(); // Prevent malicious large allocations, hash digest can never go beyond ~96 bytes, giving some - // buffer upto 128 Bytes + // buffer up to 128 Bytes if (stateLength > MAX_SHA256_DIGEST_STATE_SIZE) { throw new IllegalArgumentException( String.format("Invalid SHA256 state length in region %s table %s: %d expected <= %d", diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java index 3ea306df730..2920e81aae1 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java @@ -24,9 +24,9 @@ import java.sql.Connection; import java.sql.SQLException; import java.sql.Timestamp; -import java.util.List; -import java.util.Arrays; import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; @@ -500,8 +500,8 @@ public List> calculateUnprocessedRanges(byte[] mapperRegion PhoenixSyncTableOutputRow chunk = processedChunks.get(i); byte[] chunkStart = chunk.getStartRowKey(); byte[] chunkEnd = chunk.getEndRowKey(); - boolean initialChunk = (i == 0); - boolean lastChunk = (i == processedChunks.size() - 1); + boolean initialChunk = i == 0; + boolean lastChunk = i == processedChunks.size() - 1; // Determine effective start boundary for this chunk // Only the first chunk might start before mapperRegionStart and need clipping diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java index 429f3c7e0ef..e66a94067bf 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java @@ -52,7 +52,10 @@ public class PhoenixSyncTableOutputRepository { + " STATUS, COUNTERS) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)"; /** - * @param connection Phoenix connection + * Creates a repository for managing sync table checkpoint operations. Note: The connection is + * stored as-is and shared across operations. The caller retains ownership and is responsible for + * connection lifecycle. + * @param connection Phoenix connection (must remain open for repository lifetime) */ public PhoenixSyncTableOutputRepository(Connection connection) { this.connection = connection; @@ -99,7 +102,7 @@ public void checkpointSyncTableResult(String tableName, String targetCluster, Ty byte[] effectiveStartKey = (startKey == null || startKey.length == 0) ? EMPTY_START_KEY_SENTINEL : startKey; - boolean isFirstRegion = (startKey == null || startKey.length == 0); + boolean isFirstRegion = startKey == null || startKey.length == 0; try (PreparedStatement ps = connection.prepareStatement(UPSERT_CHECKPOINT_SQL)) { ps.setString(1, tableName); diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java index b9a284aae97..3e45435ae4f 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java @@ -46,11 +46,11 @@ public String toString() { } public byte[] getStartRowKey() { - return startRowKey; + return startRowKey != null ? Arrays.copyOf(startRowKey, startRowKey.length) : null; } public byte[] getEndRowKey() { - return endRowKey; + return endRowKey != null ? Arrays.copyOf(endRowKey, endRowKey.length) : null; } /** diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java index f1da979bef0..12b494ccc44 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java @@ -82,6 +82,7 @@ *
  * hbase org.apache.phoenix.mapreduce.PhoenixSyncTableTool \ --table-name MY_TABLE \
  * --target-cluster target-zk1,target-zk2:2181:/hbase
+ * 
*/ public class PhoenixSyncTableTool extends Configured implements Tool { @@ -286,20 +287,20 @@ public void populateSyncTableToolAttributes(CommandLine cmdLine) { schemaName = cmdLine.getOptionValue(SCHEMA_NAME_OPTION.getOpt()); if (cmdLine.hasOption(FROM_TIME_OPTION.getOpt())) { - startTime = Long.parseLong(cmdLine.getOptionValue(FROM_TIME_OPTION)); + startTime = Long.valueOf(cmdLine.getOptionValue(FROM_TIME_OPTION.getOpt())); } else { startTime = 0L; } if (cmdLine.hasOption(TO_TIME_OPTION.getOpt())) { - endTime = Long.parseLong(cmdLine.getOptionValue(TO_TIME_OPTION)); + endTime = Long.valueOf(cmdLine.getOptionValue(TO_TIME_OPTION.getOpt())); } else { // Default endTime, current time - 1 hour endTime = EnvironmentEdgeManager.currentTimeMillis(); // - (60 * 60 * 1000); } if (cmdLine.hasOption(CHUNK_SIZE_OPTION.getOpt())) { - chunkSizeBytes = Long.parseLong(cmdLine.getOptionValue(CHUNK_SIZE_OPTION)); + chunkSizeBytes = Long.valueOf(cmdLine.getOptionValue(CHUNK_SIZE_OPTION.getOpt())); } if (cmdLine.hasOption(TENANT_ID_OPTION.getOpt())) { tenantId = cmdLine.getOptionValue(TENANT_ID_OPTION.getOpt()); @@ -415,4 +416,8 @@ public static void main(String[] args) throws Exception { int exitCode = ToolRunner.run(new PhoenixSyncTableTool(), args); System.exit(exitCode); } + + public Job getJob() { + return job; + } } diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java index fca307d4791..08651ecbc3c 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java @@ -296,9 +296,6 @@ public static IndexDisableLoggingType fromValue(byte[] value) { public static final String INDEX_JOB_NAME_TEMPLATE = "PHOENIX_%s.%s_INDX_%s"; - public static final String INVALID_TIME_RANGE_EXCEPTION_MESSAGE = "startTime is greater than " - + "or equal to endTime " + "or either of them are set in the future; IndexTool can't proceed."; - public static final String FEATURE_NOT_APPLICABLE = "start-time/end-time and retry verify feature are only " + "applicable for local or non-transactional global indexes"; diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixMapReduceUtil.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixMapReduceUtil.java index c7c74601f60..f3ccddeeac0 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixMapReduceUtil.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixMapReduceUtil.java @@ -42,6 +42,8 @@ */ public final class PhoenixMapReduceUtil { + public static final String INVALID_TIME_RANGE_EXCEPTION_MESSAGE = "Invalid time range for table"; + private PhoenixMapReduceUtil() { } @@ -255,15 +257,15 @@ public static void setTenantId(final Job job, final String tenantId) { * @throws IllegalArgumentException if time range is invalid */ public static void validateTimeRange(Long startTime, Long endTime, String tableName) { - Long currentTime = EnvironmentEdgeManager.currentTimeMillis(); - Long st = (startTime == null) ? 0L : startTime; - Long et = (endTime == null) ? currentTime : endTime; + long currentTime = EnvironmentEdgeManager.currentTimeMillis(); + long st = (startTime == null) ? 0L : startTime; + long et = (endTime == null) ? currentTime : endTime; - if (st.compareTo(currentTime) > 0 || et.compareTo(currentTime) > 0 || st.compareTo(et) >= 0) { + if (et > currentTime || st >= et) { throw new IllegalArgumentException(String.format( - "Invalid time range for table %s: start and end times must be in the past and start < end. " - + "Start: %d, End: %d, Current: %d", - tableName, st, et, currentTime)); + "%s %s: start and end times must be in the past " + + "and start < end. Start: %d, End: %d, Current: %d", + INVALID_TIME_RANGE_EXCEPTION_MESSAGE, tableName, st, et, currentTime)); } } diff --git a/phoenix-core/src/test/java/org/apache/phoenix/index/IndexToolTest.java b/phoenix-core/src/test/java/org/apache/phoenix/index/IndexToolTest.java index f6b408d1067..28f6c4ae1e2 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/index/IndexToolTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/index/IndexToolTest.java @@ -18,8 +18,8 @@ package org.apache.phoenix.index; import static org.apache.phoenix.mapreduce.index.IndexTool.FEATURE_NOT_APPLICABLE; -import static org.apache.phoenix.mapreduce.index.IndexTool.INVALID_TIME_RANGE_EXCEPTION_MESSAGE; import static org.apache.phoenix.mapreduce.index.IndexTool.RETRY_VERIFY_NOT_APPLICABLE; +import static org.apache.phoenix.mapreduce.util.PhoenixMapReduceUtil.INVALID_TIME_RANGE_EXCEPTION_MESSAGE; import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.when; From d5772b24b9624d6a657aadd3551993a2cfc58ecf Mon Sep 17 00:00:00 2001 From: Rahul Kumar Date: Wed, 18 Feb 2026 12:07:45 +0530 Subject: [PATCH 9/9] fix checstyle --- .../UngroupedAggregateRegionObserver.java | 16 ++++++++-------- .../mapreduce/PhoenixSyncTableMapper.java | 4 ++-- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java index 5955bce543d..88caec587d7 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java @@ -17,14 +17,6 @@ */ package org.apache.phoenix.coprocessor; -import static org.apache.phoenix.coprocessor.GlobalIndexRegionScanner.adjustScanFilter; -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 static org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker.COMPACTION_UPDATE_STATS_ROW_COUNT; -import static org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker.CONCURRENT_UPDATE_STATS_ROW_COUNT; - import java.io.ByteArrayInputStream; import java.io.DataInputStream; import java.io.IOException; @@ -123,6 +115,14 @@ import org.apache.phoenix.thirdparty.com.google.common.base.Throwables; import org.apache.phoenix.thirdparty.com.google.common.collect.Lists; +import static org.apache.phoenix.coprocessor.GlobalIndexRegionScanner.adjustScanFilter; +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 static org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker.COMPACTION_UPDATE_STATS_ROW_COUNT; +import static org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker.CONCURRENT_UPDATE_STATS_ROW_COUNT; + /** * Region observer that aggregates ungrouped rows(i.e. SQL query with aggregation function and no * GROUP BY). diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java index 2920e81aae1..81947d6858d 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java @@ -17,8 +17,6 @@ */ package org.apache.phoenix.mapreduce; -import static org.apache.phoenix.schema.types.PDataType.TRUE_BYTES; - import java.io.IOException; import java.security.MessageDigest; import java.sql.Connection; @@ -53,6 +51,8 @@ import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; +import static org.apache.phoenix.schema.types.PDataType.TRUE_BYTES; + /** * Mapper that acts as a driver for synchronizing table between source and target clusters. The * actual work of chunking and hashing is done server-side by the coprocessor. This mapper fetches