diff --git a/phoenix-core-client/pom.xml b/phoenix-core-client/pom.xml index 6c6cea359e6..a04148b0bf6 100644 --- a/phoenix-core-client/pom.xml +++ b/phoenix-core-client/pom.xml @@ -250,6 +250,11 @@ org.hdrhistogram HdrHistogram + + org.bouncycastle + bcprov-jdk18on + 1.79 + 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 f24f51ddc4d..1e0d7d7ad0f 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 @@ -199,4 +199,23 @@ public static long getMaxLookbackInMillis(Configuration conf) { /** Exposed for testing */ public static final String SCANNER_OPENED_TRACE_INFO = "Scanner opened on server"; + + /** + * PhoenixSyncTableTool scan attributes for server-side chunk formation and checksum + */ + 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_CONTINUED_DIGEST_STATE = "_SyncTableContinuedDigestState"; + + /** + * 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_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_CHUNK_QUALIFIER = + Bytes.toBytes("IS_PARTIAL_CHUNK"); } 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 630a2d4f210..c5b31a00618 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 @@ -218,6 +218,14 @@ public interface QueryServices extends SQLCloseable { public static final String ALLOW_LOCAL_INDEX_ATTRIB = "phoenix.index.allowLocalIndex"; + // 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.sync.table.client.scanner.timeout"; + public static final String SYNC_TABLE_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"; 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 ba344af5b14..4e3c29b6c3c 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 @@ -254,6 +254,12 @@ 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 + 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/SHA256DigestUtil.java b/phoenix-core-client/src/main/java/org/apache/phoenix/util/SHA256DigestUtil.java new file mode 100644 index 00000000000..3eaa59b0ab5 --- /dev/null +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/util/SHA256DigestUtil.java @@ -0,0 +1,100 @@ +/* + * 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.util; + +import java.io.ByteArrayInputStream; +import java.io.DataInputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.util.Bytes; +import org.bouncycastle.crypto.digests.SHA256Digest; + +/** + * Utility class for SHA-256 digest state serialization and deserialization. We are not using jdk + * bundled SHA, since their digest can't be serialized/deserialized which is needed for + * PhoenixSyncTableTool for cross-region hash continuation. + */ +public class SHA256DigestUtil { + + /** + * Maximum allowed size for encoded SHA-256 digest state. SHA-256 state is ~96 bytes, we allow up + * to 128 bytes as buffer. + */ + public static final int MAX_SHA256_DIGEST_STATE_SIZE = 128; + + /** + * Encodes a SHA256Digest state to a byte array with length prefix for validation. Format: [4-byte + * integer length][encoded digest state bytes] + * @param digest The digest whose state should be encoded + * @return Byte array containing integer length prefix + encoded state + */ + public static byte[] encodeDigestState(SHA256Digest digest) { + byte[] encoded = digest.getEncodedState(); + ByteBuffer buffer = ByteBuffer.allocate(Bytes.SIZEOF_INT + encoded.length); + buffer.putInt(encoded.length); + buffer.put(encoded); + return buffer.array(); + } + + /** + * Decodes a SHA256Digest state from a byte array. + * @param encodedState Byte array containing 4-byte integer length prefix + encoded state + * @return SHA256Digest restored to the saved state + * @throws IOException if state is invalid, corrupted + */ + public static SHA256Digest decodeDigestState(byte[] encodedState) throws IOException { + if (encodedState == null) { + throw new IllegalArgumentException("Invalid encoded digest state: encodedState is null"); + } + + DataInputStream dis = new DataInputStream(new ByteArrayInputStream(encodedState)); + int stateLength = dis.readInt(); + // Prevent malicious large allocations + if (stateLength > MAX_SHA256_DIGEST_STATE_SIZE) { + throw new IllegalArgumentException( + String.format("Invalid SHA256 state length: %d, expected <= %d", stateLength, + MAX_SHA256_DIGEST_STATE_SIZE)); + } + + byte[] state = new byte[stateLength]; + dis.readFully(state); + return new SHA256Digest(state); + } + + /** + * Decodes a digest state and finalizes it to produce the SHA-256 checksum. + * @param encodedState Serialized digest state (format: [4-byte length][state bytes]) + * @return 32-byte SHA-256 hash + * @throws IOException if state decoding fails + */ + public static byte[] finalizeDigestToChecksum(byte[] encodedState) throws IOException { + SHA256Digest digest = decodeDigestState(encodedState); + return finalizeDigestToChecksum(digest); + } + + /** + * Finalizes a SHA256Digest to produce the final checksum. + * @param digest The digest to finalize + * @return 32-byte SHA-256 hash + */ + public static byte[] finalizeDigestToChecksum(SHA256Digest digest) { + byte[] hash = new byte[digest.getDigestSize()]; + digest.doFinal(hash, 0); + return hash; + } +} 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 2e4816fe589..b798553ea6c 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 @@ -1207,6 +1207,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/pom.xml b/phoenix-core-server/pom.xml index e3abcb3558b..3b4c36f5183 100644 --- a/phoenix-core-server/pom.xml +++ b/phoenix-core-server/pom.xml @@ -173,8 +173,12 @@ 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 new file mode 100644 index 00000000000..d0d0424d1cd --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixSyncTableRegionScanner.java @@ -0,0 +1,383 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.phoenix.coprocessor; + +import static org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil.DEFAULT_PHOENIX_SYNC_TABLE_CHUNK_SIZE_BYTES; +import static org.apache.phoenix.query.QueryConstants.AGG_TIMESTAMP; +import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN_FAMILY; +import static org.apache.phoenix.schema.types.PDataType.FALSE_BYTES; +import static org.apache.phoenix.schema.types.PDataType.TRUE_BYTES; +import static org.apache.phoenix.util.ScanUtil.getDummyResult; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.PrivateCellUtil; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; +import org.apache.hadoop.hbase.regionserver.PhoenixScannerContext; +import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.RegionScanner; +import org.apache.hadoop.hbase.regionserver.ScannerContext; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.phoenix.coprocessorclient.BaseScannerRegionObserverConstants; +import org.apache.phoenix.util.ByteUtil; +import org.apache.phoenix.util.PhoenixKeyValueUtil; +import org.apache.phoenix.util.SHA256DigestUtil; +import org.apache.phoenix.util.ScanUtil; +import org.bouncycastle.crypto.digests.SHA256Digest; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; + +/** + * Server-side coprocessor that performs chunk formation and SHA-256 hashing for + * PhoenixSyncTableTool. + *

+ * Accumulates rows into chunks (based on size limits) and computes a hash of all row data (keys, + * column families, qualifiers, timestamps, cell types, values). In case of paging timeout, return + * whatever is accumulated in chunk. If nothing is accumulated return dummy row either with prev + * result rowKey or max possible key < currentRowKey + *

+ * Source scan (isTargetScan=false): Returns complete chunks(if paging dint timeout) 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. + *

+ * Returns chunk metadata cells: START_KEY, END_KEY, HASH (or digest state), ROW_COUNT, + * IS_PARTIAL_CHUNK + */ +public class PhoenixSyncTableRegionScanner extends BaseRegionScanner { + + private static final Logger LOGGER = LoggerFactory.getLogger(PhoenixSyncTableRegionScanner.class); + private static final byte[] CHUNK_METADATA_FAMILY = SINGLE_COLUMN_FAMILY; + private final Region region; + private final Scan scan; + private final RegionCoprocessorEnvironment env; + private final UngroupedAggregateRegionObserver ungroupedAggregateRegionObserver; + private final long chunkSizeBytes; + private boolean isTargetScan = false; + private byte[] chunkStartKey = null; + private byte[] chunkEndKey = null; + private long currentChunkSize = 0L; + private long currentChunkRowCount = 0L; + private final SHA256Digest digest; + private boolean hasMoreRows = true; + private boolean isUsingContinuedDigest; + private byte[] previousResultRowKey = null; + private final byte[] initStartRowKey; + private final boolean includeInitStartRowKey; + private final long pageSizeMs; + + /** + * Creates a PhoenixSyncTableRegionScanner for chunk-based hashing. + * @param innerScanner The underlying region scanner + * @param region The region being scanned + * @param scan The scan request + * @param env The coprocessor environment + * @param ungroupedAggregateRegionObserver Parent observer for region state checks + * @throws IllegalStateException if digest state restoration fails + */ + @VisibleForTesting + public PhoenixSyncTableRegionScanner(final RegionScanner innerScanner, final Region region, + final Scan scan, final RegionCoprocessorEnvironment env, + final UngroupedAggregateRegionObserver ungroupedAggregateRegionObserver, long pageSizeMs) { + super(innerScanner); + this.region = region; + this.scan = scan; + this.env = env; + this.ungroupedAggregateRegionObserver = ungroupedAggregateRegionObserver; + byte[] chunkSizeAttr = + scan.getAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CHUNK_SIZE_BYTES); + if (chunkSizeAttr == null) { // Since we don't set chunk size scan attr for target cluster scan + this.isTargetScan = true; + } + this.chunkSizeBytes = chunkSizeAttr != null + ? Bytes.toLong(chunkSizeAttr) + : DEFAULT_PHOENIX_SYNC_TABLE_CHUNK_SIZE_BYTES; + + // Check if we should continue from a previous digest state (cross-region continuation) + byte[] continuedDigestStateAttr = + scan.getAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CONTINUED_DIGEST_STATE); + if (continuedDigestStateAttr != null) { + try { + this.digest = SHA256DigestUtil.decodeDigestState(continuedDigestStateAttr); + this.isUsingContinuedDigest = true; + } catch (IOException e) { + throw new IllegalStateException("Failed to restore continued digest state", e); + } + } else { + this.digest = new SHA256Digest(); + this.isUsingContinuedDigest = false; + } + this.initStartRowKey = scan.getStartRow(); + this.includeInitStartRowKey = scan.includeStartRow(); + this.pageSizeMs = pageSizeMs; + } + + @Override + public boolean next(List results) throws IOException { + return next(results, null); + } + + /** + * Accumulates rows into a chunk and returns chunk metadata cells. Supports server-side paging via + * {@link PhoenixScannerContext} following the same pattern as + * {@link GroupedAggregateRegionObserver} and {@link UncoveredIndexRegionScanner}. + * @param results Output list to populate with chunk metadata cells + * @param scannerContext Phoenix scanner context for paging timeout detection + * @return true if more chunks available, false if scanning complete + */ + @Override + public boolean next(List results, ScannerContext scannerContext) throws IOException { + region.startRegionOperation(); + try { + resetChunkState(); + RegionScanner localScanner = delegate; + synchronized (localScanner) { + List rowCells = new ArrayList<>(); + while (hasMoreRows) { + ungroupedAggregateRegionObserver.checkForRegionClosingOrSplitting(); + rowCells.clear(); + hasMoreRows = (scannerContext == null) + ? localScanner.nextRaw(rowCells) + : localScanner.nextRaw(rowCells, scannerContext); + + if (!rowCells.isEmpty() && ScanUtil.isDummy(rowCells)) { + if (chunkStartKey == null) { + LOGGER.warn("Paging timed out while fetching first row of chunk, initStartRowKey: {}", + Bytes.toStringBinary(initStartRowKey)); + updateDummyWithPrevRowKey(results, initStartRowKey, includeInitStartRowKey, scan); + return true; + } else { + break; + } + } + + if (rowCells.isEmpty()) { + break; + } + + byte[] rowKey = CellUtil.cloneRow(rowCells.get(0)); + long rowSize = calculateRowSize(rowCells); + addRowToChunk(rowKey, rowCells, rowSize); + if (!isTargetScan && willExceedChunkLimits(rowSize)) { + break; + } + if ( + hasMoreRows && (PhoenixScannerContext.isReturnImmediately(scannerContext) + || PhoenixScannerContext.isTimedOut(scannerContext, pageSizeMs)) + ) { + LOGGER.info("Paging timeout after {} rows ({} bytes) in region {}, chunk [{}:{}]", + currentChunkRowCount, currentChunkSize, + region.getRegionInfo().getRegionNameAsString(), Bytes.toStringBinary(chunkStartKey), + Bytes.toStringBinary(chunkEndKey)); + PhoenixScannerContext.setReturnImmediately(scannerContext); + break; + } + } + } + if (chunkStartKey == null) { + return false; + } + + buildChunkMetadataResult(results, isTargetScan); + previousResultRowKey = chunkEndKey; + 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(); + } + } + + /** + * 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) { + return currentChunkSize + rowSize > chunkSizeBytes; + } + + /** + * 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. + */ + private void updateDigestWithRow(byte[] rowKey, List cells) { + digest.update(rowKey, 0, rowKey.length); + byte[] timestampBuffer = new byte[8]; + for (Cell cell : cells) { + digest.update(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()); + digest.update(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()); + long ts = cell.getTimestamp(); + Bytes.putLong(timestampBuffer, 0, ts); + digest.update(timestampBuffer, 0, 8); + digest.update(cell.getType().getCode()); + digest.update(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()); + } + } + + /** + * Builds chunk metadata result cells and adds them to the results list. Returns a single + * "row"[rowKey=chunkEndKey] with multiple cells containing chunk metadata[chunkStartKey, + * hash/digest, rowCount, isPartialChunk]. For complete chunks: includes final SHA-256 hash (32 + * bytes) For partial chunks: includes serialized MessageDigest state for continuation + * @param results Output list to populate with chunk metadata cells + * @param isPartialChunk true if this is a partial chunk (region boundary reached before + * completion) + */ + private void buildChunkMetadataResult(List results, boolean isPartialChunk) + throws IOException { + byte[] resultRowKey = this.chunkEndKey; + results.add(PhoenixKeyValueUtil.newKeyValue(resultRowKey, CHUNK_METADATA_FAMILY, + BaseScannerRegionObserverConstants.SYNC_TABLE_START_KEY_QUALIFIER, AGG_TIMESTAMP, + chunkStartKey)); + results.add(PhoenixKeyValueUtil.newKeyValue(resultRowKey, CHUNK_METADATA_FAMILY, + BaseScannerRegionObserverConstants.SYNC_TABLE_ROW_COUNT_QUALIFIER, AGG_TIMESTAMP, + Bytes.toBytes(currentChunkRowCount))); + if (isPartialChunk) { + // Partial chunk digest + byte[] digestState = SHA256DigestUtil.encodeDigestState(digest); + results.add(PhoenixKeyValueUtil.newKeyValue(resultRowKey, CHUNK_METADATA_FAMILY, + BaseScannerRegionObserverConstants.SYNC_TABLE_IS_PARTIAL_CHUNK_QUALIFIER, AGG_TIMESTAMP, + TRUE_BYTES)); + results.add(PhoenixKeyValueUtil.newKeyValue(resultRowKey, CHUNK_METADATA_FAMILY, + BaseScannerRegionObserverConstants.SYNC_TABLE_HASH_QUALIFIER, AGG_TIMESTAMP, digestState)); + } else { + // Complete chunk - finalize and return hash + byte[] hash = SHA256DigestUtil.finalizeDigestToChecksum(digest); + results.add(PhoenixKeyValueUtil.newKeyValue(resultRowKey, CHUNK_METADATA_FAMILY, + BaseScannerRegionObserverConstants.SYNC_TABLE_HASH_QUALIFIER, AGG_TIMESTAMP, hash)); + results.add(PhoenixKeyValueUtil.newKeyValue(resultRowKey, CHUNK_METADATA_FAMILY, + BaseScannerRegionObserverConstants.SYNC_TABLE_IS_PARTIAL_CHUNK_QUALIFIER, AGG_TIMESTAMP, + FALSE_BYTES)); + } + } + + /** + * Add dummy cell to the result list based on either the previous rowKey returned to the client or + * the start rowKey and start rowKey include params. + * @param result result to add the dummy cell to. + * @param initStartRowKey scan start rowKey. + * @param includeInitStartRowKey scan start rowKey included. + * @param scan scan object. + */ + private void updateDummyWithPrevRowKey(List result, byte[] initStartRowKey, + boolean includeInitStartRowKey, Scan scan) { + result.clear(); + if (previousResultRowKey != null) { + getDummyResult(previousResultRowKey, result); + } else { + if (includeInitStartRowKey && initStartRowKey.length > 0) { + byte[] prevKey; + // In order to generate largest possible rowkey that is less than + // initStartRowKey, we need to check size of the region name that can be + // used by hbase client for meta lookup, in case meta cache is expired at client. + // Once we know regionLookupInMetaLen, use it to generate largest possible + // rowkey that is lower than initStartRowKey by using + // ByteUtil#previousKeyWithLength function, which appends "\\xFF" bytes to + // prev rowKey up to the length provided. e.g. for the given key + // "\\x01\\xC1\\x06", the previous key with length 5 would be + // "\\x01\\xC1\\x05\\xFF\\xFF" by padding 2 bytes "\\xFF". + // The length of the largest scan start rowkey should not exceed + // HConstants#MAX_ROW_LENGTH. + int regionLookupInMetaLen = + RegionInfo.createRegionName(region.getTableDescriptor().getTableName(), new byte[1], + HConstants.NINES, false).length; + if ( + Bytes.compareTo(initStartRowKey, initStartRowKey.length - 1, 1, ByteUtil.ZERO_BYTE, 0, 1) + == 0 + ) { + // If initStartRowKey has last byte as "\\x00", we can discard the last + // byte and send the key as dummy rowKey. + prevKey = new byte[initStartRowKey.length - 1]; + System.arraycopy(initStartRowKey, 0, prevKey, 0, prevKey.length); + } else + if (initStartRowKey.length < (HConstants.MAX_ROW_LENGTH - 1 - regionLookupInMetaLen)) { + prevKey = + ByteUtil.previousKeyWithLength( + ByteUtil.concat(initStartRowKey, + new byte[HConstants.MAX_ROW_LENGTH - initStartRowKey.length - 1 + - regionLookupInMetaLen]), + HConstants.MAX_ROW_LENGTH - 1 - regionLookupInMetaLen); + } else { + prevKey = initStartRowKey; + } + getDummyResult(prevKey, result); + } else { + getDummyResult(initStartRowKey, result); + } + } + } + + @Override + public void close() throws IOException { + try { + super.close(); + } catch (Exception e) { + LOGGER.error("Error closing PhoenixSyncTableRegionScanner", e); + } + } +} 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 d85f8005405..7974c193cc9 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 @@ -24,6 +24,7 @@ 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 static org.apache.phoenix.util.ScanUtil.adjustScanFilterForGlobalIndexRegionScanner; +import static org.apache.phoenix.util.ScanUtil.getPageSizeMsForFilter; import java.io.ByteArrayInputStream; import java.io.DataInputStream; @@ -417,6 +418,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, getPageSizeMsForFilter(scan)); + } + }); } 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..28ec1ce4404 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixNoOpSingleRecordReader.java @@ -0,0 +1,114 @@ +/* + * 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: + *

    + *
  • Doesn't need actual row data from the RecordReader
  • + *
  • Only needs split/region boundaries (accessible via {@code context.getInputSplit()})
  • + *
  • Delegates all work to a server-side coprocessor
  • + *
+ *

+ * 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: + *

    + *
  • {@link #nextKeyValue()} returns {@code true} exactly once, then {@code false}
  • + *
  • This triggers {@code map()} exactly once per InputSplit (region)
  • + *
  • The mapper extracts region boundaries from the InputSplit, not from records
  • + *
+ * @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 subsequent calls which makes Mapper task to exit calling + * map method + */ + @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..c6ddb5bbd7d --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormat.java @@ -0,0 +1,213 @@ +/* + * 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.HConstants; +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; + +import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; + +/** + * 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); + + 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 PhoenixNoOpSingleRecordReader instance + */ + @SuppressWarnings("rawtypes") + @Override + public RecordReader createRecordReader(InputSplit split, TaskAttemptContext context) { + return new PhoenixNoOpSingleRecordReader(); + } + + /** + * 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 { + 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; + try { + completedRegions = + queryCompletedMapperRegions(conf, tableName, targetZkQuorum, fromTime, toTime); + } catch (SQLException e) { + throw new RuntimeException(e); + } + if (completedRegions.isEmpty()) { + LOGGER.info("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 + */ + @VisibleForTesting + 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; + + // 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 = 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. 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 + ) { + completedIdx++; + } else if ( + !Bytes.equals(splitEnd, HConstants.EMPTY_END_ROW) + && Bytes.compareTo(completedStart, splitEnd) >= 0 + ) { + // 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 + // ---- [----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) { + 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; + } +} 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..7a036ee4e33 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapper.java @@ -0,0 +1,723 @@ +/* + * 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.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; +import org.apache.hadoop.hbase.HConstants; +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.query.QueryServicesOptions; +import org.apache.phoenix.schema.PTable; +import org.apache.phoenix.util.MetaDataUtil; +import org.apache.phoenix.util.PhoenixRuntime; +import org.apache.phoenix.util.SHA256DigestUtil; +import org.apache.phoenix.util.ScanUtil; +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 validating table data 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, + SOURCE_ROWS_PROCESSED, + TARGET_ROWS_PROCESSED + } + + private String tableName; + private String targetZkQuorum; + private Long fromTime; + private Long toTime; + private boolean isDryRun; + private long chunkSizeBytes; + 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); + 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 PhoenixSyncTableMapper 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, krb principal + */ + private void connectToTargetCluster() throws SQLException, IOException { + Configuration targetConf = + PhoenixMapReduceUtil.createConfigurationForZkQuorum(conf, targetZkQuorum); + 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 check pointed progress and records final + * status for chunks & mapper (VERIFIED/MISMATCHED). + */ + @Override + protected void map(NullWritable key, DBInputFormat.NullDBWritable value, Context context) + throws IOException, InterruptedException { + context.getCounter(PhoenixJobCounters.INPUT_RECORDS).increment(1); + try { + List processedChunks = + syncTableOutputRepository.getProcessedChunks(tableName, targetZkQuorum, fromTime, toTime, + mapperRegionStart, mapperRegionEnd); + List> unprocessedRanges = + calculateUnprocessedRanges(mapperRegionStart, mapperRegionEnd, processedChunks); + boolean isStartKeyInclusive = shouldStartKeyBeInclusive(mapperRegionStart, processedChunks); + for (Pair range : unprocessedRanges) { + processMapperRanges(range.getFirst(), range.getSecond(), isStartKeyInclusive, context); + isStartKeyInclusive = false; + } + + long mismatchedChunk = context.getCounter(SyncCounters.CHUNKS_MISMATCHED).getValue(); + long verifiedChunk = context.getCounter(SyncCounters.CHUNKS_VERIFIED).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()); + String counters = formatMapperCounters(verifiedChunk, mismatchedChunk, sourceRowsProcessed, + targetRowsProcessed); + + 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: {} verified chunks, {} mismatched chunks", + verifiedChunk, mismatchedChunk); + } else { + context.getCounter(PhoenixJobCounters.FAILED_RECORDS).increment(1); + LOGGER.warn( + "PhoenixSyncTable mapper completed with mismatch: {} verified chunks, {} mismatched chunks", + verifiedChunk, mismatchedChunk); + syncTableOutputRepository.checkpointSyncTableResult(tableName, targetZkQuorum, + 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); + } + } catch (SQLException e) { + tryClosingResources(); + 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 given mapper region boundary. Target chunking: Follows + * source chunk boundaries. 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 isSourceStartKeyInclusive Whether startKey be inclusive for source chunking + * @param context Mapper context for progress and counters + * @throws IOException if scan fails + * @throws SQLException if database operations fail + */ + private void processMapperRanges(byte[] rangeStart, byte[] rangeEnd, + boolean isSourceStartKeyInclusive, Context context) throws IOException, SQLException { + // To handle scenario of target having extra keys compared to source keys: + // For every source chunk, we track whether its first chunk of Region or whether its lastChunk + // of region + // For every source chunk, we issue scan on target with + // - FirstChunkOfRegion : target scan start boundary would be rangeStart + // - LastChunkOfRegion : target scan end boundary would be rangeEnd + // - notFirstChunkOfRegion: target scan start boundary would be previous source chunk endKey + // - notLastChunkOfRegion: target scan end boundary would be current source chunk endKey + // Lets understand with an example. + // Source region boundary is [c,n) and source chunk returns [c1,d] , here `c` key is not present + // in source + // It could be the case that target has `c` present, so we issue scan on target chunk with + // startKey as `c` and not `c1` i.e [c,d] + // Similarly, if two consecutive source chunk returns its boundary as [e,g] and [h,j] + // When target is scanning for [h,j], it would issue scan with (g,j] to ensure we cover any + // extra key which is not in source but present in target + // + // Now eventually when chunking will reach for last source chunk on this region boundary, we + // again pass rangeEnd(with Exclusive) as target chunk boundary. + // Lets say, for above region boundary example second last and last sourceChunk returns [j,k] + // and [l,m]. Target chunk would issue scan for last chunk (k,n) + boolean isLastChunkOfRegion = false; + // We only want target startKey to be inclusive if source startKey is inclusive as well + // Source start key won't be inclusive if start of region boundary is already processed as chunk + // and check pointed + // Refer to shouldStartKeyBeInclusive() method to understand more about when source start key + // would be exclusive + boolean isTargetStartKeyInclusive = isSourceStartKeyInclusive; + try (ChunkScannerContext sourceScanner = createChunkScanner(sourceConnection, rangeStart, + rangeEnd, null, isSourceStartKeyInclusive, false, false)) { + ChunkInfo previousSourceChunk = null; + ChunkInfo sourceChunk = sourceScanner.getNextChunk(); + while (sourceChunk != null) { + sourceChunk.executionStartTime = new Timestamp(System.currentTimeMillis()); + // Peek ahead to see if this is the last chunk + ChunkInfo nextSourceChunk = sourceScanner.getNextChunk(); + if (nextSourceChunk == null) { + isLastChunkOfRegion = true; + } + ChunkInfo targetChunk = getTargetChunkWithSourceBoundary(targetConnection, + previousSourceChunk == null ? rangeStart : previousSourceChunk.endKey, + isLastChunkOfRegion ? rangeEnd : sourceChunk.endKey, isTargetStartKeyInclusive, + !isLastChunkOfRegion); + 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( + "isSourceStartKeyInclusive: {}, isTargetStartKeyInclusive: {}," + + "isTargetEndKeyInclusive: {}, isFirstChunkOfRegion: {}, isLastChunkOfRegion: {}." + + "Chunk comparison source {}, {}. Key range passed to target chunk: {}, {}." + + "target chunk returned {}, {}: source={} rows, target={} rows, matched={}", + isSourceStartKeyInclusive, isTargetStartKeyInclusive, !isLastChunkOfRegion, + previousSourceChunk == null, isLastChunkOfRegion, + Bytes.toStringBinary(sourceChunk.startKey), Bytes.toStringBinary(sourceChunk.endKey), + Bytes.toStringBinary( + previousSourceChunk == null ? rangeStart : previousSourceChunk.endKey), + Bytes.toStringBinary(isLastChunkOfRegion ? rangeEnd : sourceChunk.endKey), + Bytes.toStringBinary(targetChunk.startKey), Bytes.toStringBinary(targetChunk.endKey), + sourceChunk.rowCount, targetChunk.rowCount, matched); + } + sourceChunk.executionEndTime = new Timestamp(System.currentTimeMillis()); + String counters = formatChunkCounters(sourceChunk.rowCount, targetChunk.rowCount); + if (matched) { + handleVerifiedChunk(sourceChunk, context, counters); + } else { + handleMismatchedChunk(sourceChunk, context, counters); + } + previousSourceChunk = sourceChunk; + sourceChunk = nextSourceChunk; + // After first chunk, our target chunk boundary would be previousSourceChunk.endKey, + // so start key should not be inclusive + isTargetStartKeyInclusive = false; + context.progress(); + } + } + 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 + * 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 + * @return Single ChunkInfo with final hash from all target regions + */ + private ChunkInfo getTargetChunkWithSourceBoundary(Connection conn, byte[] startKey, + byte[] endKey, boolean isTargetStartKeyInclusive, boolean isTargetEndKeyInclusive) + throws IOException, SQLException { + ChunkInfo combinedTargetChunk = new ChunkInfo(); + combinedTargetChunk.startKey = null; + combinedTargetChunk.endKey = null; + combinedTargetChunk.hash = null; + combinedTargetChunk.rowCount = 0; + byte[] currentStartKey = startKey; + byte[] continuedDigestState = null; + ChunkInfo chunk; + while (true) { + // Each iteration scans one target region. The coprocessor processes all rows in + // that region within the scan range. For target boundary, the chunk is always + // marked partial and the digest state is passed to the next + // scanner for cross-region hash continuation. + try (ChunkScannerContext scanner = createChunkScanner(conn, currentStartKey, endKey, + continuedDigestState, isTargetStartKeyInclusive, isTargetEndKeyInclusive, true)) { + chunk = scanner.getNextChunk(); + // chunk == null means no more rows in the target range. + // We must finalize the digest to produce a proper checksum for comparison. + if (chunk == null) { + if (continuedDigestState != null) { + combinedTargetChunk.hash = + SHA256DigestUtil.finalizeDigestToChecksum(continuedDigestState); + } + break; + } + if (combinedTargetChunk.startKey == null) { + combinedTargetChunk.startKey = chunk.startKey; + } + combinedTargetChunk.endKey = chunk.endKey; + combinedTargetChunk.rowCount += chunk.rowCount; + continuedDigestState = chunk.hash; + currentStartKey = chunk.endKey; + isTargetStartKeyInclusive = false; + } + } + return combinedTargetChunk; + } + + /** + * 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 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 + * @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 { + // 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.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); + } + + if (!isTargetScan) { + scan.setAttribute(BaseScannerRegionObserverConstants.SYNC_TABLE_CHUNK_SIZE_BYTES, + Bytes.toBytes(chunkSizeBytes)); + } + long syncTablePageTimeoutMs = (long) (conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, + QueryServicesOptions.DEFAULT_SYNC_TABLE_RPC_TIMEOUT) * 0.5); + scan.setAttribute(BaseScannerRegionObserverConstants.SERVER_PAGE_SIZE_MS, + Bytes.toBytes(syncTablePageTimeoutMs)); + 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()); + Cell startKeyCell = MetaDataUtil.getCell(cells, + BaseScannerRegionObserverConstants.SYNC_TABLE_START_KEY_QUALIFIER); + Cell rowCountCell = MetaDataUtil.getCell(cells, + BaseScannerRegionObserverConstants.SYNC_TABLE_ROW_COUNT_QUALIFIER); + Cell isPartialChunkCell = MetaDataUtil.getCell(cells, + BaseScannerRegionObserverConstants.SYNC_TABLE_IS_PARTIAL_CHUNK_QUALIFIER); + Cell hashCell = + MetaDataUtil.getCell(cells, BaseScannerRegionObserverConstants.SYNC_TABLE_HASH_QUALIFIER); + + if ( + startKeyCell == null || rowCountCell == null || isPartialChunkCell == null || hashCell == null + ) { + throw new RuntimeException("Missing required chunk metadata cells."); + } + + ChunkInfo info = new ChunkInfo(); + info.startKey = CellUtil.cloneValue(startKeyCell); + info.endKey = result.getRow(); + info.rowCount = Bytes.toLong(rowCountCell.getValueArray(), rowCountCell.getValueOffset(), + rowCountCell.getValueLength()); + info.isPartial = isPartialChunkCell.getValueArray()[isPartialChunkCell.getValueOffset()] != 0; + info.hash = CellUtil.cloneValue(hashCell); + return info; + } + + /** + * Formats chunk counters as a comma-separated string. + * @param sourceRows Source rows processed + * @param targetRows Target rows processed + * @return Formatted string: "SOURCE_ROWS_PROCESSED=123,TARGET_ROWS_PROCESSED=456" + */ + 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. + * @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); + } + + 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.executionEndTime, counters); + context.getCounter(SyncCounters.CHUNKS_VERIFIED).increment(1); + } + + 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); + + 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, boolean isTargetScan) throws IOException { + Scan scan = new Scan(); + scan.withStartRow(startKey, isStartKeyInclusive); + scan.withStopRow(endKey, isEndKeyInclusive); + scan.setRaw(true); + scan.readAllVersions(); + scan.setCacheBlocks(false); + scan.setTimeRange(fromTime, toTime); + if (isTargetScan) { + scan.setLimit(1); + scan.setCaching(1); + } + return scan; + } + + /** + * 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() + * @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 is null or empty, the entire mapper region needs processing + if (processedChunks == null || processedChunks.isEmpty()) { + gaps.add(new Pair<>(mapperRegionStart, mapperRegionEnd)); + return gaps; + } + + // 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(); + 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 + // ---[20---MapperRegion---------------85) + // [10---chunk1---30]------- + 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; + } + + // 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 + // ---[20---MapperRegion---------------85) + // ------------------------------[70---chunk1---90]------- + 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; + } + + // 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; + } + + /*** + * Checking if start key should be inclusive, this is specific to scenario when there are + * processed chunks within this Mapper region boundary. + */ + boolean shouldStartKeyBeInclusive(byte[] mapperRegionStart, + List processedChunks) { + // Only with processed chunk like below we need to + // have first unprocessedRanges startKeyInclusive = true. + // [---MapperRegion---------------) + // -----[--chunk1--] [--chunk2--] + // + // Otherwise with processed chunk like below, we don't want startKeyInclusive = true + // for any of unprocessedRange + // ---[---MapperRegion---------------) + // [--chunk1--] [--chunk2--] + // OR + // [---MapperRegion---------------) + // [--chunk1--] [--chunk2--] + 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(); + super.cleanup(context); + } + + private void tryClosingResources() { + if (sourceConnection != null) { + try { + sourceConnection.close(); + } catch (SQLException e) { + LOGGER.error("Error while closing source connection in PhoenixSyncTableMapper", e); + } + } + if (targetConnection != null) { + try { + targetConnection.close(); + } catch (SQLException e) { + LOGGER.error("Error while closing target connection in PhoenixSyncTableMapper", e); + } + } + if (globalConnection != null) { + try { + globalConnection.close(); + } 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; + Timestamp executionStartTime; + Timestamp executionEndTime; + + @Override + public String toString() { + return String.format("Chunk[start=%s, end=%s, rows=%d, partial=%s]", + Bytes.toStringBinary(startKey), Bytes.toStringBinary(endKey), rowCount, isPartial); + } + } + + /** + * 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 { + while (true) { + Result result = scanner.next(); + if (result == null || result.isEmpty()) { + return null; + } + // Skip dummy results and continue scanning + if (ScanUtil.isDummy(result)) { + LOGGER.info("Skipping dummy paging result at row {}, continuing scan", + Bytes.toStringBinary(result.getRow())); + continue; + } + 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..3ab34be1c6c --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepository.java @@ -0,0 +1,215 @@ +/* + * 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.List; +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_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 { + + 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 = 90 * 24 * 60 * 60; // 90 days + private final Connection connection; + private static final String UPSERT_CHECKPOINT_SQL = "UPSERT INTO " + + SYNC_TABLE_CHECKPOINT_TABLE_NAME + " (TABLE_NAME, TARGET_CLUSTER, TYPE, FROM_TIME, TO_TIME," + + " START_ROW_KEY, END_ROW_KEY, IS_DRY_RUN, EXECUTION_START_TIME, EXECUTION_END_TIME," + + " STATUS, COUNTERS) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)"; + + /** + * 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; + } + + 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" + " START_ROW_KEY VARBINARY_ENCODED,\n" + + " END_ROW_KEY VARBINARY_ENCODED,\n" + " IS_DRY_RUN 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" + + " 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); + } + } + + 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"); + } + + try (PreparedStatement ps = connection.prepareStatement(UPSERT_CHECKPOINT_SQL)) { + ps.setString(1, tableName); + ps.setString(2, targetCluster); + ps.setString(3, type.name()); + ps.setLong(4, fromTime); + ps.setLong(5, toTime); + ps.setBytes(6, startKey); + ps.setBytes(7, endKey); + ps.setBoolean(8, isDryRun); + ps.setTimestamp(9, executionStartTime); + ps.setTimestamp(10, executionEndTime); + ps.setString(11, status != null ? status.name() : null); + ps.setString(12, counters); + ps.executeUpdate(); + connection.commit(); + } + } + + /** + * 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 FROM " + SYNC_TABLE_CHECKPOINT_TABLE_NAME + + " WHERE TABLE_NAME = ? AND TARGET_CLUSTER = ?" + + " AND TYPE = ? AND FROM_TIME = ? AND TO_TIME = ? AND STATUS IN ( ?, ?)"; + 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()) { + PhoenixSyncTableOutputRow row = + new PhoenixSyncTableOutputRow.Builder().setStartRowKey(rs.getBytes("START_ROW_KEY")) + .setEndRowKey(rs.getBytes("END_ROW_KEY")).build(); + results.add(row); + } + } + } + return results; + } + + /** + * 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) + * @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 { + StringBuilder queryBuilder = new StringBuilder(); + queryBuilder.append("SELECT START_ROW_KEY, END_ROW_KEY 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(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); + 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()) { + while (rs.next()) { + byte[] rawStartKey = rs.getBytes("START_ROW_KEY"); + byte[] endRowKey = rs.getBytes("END_ROW_KEY"); + PhoenixSyncTableOutputRow row = new PhoenixSyncTableOutputRow.Builder() + .setStartRowKey(rawStartKey).setEndRowKey(endRowKey).build(); + results.add(row); + } + } + } + return results; + } +} 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..bb0623cc223 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRow.java @@ -0,0 +1,265 @@ +/* + * 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.Timestamp; +import java.util.Arrays; +import java.util.Objects; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.util.Bytes; + +import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; + +/** + * Data model class representing required row in the PHOENIX_SYNC_TABLE_CHECKPOINT table + */ +public class PhoenixSyncTableOutputRow { + + public enum Type { + CHUNK, + MAPPER_REGION + } + + public enum Status { + VERIFIED, + MISMATCHED + } + + private String tableName; + private String targetCluster; + private Type type; + private Long fromTime; + private Long toTime; + private Boolean isDryRun; + private byte[] startRowKey; + private byte[] endRowKey; + private Boolean isFirstRegion; + private Timestamp executionStartTime; + private Timestamp executionEndTime; + private Status status; + private String counters; + + @Override + public String toString() { + return String.format("SyncOutputRow[table=%s, target=%s, type=%s, start=%s, end=%s, status=%s]", + tableName, targetCluster, type, Bytes.toStringBinary(startRowKey), + Bytes.toStringBinary(endRowKey), status); + } + + @Override + @VisibleForTesting + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PhoenixSyncTableOutputRow that = (PhoenixSyncTableOutputRow) o; + return Objects.equals(tableName, that.tableName) + && Objects.equals(targetCluster, that.targetCluster) && type == that.type + && Objects.equals(fromTime, that.fromTime) && Objects.equals(toTime, that.toTime) + && Objects.equals(isDryRun, that.isDryRun) && Arrays.equals(startRowKey, that.startRowKey) + && Arrays.equals(endRowKey, that.endRowKey) + && Objects.equals(isFirstRegion, that.isFirstRegion) + && Objects.equals(executionStartTime, that.executionStartTime) + && Objects.equals(executionEndTime, that.executionEndTime) && status == that.status + && Objects.equals(counters, that.counters); + } + + @Override + public int hashCode() { + int result = Objects.hash(tableName, targetCluster, type, fromTime, toTime, isDryRun, + isFirstRegion, executionStartTime, executionEndTime, status, counters); + result = 31 * result + Arrays.hashCode(startRowKey); + result = 31 * result + Arrays.hashCode(endRowKey); + return result; + } + + @VisibleForTesting + public String getTableName() { + return tableName; + } + + @VisibleForTesting + public String getTargetCluster() { + return targetCluster; + } + + @VisibleForTesting + public Type getType() { + return type; + } + + @VisibleForTesting + public Long getFromTime() { + return fromTime; + } + + @VisibleForTesting + public Long getToTime() { + return toTime; + } + + public byte[] getStartRowKey() { + return startRowKey != null ? Arrays.copyOf(startRowKey, startRowKey.length) : null; + } + + public byte[] getEndRowKey() { + return endRowKey != null ? Arrays.copyOf(endRowKey, endRowKey.length) : null; + } + + @VisibleForTesting + public Timestamp getExecutionStartTime() { + return executionStartTime; + } + + @VisibleForTesting + public Timestamp getExecutionEndTime() { + return executionEndTime; + } + + @VisibleForTesting + public Status getStatus() { + return status; + } + + @VisibleForTesting + public String getCounters() { + return counters; + } + + @VisibleForTesting + public long getSourceRowsProcessed() { + return parseCounterValue(PhoenixSyncTableMapper.SyncCounters.SOURCE_ROWS_PROCESSED.name()); + } + + @VisibleForTesting + public long getTargetRowsProcessed() { + return parseCounterValue(PhoenixSyncTableMapper.SyncCounters.TARGET_ROWS_PROCESSED.name()); + } + + @VisibleForTesting + private long parseCounterValue(String counterName) { + if (counters == null || counters.isEmpty()) { + return 0; + } + + String[] pairs = counters.split(","); + for (String pair : pairs) { + String[] keyValue = pair.split("="); + if (keyValue.length == 2 && keyValue[0].trim().equals(counterName)) { + return Long.parseLong(keyValue[1].trim()); + } + } + return 0; + } + + /** + * Builder for PhoenixSyncTableOutputRow + */ + public static class Builder { + private final PhoenixSyncTableOutputRow row; + + public Builder() { + this.row = new PhoenixSyncTableOutputRow(); + } + + @VisibleForTesting + public Builder setTableName(String tableName) { + row.tableName = tableName; + return this; + } + + @VisibleForTesting + public Builder setTargetCluster(String targetCluster) { + row.targetCluster = targetCluster; + return this; + } + + @VisibleForTesting + public Builder setType(Type type) { + row.type = type; + return this; + } + + @VisibleForTesting + public Builder setFromTime(Long fromTime) { + row.fromTime = fromTime; + return this; + } + + @VisibleForTesting + public Builder setToTime(Long toTime) { + row.toTime = toTime; + return this; + } + + @VisibleForTesting + public Builder setIsDryRun(Boolean isDryRun) { + row.isDryRun = isDryRun; + return this; + } + + 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 || endRowKey.length == 0) + ? HConstants.EMPTY_END_ROW + : Arrays.copyOf(endRowKey, endRowKey.length); + return this; + } + + @VisibleForTesting + public Builder setIsFirstRegion(Boolean isFirstRegion) { + row.isFirstRegion = isFirstRegion; + return this; + } + + @VisibleForTesting + public Builder setExecutionStartTime(Timestamp executionStartTime) { + row.executionStartTime = executionStartTime; + return this; + } + + @VisibleForTesting + public Builder setExecutionEndTime(Timestamp executionEndTime) { + row.executionEndTime = executionEndTime; + return this; + } + + @VisibleForTesting + public Builder setStatus(Status status) { + row.status = status; + return this; + } + + @VisibleForTesting + public Builder setCounters(String counters) { + row.counters = counters; + return this; + } + + public PhoenixSyncTableOutputRow build() { + 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..af77f6cd75b --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/PhoenixSyncTableTool.java @@ -0,0 +1,468 @@ +/* + * 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.PhoenixMRJobUtil; +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.
  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 an 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 check pointed 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("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 = + 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 an 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); + PhoenixMRJobUtil.updateCapacityQueueInfo(configuration); + Job job = Job.getInstance(configuration, getJobName()); + job.setMapperClass(PhoenixSyncTableMapper.class); + job.setJarByClass(PhoenixSyncTableTool.class); + TableMapReduceUtil.initCredentials(job); + TableMapReduceUtil.addDependencyJars(job); + configureInput(job, tableType); + configureOutput(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(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) { + // 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("PhoenixSyncTableTool"); + if (qSchemaName != null) { + jobName.append("-").append(qSchemaName); + } + jobName.append("-").append(tableName); + jobName.append("-").append(System.currentTimeMillis()); + return jobName.toString(); + } + + public 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) { + 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 { + // 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())); + if (chunkSizeBytes <= 0) { + throw new IllegalArgumentException( + "Chunk size must be a positive value, got: " + chunkSizeBytes); + } + } + 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 table + */ + private void createSyncOutputTable(Connection connection) throws SQLException { + PhoenixSyncTableOutputRepository repository = new PhoenixSyncTableOutputRepository(connection); + repository.createSyncCheckpointTableIfNotExists(); + } + + /** + * Sets up the table reference and validates it exists and is suitable for sync operations. + * Validates that the table is not a VIEW or INDEX + */ + 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); + } + Counters counters = job.getCounters(); + 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; + } + + @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(); + 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); + } + + public Job getJob() { + return job; + } + + @VisibleForTesting + public Long getStartTime() { + return startTime; + } + + @VisibleForTesting + public Long getEndTime() { + return endTime; + } + + @VisibleForTesting + public String getTenantId() { + return tenantId; + } + + @VisibleForTesting + public String getSchemaName() { + return schemaName; + } + + @VisibleForTesting + public Long getChunkSizeBytes() { + return chunkSizeBytes; + } + + @VisibleForTesting + public boolean isDryRun() { + return isDryRun; + } + + @VisibleForTesting + public boolean isForeground() { + return isForeground; + } + + @VisibleForTesting + public void initializeConfiguration() { + configuration = HBaseConfiguration.addHbaseResources(getConf()); + } +} 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 c9111aa7b88..7fbc213b14e 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 @@ -39,7 +39,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; @@ -403,7 +402,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 6edfc9370c1..cc918dc46f3 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 @@ -298,9 +298,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"; @@ -927,9 +924,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); @@ -954,6 +948,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; } @@ -984,15 +981,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 473c2fa33b2..daca9a04616 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; @@ -334,10 +333,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())) ) { @@ -362,6 +357,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 0bd4830c291..96c159cb02f 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 @@ -197,6 +197,21 @@ 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 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 * aggregates 2. UPDATE_STATS collects statistics for the table @@ -890,4 +905,67 @@ 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, true); + } + + 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); + } } 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 4378ed56cfe..6dcab4690c6 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,20 +18,30 @@ 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.PhoenixOutputFormat; 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 */ public final class PhoenixMapReduceUtil { + public static final String INVALID_TIME_RANGE_EXCEPTION_MESSAGE = "Invalid time range for table"; + private PhoenixMapReduceUtil() { } @@ -223,4 +233,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 (et > currentTime || st >= et) { + throw new IllegalArgumentException(String.format( + "%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)); + } + } + + /** + * 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 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; + } } diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml index 99d45dc6b9e..8c242392170 100644 --- a/phoenix-core/pom.xml +++ b/phoenix-core/pom.xml @@ -404,6 +404,12 @@ log4j-1.2-api test + + org.bouncycastle + bcprov-jdk18on + 1.79 + test + diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixSyncTableToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixSyncTableToolIT.java new file mode 100644 index 00000000000..fe8ba4967e0 --- /dev/null +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixSyncTableToolIT.java @@ -0,0 +1,2267 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.phoenix.end2end; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.DriverManager; +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.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.RegionInfo; +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.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.mapreduce.Counters; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.phoenix.jdbc.HighAvailabilityTestingUtility.HBaseTestingUtilityPair; +import org.apache.phoenix.jdbc.PhoenixConnection; +import org.apache.phoenix.jdbc.PhoenixDriver; +import org.apache.phoenix.mapreduce.PhoenixSyncTableMapper.SyncCounters; +import org.apache.phoenix.mapreduce.PhoenixSyncTableOutputRepository; +import org.apache.phoenix.mapreduce.PhoenixSyncTableOutputRow; +import org.apache.phoenix.mapreduce.PhoenixSyncTableTool; +import org.apache.phoenix.query.BaseTest; +import org.apache.phoenix.query.QueryServices; +import org.apache.phoenix.schema.PTable; +import org.apache.phoenix.schema.types.PInteger; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category(NeedsOwnMiniClusterTest.class) +public class PhoenixSyncTableToolIT { + private static final Logger LOGGER = LoggerFactory.getLogger(PhoenixSyncTableToolIT.class); + + @Rule + public final TestName testName = new TestName(); + + private static final HBaseTestingUtilityPair CLUSTERS = new HBaseTestingUtilityPair(); + private static final int REPLICATION_WAIT_TIMEOUT_MS = 10000; + + private Connection sourceConnection; + private Connection targetConnection; + private String targetZkQuorum; + private String uniqueTableName; + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + CLUSTERS.start(); + DriverManager.registerDriver(PhoenixDriver.INSTANCE); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + DriverManager.deregisterDriver(PhoenixDriver.INSTANCE); + CLUSTERS.close(); + } + + @Before + public void setUp() throws Exception { + sourceConnection = DriverManager.getConnection("jdbc:phoenix:" + CLUSTERS.getZkUrl1()); + targetConnection = DriverManager.getConnection("jdbc:phoenix:" + CLUSTERS.getZkUrl2()); + uniqueTableName = BaseTest.generateUniqueName(); + targetZkQuorum = String.format("%s:%d:/hbase", + CLUSTERS.getHBaseCluster2().getConfiguration().get("hbase.zookeeper.quorum"), + CLUSTERS.getHBaseCluster2().getZkCluster().getClientPort()); + } + + @After + public void tearDown() throws Exception { + if (sourceConnection != null && uniqueTableName != null) { + try { + dropTableIfExists(sourceConnection, uniqueTableName); + dropTableIfExists(sourceConnection, uniqueTableName + "_IDX"); // For global index test + dropTableIfExists(sourceConnection, uniqueTableName + "_LOCAL_IDX"); // For local index test + cleanupCheckpointTable(sourceConnection, uniqueTableName, targetZkQuorum); + cleanupCheckpointTable(sourceConnection, uniqueTableName + "_IDX", targetZkQuorum); + cleanupCheckpointTable(sourceConnection, uniqueTableName + "_LOCAL_IDX", targetZkQuorum); + } catch (Exception e) { + LOGGER.warn("Failed to cleanup tables for {}: {}", uniqueTableName, e.getMessage()); + } + } + + if (targetConnection != null && uniqueTableName != null) { + try { + dropTableIfExists(targetConnection, uniqueTableName); + dropTableIfExists(targetConnection, uniqueTableName + "_IDX"); // For global index test + dropTableIfExists(targetConnection, uniqueTableName + "_LOCAL_IDX"); // For local index test + } catch (Exception e) { + LOGGER.warn("Failed to cleanup tables on target for {}: {}", uniqueTableName, + e.getMessage()); + } + } + + if (sourceConnection != null) { + sourceConnection.close(); + } + if (targetConnection != null) { + targetConnection.close(); + } + } + + @Test + public void testSyncTableValidateWithDataDifference() throws Exception { + setupStandardTestWithReplication(uniqueTableName, 1, 10); + + introduceAndVerifyTargetDifferences(uniqueTableName); + + Job job = runSyncToolWithLargeChunks(uniqueTableName); + SyncCountersResult counters = getSyncCounters(job); + + validateSyncCounters(counters, 10, 10, 1, 3); + + List checkpointEntries = + queryCheckpointTable(sourceConnection, uniqueTableName, targetZkQuorum); + validateCheckpointEntries(checkpointEntries, uniqueTableName, targetZkQuorum, 10, 10, 1, 3, 4, + 3); + } + + @Test + public void testSyncTableWithDeletedRowsOnTarget() throws Exception { + setupStandardTestWithReplication(uniqueTableName, 1, 10); + + deleteRowsOnTarget(targetConnection, uniqueTableName, 1, 4, 9); + + // Verify row counts differ between source and target + int sourceCount = getRowCount(sourceConnection, uniqueTableName); + int targetCount = getRowCount(targetConnection, uniqueTableName); + assertEquals("Source should have 10 rows", 10, sourceCount); + assertEquals("Target should have 7 rows (3 deleted)", 7, targetCount); + + Job job = runSyncTool(uniqueTableName); + SyncCountersResult counters = getSyncCounters(job); + + validateSyncCounters(counters, 10, 10, 7, 3); + } + + @Test + public void testSyncValidateIndexTable() throws Exception { + // Create data table on both clusters with replication + createTableOnBothClusters(sourceConnection, targetConnection, uniqueTableName); + + // Create index on both clusters + String indexName = uniqueTableName + "_IDX"; + createIndexOnBothClusters(sourceConnection, targetConnection, uniqueTableName, indexName); + + // Insert data on source + insertTestData(sourceConnection, uniqueTableName, 1, 10); + + // Wait for replication to target (both data table and index) + waitForReplication(targetConnection, uniqueTableName, 10); + + // Verify initial replication + verifyDataIdentical(sourceConnection, targetConnection, uniqueTableName); + + deleteHBaseRows(CLUSTERS.getHBaseCluster2(), uniqueTableName, 3); + deleteHBaseRows(CLUSTERS.getHBaseCluster2(), indexName, 3); + + Job job = runSyncTool(indexName); + SyncCountersResult counters = getSyncCounters(job); + + assertEquals("Should process 10 source rows", 10, counters.sourceRowsProcessed); + assertTrue("Some chunk should be verified", counters.chunksVerified > 0); + assertTrue("Some chunk should be mismatched", counters.chunksMismatched > 0); + + // Verify checkpoint entries show mismatches + List checkpointEntries = + queryCheckpointTable(sourceConnection, indexName, targetZkQuorum); + + assertFalse("Should have checkpointEntries", checkpointEntries.isEmpty()); + } + + @Test + public void testSyncValidateLocalIndexTable() throws Exception { + // Create data table on both clusters with replication + createTableOnBothClusters(sourceConnection, targetConnection, uniqueTableName); + + // Create LOCAL index on both clusters + String indexName = uniqueTableName + "_LOCAL_IDX"; + createLocalIndexOnBothClusters(sourceConnection, targetConnection, uniqueTableName, indexName); + + // Insert data on source + insertTestData(sourceConnection, uniqueTableName, 1, 10); + + // Wait for replication to target (both data table and local index) + waitForReplication(targetConnection, uniqueTableName, 10); + + // Verify initial replication + verifyDataIdentical(sourceConnection, targetConnection, uniqueTableName); + + deleteHBaseRows(CLUSTERS.getHBaseCluster2(), uniqueTableName, 5); + + // Run sync tool on the LOCAL INDEX table (not the data table) + Job job = runSyncTool(indexName); + SyncCountersResult counters = getSyncCounters(job); + + assertEquals("Should process 20 source rows", 20, counters.sourceRowsProcessed); + assertTrue("Some chunk should be verified", counters.chunksVerified > 0); + assertTrue("Some chunk should be mismatched", counters.chunksMismatched > 0); + + // Verify checkpoint entries + List checkpointEntries = + queryCheckpointTable(sourceConnection, indexName, targetZkQuorum); + + assertFalse("Should have checkpoint entries for local index", checkpointEntries.isEmpty()); + } + + @Test + public void testSyncValidateMultiTenantSaltedTableDifferences() throws Exception { + String[] tenantIds = new String[] { "TENANT_001", "TENANT_002", "TENANT_003" }; + int rowsPerTenant = 10; + createMultiTenantSaltedTableOnBothClusters(sourceConnection, targetConnection, uniqueTableName); + + for (String tenantId : tenantIds) { + Connection tenantSourceConn = getTenantConnection(sourceConnection, tenantId); + insertMultiTenantTestData(tenantSourceConn, uniqueTableName, 1, rowsPerTenant); + tenantSourceConn.close(); + } + + waitForReplication(targetConnection, uniqueTableName, 30); + + for (String tenantId : tenantIds) { + withTenantConnections(tenantId, + (sourceConn, targetConn) -> verifyDataIdentical(sourceConn, targetConn, uniqueTableName)); + } + + // Introduce differences specific to TENANT_002 on target cluster + Connection tenant002TargetConnForUpdate = getTenantConnection(targetConnection, tenantIds[1]); + introduceMultiTenantTargetDifferences(tenant002TargetConnForUpdate, uniqueTableName); + tenant002TargetConnForUpdate.close(); + + // Verify TENANT_001 and TENANT_003 still have identical data + for (int i = 0; i < tenantIds.length; i++) { + if (i == 1) continue; // Skip TENANT_002 as we introduced differences + + final String tenantId = tenantIds[i]; + withTenantConnections(tenantId, (sourceConn, targetConn) -> { + List sourceRows = queryAllRows(sourceConn, + "SELECT ID, NAME, NAME_VALUE FROM " + uniqueTableName + " ORDER BY ID"); + List targetRows = queryAllRows(targetConn, + "SELECT ID, NAME, NAME_VALUE FROM " + uniqueTableName + " ORDER BY ID"); + assertEquals("Tenant " + tenantId + " should still have identical data", sourceRows, + targetRows); + }); + } + + // TENANT_001 has no differences, expect all rows verified + Job job1 = runSyncTool(uniqueTableName, "--tenant-id", tenantIds[0]); + SyncCountersResult counters1 = getSyncCounters(job1); + validateSyncCounters(counters1, 10, 10, 10, 0); + + // TENANT_002 has 3 modified rows + Job job2 = runSyncTool(uniqueTableName, "--tenant-id", tenantIds[1]); + SyncCountersResult counters2 = getSyncCounters(job2); + validateSyncCounters(counters2, 10, 10, 7, 3); + } + + @Test + public void testSyncTableValidateWithTimeRangeFilter() throws Exception { + createTableOnBothClusters(sourceConnection, targetConnection, uniqueTableName); + + // Insert data BEFORE the time range window + insertTestData(sourceConnection, uniqueTableName, 1, 10); + + long startTime = System.currentTimeMillis(); + + // Insert data WITHIN the time range window + insertTestData(sourceConnection, uniqueTableName, 11, 20); + + long endTime = System.currentTimeMillis(); + + // Insert data AFTER the time range window + insertTestData(sourceConnection, uniqueTableName, 21, 30); + + // Wait for replication to complete + waitForReplication(targetConnection, uniqueTableName, 30); + + // Verify all data replicated correctly + verifyDataIdentical(sourceConnection, targetConnection, uniqueTableName); + + // Modify rows BEFORE startTime time + upsertRowsOnTarget(targetConnection, uniqueTableName, new int[] { 3, 5, 8 }, + new String[] { "MODIFIED_NAME_3", "MODIFIED_NAME_5", "MODIFIED_NAME_8" }); + + // Modify rows AFTER end time + upsertRowsOnTarget(targetConnection, uniqueTableName, new int[] { 23, 25, 28 }, + new String[] { "MODIFIED_NAME_23", "MODIFIED_NAME_25", "MODIFIED_NAME_28" }); + + // Run sync tool with time range covering ONLY the middle window + Job job = runSyncTool(uniqueTableName, "--from-time", String.valueOf(startTime), "--to-time", + String.valueOf(endTime)); + SyncCountersResult counters = getSyncCounters(job); + + validateSyncCounters(counters, 10, 10, 10, 0); + } + + @Test + public void testSyncTableValidateCheckpointWithPartialReRunAndRegionSplits() throws Exception { + setupStandardTestWithReplication(uniqueTableName, 1, 100); + + List sourceSplits = Arrays.asList(15, 45, 51, 75, 95); + splitTableAt(sourceConnection, uniqueTableName, sourceSplits); + + // Introduce differences on target scattered across the dataset + List mismatchIds = Arrays.asList(10, 25, 40, 55, 70, 85, 95); + for (int id : mismatchIds) { + upsertRowsOnTarget(targetConnection, uniqueTableName, new int[] { id }, + new String[] { "MODIFIED_NAME_" + id }); + } + + // Capture consistent time range for both runs + long fromTime = 0L; + long toTime = System.currentTimeMillis(); + + // Run sync tool for the FIRST time with explicit time range + Job job1 = runSyncTool(uniqueTableName, "--from-time", String.valueOf(fromTime), "--to-time", + String.valueOf(toTime)); + SyncCountersResult counters1 = getSyncCounters(job1); + + // Validate first run counters - should process all 100 rows + assertEquals("Should process 100 source rows", 100, counters1.sourceRowsProcessed); + assertEquals("Should process 100 target rows", 100, counters1.targetRowsProcessed); + assertTrue("Should have at least 1 mismatched chunks", counters1.chunksMismatched > 0); + + List checkpointEntries = + queryCheckpointTable(sourceConnection, uniqueTableName, targetZkQuorum); + + assertFalse("Should have checkpoint entries after first run", checkpointEntries.isEmpty()); + + // Separate mapper and chunk entries using utility method + SeparatedCheckpointEntries separated = separateMapperAndChunkEntries(checkpointEntries); + List allMappers = separated.mappers; + List allChunks = separated.chunks; + + assertFalse("Should have mapper region entries", allMappers.isEmpty()); + assertFalse("Should have chunk entries", allChunks.isEmpty()); + + // Select 3/4th of chunks from each mapper to delete (simulating partial rerun) + // We repro the partial run via deleting some entries from checkpoint table and re-running the + // tool + List chunksToDelete = selectChunksToDeleteFromMappers( + sourceConnection, uniqueTableName, targetZkQuorum, fromTime, toTime, allMappers, 0.75); + + // Delete all mappers and selected chunks + int deletedCount = deleteCheckpointEntries(sourceConnection, uniqueTableName, targetZkQuorum, + allMappers, chunksToDelete); + + assertEquals("Should have deleted all mapper and selected chunk entries", + allMappers.size() + chunksToDelete.size(), deletedCount); + + List checkpointEntriesAfterDelete = + queryCheckpointTable(sourceConnection, uniqueTableName, targetZkQuorum); + assertEquals("Should have fewer checkpoint entries after deletion", + allMappers.size() + chunksToDelete.size(), + checkpointEntries.size() - checkpointEntriesAfterDelete.size()); + + // Calculate totals from REMAINING CHUNK entries in checkpoint table using utility method + CheckpointAggregateCounters remainingCounters = + calculateAggregateCountersFromCheckpoint(checkpointEntriesAfterDelete); + + List additionalSourceSplits = + Arrays.asList(12, 22, 28, 32, 42, 52, 58, 62, 72, 78, 82, 92); + splitTableAt(sourceConnection, uniqueTableName, additionalSourceSplits); + + List targetSplits = Arrays.asList(25, 40, 50, 65, 70, 80, 90); + splitTableAt(targetConnection, uniqueTableName, targetSplits); + + // Run sync tool again with SAME time range - should reprocess only deleted regions + // despite the new region boundaries from splits + Job job2 = runSyncTool(uniqueTableName, "--from-time", String.valueOf(fromTime), "--to-time", + String.valueOf(toTime)); + SyncCountersResult counters2 = getSyncCounters(job2); + + LOGGER.info( + "Second run - Processed: {} source rows, {} target rows, {} verified chunks, {} mismatched chunks", + counters2.sourceRowsProcessed, counters2.targetRowsProcessed, counters2.chunksVerified, + counters2.chunksMismatched); + + // (Remaining chunks from checkpoint) + (Second run) should equal (First run) + long totalSourceRows = remainingCounters.sourceRowsProcessed + counters2.sourceRowsProcessed; + long totalTargetRows = remainingCounters.targetRowsProcessed + counters2.targetRowsProcessed; + long totalVerifiedChunks = remainingCounters.chunksVerified + counters2.chunksVerified; + long totalMismatchedChunks = remainingCounters.chunksMismatched + counters2.chunksMismatched; + + assertEquals( + "Remaining + Second run source rows should equal first run source rows. " + "Remaining: " + + remainingCounters.sourceRowsProcessed + ", Second run: " + counters2.sourceRowsProcessed + + ", Total: " + totalSourceRows + ", Expected: " + counters1.sourceRowsProcessed, + counters1.sourceRowsProcessed, totalSourceRows); + + assertEquals( + "Remaining + Second run target rows should equal first run target rows. " + "Remaining: " + + remainingCounters.targetRowsProcessed + ", Second run: " + counters2.targetRowsProcessed + + ", Total: " + totalTargetRows + ", Expected: " + counters1.targetRowsProcessed, + counters1.targetRowsProcessed, totalTargetRows); + + assertEquals("Remaining + Second run verified chunks should equal first run verified chunks. " + + "Remaining: " + remainingCounters.chunksVerified + ", Second run: " + + counters2.chunksVerified + ", Total: " + totalVerifiedChunks + ", Expected: " + + counters1.chunksVerified, counters1.chunksVerified, totalVerifiedChunks); + + assertEquals( + "Remaining + Second run mismatched chunks should equal first run mismatched chunks. " + + "Remaining: " + remainingCounters.chunksMismatched + ", Second run: " + + counters2.chunksMismatched + ", Total: " + totalMismatchedChunks + ", Expected: " + + counters1.chunksMismatched, + counters1.chunksMismatched, totalMismatchedChunks); + + // Verify checkpoint table has entries for the reprocessed regions + List checkpointEntriesAfterRerun = + queryCheckpointTable(sourceConnection, uniqueTableName, targetZkQuorum); + + // After rerun, we should have at least more entries compared to delete table + assertTrue("Should have checkpoint entries after rerun", + checkpointEntriesAfterRerun.size() > checkpointEntriesAfterDelete.size()); + } + + @Test + public void testSyncTableValidateCheckpointWithPartialReRunAndRegionMerges() throws Exception { + setupStandardTestWithReplication(uniqueTableName, 1, 100); + + List sourceSplits = Arrays.asList(10, 20, 30, 40, 50, 60, 70, 80, 90); + splitTableAt(sourceConnection, uniqueTableName, sourceSplits); + + List mismatchIds = Arrays.asList(5, 15, 25, 35, 45, 55, 65, 75, 85, 95); + for (int id : mismatchIds) { + upsertRowsOnTarget(targetConnection, uniqueTableName, new int[] { id }, + new String[] { "MODIFIED_NAME_" + id }); + } + + long fromTime = 0L; + long toTime = System.currentTimeMillis(); + + Job job1 = runSyncTool(uniqueTableName, "--from-time", String.valueOf(fromTime), "--to-time", + String.valueOf(toTime)); + SyncCountersResult counters1 = getSyncCounters(job1); + + assertEquals("Should process 100 source rows", 100, counters1.sourceRowsProcessed); + assertEquals("Should process 100 target rows", 100, counters1.targetRowsProcessed); + assertTrue("Should have mismatched chunks", counters1.chunksMismatched > 0); + + List checkpointEntries = + queryCheckpointTable(sourceConnection, uniqueTableName, targetZkQuorum); + + assertTrue("Should have checkpoint entries after first run", !checkpointEntries.isEmpty()); + + // Separate mapper and chunk entries using utility method + SeparatedCheckpointEntries separated = separateMapperAndChunkEntries(checkpointEntries); + List allMappers = separated.mappers; + List allChunks = separated.chunks; + + assertFalse("Should have mapper region entries", allMappers.isEmpty()); + assertFalse("Should have chunk entries", allChunks.isEmpty()); + + // Select 3/4th of chunks from each mapper to delete (simulating partial rerun) + // We repro the partial run via deleting some entries from checkpoint table and re-running the + List chunksToDelete = selectChunksToDeleteFromMappers( + sourceConnection, uniqueTableName, targetZkQuorum, fromTime, toTime, allMappers, 0.75); + + // Delete all mappers and selected chunks + int deletedCount = deleteCheckpointEntries(sourceConnection, uniqueTableName, targetZkQuorum, + allMappers, chunksToDelete); + + assertEquals("Should have deleted all mapper and selected chunk entries", + allMappers.size() + chunksToDelete.size(), deletedCount); + + List checkpointEntriesAfterDelete = + queryCheckpointTable(sourceConnection, uniqueTableName, targetZkQuorum); + assertEquals("Should have fewer checkpoint entries after deletion", + allMappers.size() + chunksToDelete.size(), + checkpointEntries.size() - checkpointEntriesAfterDelete.size()); + + // Calculate totals from REMAINING CHUNK entries in checkpoint table using utility method + CheckpointAggregateCounters remainingCounters = + calculateAggregateCountersFromCheckpoint(checkpointEntriesAfterDelete); + + // Merge adjacent regions on source (merge 6 pairs of regions) + mergeAdjacentRegions(sourceConnection, uniqueTableName, 6); + + // Merge adjacent regions on target (merge 6 pairs of regions) + mergeAdjacentRegions(targetConnection, uniqueTableName, 6); + + // Run sync tool again with SAME time range - should reprocess only deleted regions + // despite the new region boundaries from merges + Job job2 = runSyncTool(uniqueTableName, "--from-time", String.valueOf(fromTime), "--to-time", + String.valueOf(toTime)); + SyncCountersResult counters2 = getSyncCounters(job2); + + long totalSourceRows = remainingCounters.sourceRowsProcessed + counters2.sourceRowsProcessed; + long totalTargetRows = remainingCounters.targetRowsProcessed + counters2.targetRowsProcessed; + long totalVerifiedChunks = remainingCounters.chunksVerified + counters2.chunksVerified; + long totalMismatchedChunks = remainingCounters.chunksMismatched + counters2.chunksMismatched; + + assertEquals( + "Remaining + Second run source rows should equal first run source rows. " + "Remaining: " + + remainingCounters.sourceRowsProcessed + ", Second run: " + counters2.sourceRowsProcessed + + ", Total: " + totalSourceRows + ", Expected: " + counters1.sourceRowsProcessed, + counters1.sourceRowsProcessed, totalSourceRows); + + assertEquals( + "Remaining + Second run target rows should equal first run target rows. " + "Remaining: " + + remainingCounters.targetRowsProcessed + ", Second run: " + counters2.targetRowsProcessed + + ", Total: " + totalTargetRows + ", Expected: " + counters1.targetRowsProcessed, + counters1.targetRowsProcessed, totalTargetRows); + + assertEquals("Remaining + Second run verified chunks should equal first run verified chunks. " + + "Remaining: " + remainingCounters.chunksVerified + ", Second run: " + + counters2.chunksVerified + ", Total: " + totalVerifiedChunks + ", Expected: " + + counters1.chunksVerified, counters1.chunksVerified, totalVerifiedChunks); + + assertEquals( + "Remaining + Second run mismatched chunks should equal first run mismatched chunks. " + + "Remaining: " + remainingCounters.chunksMismatched + ", Second run: " + + counters2.chunksMismatched + ", Total: " + totalMismatchedChunks + ", Expected: " + + counters1.chunksMismatched, + counters1.chunksMismatched, totalMismatchedChunks); + + List checkpointEntriesAfterRerun = + queryCheckpointTable(sourceConnection, uniqueTableName, targetZkQuorum); + + // After rerun with merges, we should have more entries as after deletion + assertTrue("Should have checkpoint entries after rerun", + checkpointEntriesAfterRerun.size() > checkpointEntriesAfterDelete.size()); + } + + @Test + public void testSyncTableValidateIdempotentOnReRun() throws Exception { + setupStandardTestWithReplication(uniqueTableName, 1, 10); + + // Capture consistent time range for both runs (ensures checkpoint lookup will match) + long fromTime = 0L; + long toTime = System.currentTimeMillis(); + + // Run sync tool for the FIRST time + Job job1 = runSyncTool(uniqueTableName, "--from-time", String.valueOf(fromTime), "--to-time", + String.valueOf(toTime)); + SyncCountersResult counters1 = getSyncCounters(job1); + + // Validate first run counters + validateSyncCounters(counters1, 10, 10, 10, 0); + + // Query checkpoint table to verify entries were created + List checkpointEntriesAfterFirstRun = + queryCheckpointTable(sourceConnection, uniqueTableName, targetZkQuorum); + + // Run sync tool for the SECOND time WITHOUT deleting any checkpoints (idempotent behavior) + Job job2 = runSyncTool(uniqueTableName, "--from-time", String.valueOf(fromTime), "--to-time", + String.valueOf(toTime)); + SyncCountersResult counters2 = getSyncCounters(job2); + + // Second run should process ZERO rows (idempotent behavior) + validateSyncCounters(counters2, 0, 0, 0, 0); + + List checkpointEntriesAfterSecondRun = + queryCheckpointTable(sourceConnection, uniqueTableName, targetZkQuorum); + + assertEquals("Checkpoint entries should be identical after idempotent run", + checkpointEntriesAfterFirstRun, checkpointEntriesAfterSecondRun); + } + + @Test + public void testSyncTableValidateIdempotentAfterRegionSplits() throws Exception { + // Setup table with initial splits and data + setupStandardTestWithReplication(uniqueTableName, 1, 10); + + // Verify data is identical after replication + verifyDataIdentical(sourceConnection, targetConnection, uniqueTableName); + + // Capture consistent time range for both runs + long fromTime = 0L; + long toTime = System.currentTimeMillis(); + + // Run sync tool for the FIRST time (no differences, all chunks verified) + Job job1 = runSyncToolWithLargeChunks(uniqueTableName, "--from-time", String.valueOf(fromTime), + "--to-time", String.valueOf(toTime)); + SyncCountersResult counters1 = getSyncCounters(job1); + + // Validate first run: all rows processed, no mismatches + validateSyncCounters(counters1, 10, 10, 4, 0); + + // Query checkpoint table to verify entries were created + List checkpointEntriesAfterFirstRun = + queryCheckpointTable(sourceConnection, uniqueTableName, targetZkQuorum); + + assertFalse("Should have checkpoint entries after first run", + checkpointEntriesAfterFirstRun.isEmpty()); + + // Attempt to split tables on BOTH source and target at new split points + // Some splits may fail if regions are in transition, which is acceptable for this test + splitTableAt(sourceConnection, uniqueTableName, Arrays.asList(2, 6)); + splitTableAt(targetConnection, uniqueTableName, Arrays.asList(3, 7)); + + // Run sync tool for the SECOND time after splits (idempotent behavior) + Job job2 = runSyncTool(uniqueTableName, "--from-time", String.valueOf(fromTime), "--to-time", + String.valueOf(toTime)); + SyncCountersResult counters2 = getSyncCounters(job2); + + // KEY VALIDATION: Second run should process ZERO rows despite new region boundaries + validateSyncCounters(counters2, 0, 0, 0, 0); + + List checkpointEntriesAfterSecondRun = + queryCheckpointTable(sourceConnection, uniqueTableName, targetZkQuorum); + + // Checkpoint entries may differ in count due to new regions, but all original data is + // checkpointed + assertFalse("Should have checkpoint entries after second run", + checkpointEntriesAfterSecondRun.isEmpty()); + } + + @Test + public void testSyncTableValidateWithSchemaAndTableNameOptions() throws Exception { + setupStandardTestWithReplication(uniqueTableName, 1, 10); + + // Introduce differences on target + introduceAndVerifyTargetDifferences(uniqueTableName); + + // Run sync tool with both --schema and --table-name options + Job job = runSyncTool(uniqueTableName, "--schema", ""); + SyncCountersResult counters = getSyncCounters(job); + + // Validate counters + validateSyncCounters(counters, 10, 10, 7, 3); + } + + @Test + public void testSyncTableValidateInBackgroundMode() throws Exception { + setupStandardTestWithReplication(uniqueTableName, 1, 10); + + introduceAndVerifyTargetDifferences(uniqueTableName); + + Configuration conf = new Configuration(CLUSTERS.getHBaseCluster1().getConfiguration()); + String[] args = + new String[] { "--table-name", uniqueTableName, "--target-cluster", targetZkQuorum, + "--chunk-size", "1", "--to-time", String.valueOf(System.currentTimeMillis()) }; + + PhoenixSyncTableTool tool = new PhoenixSyncTableTool(); + tool.setConf(conf); + int exitCode = tool.run(args); + + Job job = tool.getJob(); + assertNotNull("Job should not be null", job); + assertEquals("Tool should submit job successfully", 0, exitCode); + + boolean jobCompleted = job.waitForCompletion(true); + assertTrue("Background job should complete successfully", jobCompleted); + + SyncCountersResult counters = new SyncCountersResult(job.getCounters()); + counters.logCounters(testName.getMethodName()); + + validateSyncCounters(counters, 10, 10, 7, 3); + } + + @Test + public void testSyncTableValidateWithCustomTimeouts() throws Exception { + setupStandardTestWithReplication(uniqueTableName, 1, 10); + + introduceAndVerifyTargetDifferences(uniqueTableName); + + // Create configuration with custom timeout values + Configuration conf = new Configuration(CLUSTERS.getHBaseCluster1().getConfiguration()); + + // Set custom timeout values (higher than defaults to ensure job succeeds) + long customQueryTimeout = 900000L; // 15 minutes + long customRpcTimeout = 1200000L; // 20 minutes + long customScannerTimeout = 2400000L; // 40 minutes + int customRpcRetries = 10; + + conf.setLong(QueryServices.SYNC_TABLE_QUERY_TIMEOUT_ATTRIB, customQueryTimeout); + conf.setLong(QueryServices.SYNC_TABLE_RPC_TIMEOUT_ATTRIB, customRpcTimeout); + conf.setLong(QueryServices.SYNC_TABLE_CLIENT_SCANNER_TIMEOUT_ATTRIB, customScannerTimeout); + conf.setInt(QueryServices.SYNC_TABLE_RPC_RETRIES_COUNTER, customRpcRetries); + + String[] args = new String[] { "--table-name", uniqueTableName, "--target-cluster", + targetZkQuorum, "--chunk-size", "1", "--run-foreground", "--to-time", + String.valueOf(System.currentTimeMillis()) }; + + PhoenixSyncTableTool tool = new PhoenixSyncTableTool(); + tool.setConf(conf); + int exitCode = tool.run(args); + + Job job = tool.getJob(); + assertNotNull("Job should not be null", job); + assertEquals("Tool should complete successfully with custom timeouts", 0, exitCode); + + // Verify that custom timeout configurations were applied to the job + Configuration jobConf = job.getConfiguration(); + assertEquals("Custom query timeout should be applied", customQueryTimeout, + jobConf.getLong(MRJobConfig.TASK_TIMEOUT, -1)); + assertEquals("Custom RPC timeout should be applied", customRpcTimeout, + jobConf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, -1)); + assertEquals("Custom scanner timeout should be applied", customScannerTimeout, + jobConf.getLong(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, -1)); + assertEquals("Custom RPC retries should be applied", customRpcRetries, + jobConf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, -1)); + + // Verify sync completed successfully + SyncCountersResult counters = new SyncCountersResult(job.getCounters()); + counters.logCounters(testName.getMethodName()); + validateSyncCounters(counters, 10, 10, 7, 3); + } + + @Test + public void testSyncTableValidateWithExtraRowsOnTarget() throws Exception { + // Create tables on both clusters + createTableOnBothClusters(sourceConnection, targetConnection, uniqueTableName); + + // Insert data on source with HOLES (gaps in the sequence) + List oddIds = Arrays.asList(1, 3, 5, 7, 9, 11, 13, 15, 17, 19); + insertTestData(sourceConnection, uniqueTableName, oddIds); + + // Wait for replication to target + waitForReplication(targetConnection, uniqueTableName, 10); + + // Verify initial replication + verifyDataIdentical(sourceConnection, targetConnection, uniqueTableName); + + // Insert rows in the HOLES on target cluster only + // Target gets: 2, 4, 6, 8, 10 (5 even numbers filling the gaps in first half) + List evenIds = Arrays.asList(2, 4, 6, 8, 10); + insertTestData(targetConnection, uniqueTableName, evenIds); + + // Verify target now has more rows than source + int sourceCount = getRowCount(sourceConnection, uniqueTableName); + int targetCount = getRowCount(targetConnection, uniqueTableName); + assertEquals("Source should have 10 rows (odd numbers 1-19)", 10, sourceCount); + assertEquals("Target should have 15 rows (odd 1-19 + even 2-10)", 15, targetCount); + + // Run sync tool to detect the extra rows interspersed on target + Job job = runSyncTool(uniqueTableName); + SyncCountersResult counters = getSyncCounters(job); + + validateSyncCounters(counters, 10, 15, 5, 5); + + List checkpointEntries = + queryCheckpointTable(sourceConnection, uniqueTableName, targetZkQuorum); + + // Count mismatched entries in checkpoint table + int mismatchedCount = 0; + for (PhoenixSyncTableOutputRow entry : checkpointEntries) { + if (PhoenixSyncTableOutputRow.Status.MISMATCHED.equals(entry.getStatus())) { + mismatchedCount++; + } + } + assertTrue("Should have mismatched entries for chunks with extra rows", mismatchedCount > 0); + + // Verify source and target are still different + List sourceRows = queryAllRows(sourceConnection, + "SELECT ID, NAME, NAME_VALUE FROM " + uniqueTableName + " ORDER BY ID"); + List targetRows = queryAllRows(targetConnection, + "SELECT ID, NAME, NAME_VALUE FROM " + uniqueTableName + " ORDER BY ID"); + assertEquals("Source should still have 10 rows", 10, sourceRows.size()); + assertEquals("Target should still have 15 rows", 15, targetRows.size()); + assertNotEquals("Source and target should have different data", sourceRows, targetRows); + + // Verify that source has only odd numbers + for (TestRow row : sourceRows) { + assertEquals("Source should only have odd IDs", 1, row.id % 2); + } + + // Verify that target has all numbers 1-11 (with gaps filled) and 13,15,17,19 + assertEquals("Target should have ID=1", 1, targetRows.get(0).id); + assertEquals("Target should have ID=2", 2, targetRows.get(1).id); + assertEquals("Target should have ID=10", 10, targetRows.get(9).id); + assertEquals("Target should have ID=11", 11, targetRows.get(10).id); + assertEquals("Target should have ID=19", 19, targetRows.get(14).id); + } + + @Test + public void testSyncTableValidateWithConcurrentRegionSplits() throws Exception { + setupStandardTestWithReplication(uniqueTableName, 1, 100); + // Introduce some mismatches on target before sync + List mismatchIds = Arrays.asList(15, 35, 55, 75, 95); + for (int id : mismatchIds) { + upsertRowsOnTarget(targetConnection, uniqueTableName, new int[] { id }, + new String[] { "MODIFIED_NAME_" + id }); + } + + // Capture time range for the sync + long fromTime = 0L; + long toTime = System.currentTimeMillis(); + + // Create a thread that will perform splits on source cluster during sync + Thread sourceSplitThread = new Thread(() -> { + try { + // Split source at multiple points (creating more regions during sync) + List sourceSplits = Arrays.asList(20, 25, 40, 45, 60, 65, 80, 85, 95); + splitTableAt(sourceConnection, uniqueTableName, sourceSplits); + } catch (Exception e) { + LOGGER.error("Error during source splits", e); + } + }); + + // Create a thread that will perform splits on target cluster during sync + Thread targetSplitThread = new Thread(() -> { + try { + // Split target at different points than source (asymmetric region boundaries) + List targetSplits = Arrays.asList(11, 21, 31, 41, 51, 75, 81, 91); + splitTableAt(targetConnection, uniqueTableName, targetSplits); + } catch (Exception e) { + LOGGER.error("Error during target splits", e); + } + }); + + // Start split threads + sourceSplitThread.start(); + targetSplitThread.start(); + + // Run sync tool while splits are happening + // Use smaller chunk size to increase chances of hitting split boundaries + Job job = runSyncToolWithChunkSize(uniqueTableName, 512, "--from-time", + String.valueOf(fromTime), "--to-time", String.valueOf(toTime)); + + // Wait for split threads to complete + sourceSplitThread.join(30000); // 30 second timeout + targetSplitThread.join(30000); + + // Verify the job completed successfully despite concurrent splits + assertTrue("Sync job should complete successfully despite concurrent splits", + job.isSuccessful()); + + SyncCountersResult counters = getSyncCounters(job); + + // Validate counters - should process all 100 rows and detect the 5 mismatched rows + validateSyncCountersExactSourceTarget(counters, 100, 100, 1, 1); + + // Verify checkpoint entries were created + List checkpointEntries = + queryCheckpointTable(sourceConnection, uniqueTableName, targetZkQuorum); + assertFalse("Should have checkpoint entries", checkpointEntries.isEmpty()); + + // Count mismatched entries + int mismatchedCount = 0; + for (PhoenixSyncTableOutputRow entry : checkpointEntries) { + if (PhoenixSyncTableOutputRow.Status.MISMATCHED.equals(entry.getStatus())) { + mismatchedCount++; + } + } + assertTrue("Should have mismatched entries for modified rows", mismatchedCount >= 5); + } + + @Test + public void testSyncTableValidateWithOnlyTimestampDifferences() throws Exception { + createTableOnBothClusters(sourceConnection, targetConnection, uniqueTableName); + + // Define two different timestamps + long timestamp1 = System.currentTimeMillis(); + Thread.sleep(100); // Ensure different timestamp + long timestamp2 = System.currentTimeMillis(); + + // Insert same data on source with timestamp1 + insertTestData(sourceConnection, uniqueTableName, 1, 10, timestamp1); + + // Insert same data on target with timestamp2 (different timestamp, same values) + insertTestData(targetConnection, uniqueTableName, 1, 10, timestamp2); + + // Verify both have same row count and same values + int sourceCount = getRowCount(sourceConnection, uniqueTableName); + int targetCount = getRowCount(targetConnection, uniqueTableName); + assertEquals("Both should have 10 rows", sourceCount, targetCount); + + // Query and verify data values are identical (but timestamps differ) + List sourceRows = queryAllRows(sourceConnection, + "SELECT ID, NAME, NAME_VALUE FROM " + uniqueTableName + " ORDER BY ID"); + List targetRows = queryAllRows(targetConnection, + "SELECT ID, NAME, NAME_VALUE FROM " + uniqueTableName + " ORDER BY ID"); + assertEquals("Row values should be identical", sourceRows, targetRows); + + // Run sync tool - should detect timestamp differences as mismatches + Job job = runSyncTool(uniqueTableName); + SyncCountersResult counters = getSyncCounters(job); + + // Validate counters - all rows should be processed and all chunks mismatched + // because timestamps are included in the hash calculation + validateSyncCounters(counters, 10, 10, 0, 10); + + // Verify checkpoint entries show mismatches + List checkpointEntries = + queryCheckpointTable(sourceConnection, uniqueTableName, targetZkQuorum); + + int mismatchedCount = 0; + for (PhoenixSyncTableOutputRow entry : checkpointEntries) { + if (PhoenixSyncTableOutputRow.Status.MISMATCHED.equals(entry.getStatus())) { + mismatchedCount++; + } + } + assertTrue("Should have mismatched entries due to timestamp differences", mismatchedCount > 0); + } + + @Test + public void testSyncTableValidateWithConcurrentRegionMerges() throws Exception { + setupStandardTestWithReplication(uniqueTableName, 1, 100); + // Explicitly split tables to create many regions for merging + List sourceSplits = Arrays.asList(10, 15, 20, 25, 40, 45, 60, 65, 80, 85); + splitTableAt(sourceConnection, uniqueTableName, sourceSplits); + + List targetSplits = Arrays.asList(12, 18, 22, 28, 42, 48, 62, 68, 82, 88); + splitTableAt(targetConnection, uniqueTableName, targetSplits); + + // Introduce some mismatches on target before sync + List mismatchIds = Arrays.asList(10, 30, 50, 70, 90); + for (int id : mismatchIds) { + upsertRowsOnTarget(targetConnection, uniqueTableName, new int[] { id }, + new String[] { "MODIFIED_NAME_" + id }); + } + + // Capture time range for the sync + long fromTime = 0L; + long toTime = System.currentTimeMillis(); + + // Create a thread that will perform merges on source cluster during sync + Thread sourceMergeThread = new Thread(() -> { + try { + // Merge adjacent regions on source + mergeAdjacentRegions(sourceConnection, uniqueTableName, 6); + } catch (Exception e) { + LOGGER.error("Error during source merges", e); + } + }); + + // Create a thread that will perform merges on target cluster during sync + Thread targetMergeThread = new Thread(() -> { + try { + mergeAdjacentRegions(targetConnection, uniqueTableName, 6); + } catch (Exception e) { + LOGGER.error("Error during target merges", e); + } + }); + + // Start merge threads + sourceMergeThread.start(); + targetMergeThread.start(); + + // Run sync tool while merges are happening + Job job = runSyncToolWithChunkSize(uniqueTableName, 512, "--from-time", + String.valueOf(fromTime), "--to-time", String.valueOf(toTime)); + + // Wait for merge threads to complete + sourceMergeThread.join(30000); // 30 second timeout + targetMergeThread.join(30000); + + // Verify the job completed successfully despite concurrent merges + assertTrue("Sync job should complete successfully despite concurrent merges", + job.isSuccessful()); + + SyncCountersResult counters = getSyncCounters(job); + + // Validate counters - should process all 100 rows and detect mismatched chunks + validateSyncCountersExactSourceTarget(counters, 100, 100, 1, 1); + + // Verify checkpoint entries were created + List checkpointEntries = + queryCheckpointTable(sourceConnection, uniqueTableName, targetZkQuorum); + assertFalse("Should have checkpoint entries", checkpointEntries.isEmpty()); + + // Run sync again to verify idempotent behavior after merges + Job job2 = runSyncToolWithChunkSize(uniqueTableName, 512, "--from-time", + String.valueOf(fromTime), "--to-time", String.valueOf(toTime)); + SyncCountersResult counters2 = getSyncCounters(job2); + + // Second run should process ZERO rows (all checkpointed despite region merges) + validateSyncCounters(counters2, 0, 0, 0, 0); + } + + @Test + public void testSyncTableValidateWithPagingTimeout() throws Exception { + // Create tables on both clusters + setupStandardTestWithReplication(uniqueTableName, 1, 100); + + // Introduce mismatches scattered across the dataset + List mismatchIds = Arrays.asList(15, 25, 35, 45, 55, 75); + for (int id : mismatchIds) { + upsertRowsOnTarget(targetConnection, uniqueTableName, new int[] { id }, + new String[] { "MODIFIED_NAME_" + id }); + } + + // First, run without aggressive paging to establish baseline chunk count + Configuration baselineConf = new Configuration(CLUSTERS.getHBaseCluster1().getConfiguration()); + String[] baselineArgs = new String[] { "--table-name", uniqueTableName, "--target-cluster", + targetZkQuorum, "--run-foreground", "--chunk-size", "10240", "--to-time", + String.valueOf(System.currentTimeMillis()) }; + + PhoenixSyncTableTool baselineTool = new PhoenixSyncTableTool(); + baselineTool.setConf(baselineConf); + baselineTool.run(baselineArgs); + Job baselineJob = baselineTool.getJob(); + long baselineChunkCount = + baselineJob.getCounters().findCounter(SyncCounters.CHUNKS_VERIFIED).getValue(); + + // Configure paging with aggressive timeouts to force mid-chunk timeouts + Configuration conf = new Configuration(CLUSTERS.getHBaseCluster1().getConfiguration()); + + long aggressiveRpcTimeout = 2L; + conf.setLong(QueryServices.SYNC_TABLE_RPC_TIMEOUT_ATTRIB, aggressiveRpcTimeout); + conf.setLong(HConstants.HBASE_RPC_TIMEOUT_KEY, aggressiveRpcTimeout); + + int chunkSize = 10240; + + long fromTime = 0L; + long toTime = System.currentTimeMillis(); + + // Run sync tool while splits are happening + Job job = runSyncToolWithChunkSize(uniqueTableName, chunkSize, conf, "--from-time", + String.valueOf(fromTime), "--to-time", String.valueOf(toTime)); + + // Verify the job completed successfully despite paging timeouts + assertTrue("Sync job should complete successfully despite paging", job.isSuccessful()); + + SyncCountersResult counters = getSyncCounters(job); + + // Validate that all 100 rows were processed on both sides + // Despite paging timeouts, no rows should be lost + validateSyncCountersExactSourceTarget(counters, 100, 100, 1, 1); + + long pagingChunkCount = counters.chunksVerified; + + assertTrue( + "Paging should create more chunks than baseline due to mid-chunk timeouts. " + "Baseline: " + + baselineChunkCount + ", Paging: " + pagingChunkCount, + pagingChunkCount > baselineChunkCount); + + // Verify checkpoint entries were created + List checkpointEntries = + queryCheckpointTable(sourceConnection, uniqueTableName, targetZkQuorum); + assertFalse("Should have checkpoint entries", checkpointEntries.isEmpty()); + } + + @Test + public void testSyncTableValidateWithPagingTimeoutWithSplits() throws Exception { + // Create tables on both clusters + setupStandardTestWithReplication(uniqueTableName, 1, 100); + + // Introduce mismatches scattered across the dataset + List mismatchIds = Arrays.asList(15, 25, 35, 45, 55, 75); + for (int id : mismatchIds) { + upsertRowsOnTarget(targetConnection, uniqueTableName, new int[] { id }, + new String[] { "MODIFIED_NAME_" + id }); + } + + // First, run without aggressive paging to establish baseline chunk count + Configuration baselineConf = new Configuration(CLUSTERS.getHBaseCluster1().getConfiguration()); + String[] baselineArgs = new String[] { "--table-name", uniqueTableName, "--target-cluster", + targetZkQuorum, "--run-foreground", "--chunk-size", "10240", "--to-time", + String.valueOf(System.currentTimeMillis()) }; + + PhoenixSyncTableTool baselineTool = new PhoenixSyncTableTool(); + baselineTool.setConf(baselineConf); + baselineTool.run(baselineArgs); + Job baselineJob = baselineTool.getJob(); + long baselineChunkCount = + baselineJob.getCounters().findCounter(SyncCounters.CHUNKS_VERIFIED).getValue(); + + // Configure paging with aggressive timeouts to force mid-chunk timeouts + Configuration conf = new Configuration(CLUSTERS.getHBaseCluster1().getConfiguration()); + + // Enable server-side paging + conf.setBoolean(QueryServices.PHOENIX_SERVER_PAGING_ENABLED_ATTRIB, true); + // Set extremely short paging timeout to force frequent paging + long aggressiveRpcTimeout = 1000L; // 1 second RPC timeout + conf.setLong(QueryServices.SYNC_TABLE_RPC_TIMEOUT_ATTRIB, aggressiveRpcTimeout); + conf.setLong(HConstants.HBASE_RPC_TIMEOUT_KEY, aggressiveRpcTimeout); + // Force server-side paging to occur by setting page size to 1ms + conf.setLong(QueryServices.PHOENIX_SERVER_PAGE_SIZE_MS, 1); + + int chunkSize = 10240; // 100KB + + // Create a thread that will perform splits on source cluster during sync + Thread sourceSplitThread = new Thread(() -> { + try { + List sourceSplits = Arrays.asList(12, 22, 32, 42, 52, 63, 72, 82, 92, 98); + splitTableAt(sourceConnection, uniqueTableName, sourceSplits); + } catch (Exception e) { + LOGGER.error("Error during source splits", e); + } + }); + + // Create a thread that will perform splits on target cluster during sync + Thread targetSplitThread = new Thread(() -> { + try { + List targetSplits = Arrays.asList(13, 23, 33, 43, 53, 64, 74, 84, 95, 99); + splitTableAt(targetConnection, uniqueTableName, targetSplits); + } catch (Exception e) { + LOGGER.error("Error during target splits", e); + } + }); + + // Start split threads + sourceSplitThread.start(); + targetSplitThread.start(); + + long fromTime = 0L; + long toTime = System.currentTimeMillis(); + + // Run sync tool while splits are happening + Job job = runSyncToolWithChunkSize(uniqueTableName, chunkSize, conf, "--from-time", + String.valueOf(fromTime), "--to-time", String.valueOf(toTime)); + + // Wait for split threads to complete + sourceSplitThread.join(30000); // 30 second timeout + targetSplitThread.join(30000); + + // Verify the job completed successfully despite concurrent splits and paging timeouts + assertTrue("Sync job should complete successfully despite paging and concurrent splits", + job.isSuccessful()); + + SyncCountersResult counters = getSyncCounters(job); + + // Validate that all 100 rows were processed on both sides + // Despite paging timeouts AND concurrent region splits, no rows should be lost + validateSyncCountersExactSourceTarget(counters, 100, 100, 1, 1); + + // Paging should create MORE chunks than baseline + // Concurrent region splits may also create additional chunks as mappers process new regions + long pagingChunkCount = counters.chunksVerified; + + assertTrue( + "Paging should create more chunks than baseline due to mid-chunk timeouts. " + "Baseline: " + + baselineChunkCount + ", Paging: " + pagingChunkCount, + pagingChunkCount > baselineChunkCount); + + // Verify checkpoint entries were created + List checkpointEntries = + queryCheckpointTable(sourceConnection, uniqueTableName, targetZkQuorum); + assertFalse("Should have checkpoint entries", checkpointEntries.isEmpty()); + } + + @Test + public void testSyncTableMapperFailsWithNonExistentTable() throws Exception { + setupStandardTestWithReplication(uniqueTableName, 1, 10); + + // Try to run sync tool on a NON-EXISTENT table + String nonExistentTable = "NON_EXISTENT_TABLE_" + System.currentTimeMillis(); + Configuration conf = new Configuration(CLUSTERS.getHBaseCluster1().getConfiguration()); + String[] args = new String[] { "--table-name", nonExistentTable, "--target-cluster", + targetZkQuorum, "--run-foreground", "--to-time", String.valueOf(System.currentTimeMillis()) }; + + PhoenixSyncTableTool tool = new PhoenixSyncTableTool(); + tool.setConf(conf); + + try { + int exitCode = tool.run(args); + assertTrue( + String.format("Table %s does not exist, mapper setup should fail", nonExistentTable), + exitCode != 0); + } catch (Exception ex) { + fail("Tool should return non-zero exit code on failure instead of throwing exception: " + + ex.getMessage()); + } + } + + @Test + public void testSyncTableMapperFailsWithInvalidTargetCluster() throws Exception { + // Create table on source cluster + setupStandardTestWithReplication(uniqueTableName, 1, 10); + + // Try to run sync tool with INVALID target cluster ZK quorum. + String invalidTargetZk = "invalid-zk-host:2181:/hbase"; + Configuration conf = new Configuration(CLUSTERS.getHBaseCluster1().getConfiguration()); + String[] args = + new String[] { "--table-name", uniqueTableName, "--target-cluster", invalidTargetZk, + "--run-foreground", "--to-time", String.valueOf(System.currentTimeMillis()) }; + + PhoenixSyncTableTool tool = new PhoenixSyncTableTool(); + tool.setConf(conf); + + try { + int exitCode = tool.run(args); + assertTrue( + String.format("Target cluster %s is invalid, mapper setup should fail", invalidTargetZk), + exitCode != 0); + } catch (Exception ex) { + fail("Tool should return non-zero exit code on failure instead of throwing exception: " + + ex.getMessage()); + } + } + + @Test + public void testSyncTableMapperFailsWithMissingTargetTable() throws Exception { + // Create table on source cluster ONLY (not on target); no replication needed + String sourceDdl = buildStandardTableDdl(uniqueTableName, false, "3, 5, 7"); + executeTableCreation(sourceConnection, sourceDdl); + + // Insert data on source + insertTestData(sourceConnection, uniqueTableName, 1, 10); + + // Don't create table on target - this will cause mapper map() to fail + // when trying to scan the non-existent target table + Configuration conf = new Configuration(CLUSTERS.getHBaseCluster1().getConfiguration()); + String[] args = new String[] { "--table-name", uniqueTableName, "--target-cluster", + targetZkQuorum, "--run-foreground", "--to-time", String.valueOf(System.currentTimeMillis()) }; + + PhoenixSyncTableTool tool = new PhoenixSyncTableTool(); + tool.setConf(conf); + + try { + int exitCode = tool.run(args); + assertTrue(String.format( + "Table %s does not exist on target cluster, mapper map() should fail during target scan", + uniqueTableName), exitCode != 0); + } catch (Exception ex) { + fail("Tool should return non-zero exit code on failure instead of throwing exception: " + + ex.getMessage()); + } + } + + @Test + public void testSyncTableCheckpointPersistsAcrossFailedRuns() throws Exception { + // Setup table with replication and insert data + // setupStandardTestWithReplication creates splits, resulting in multiple mapper regions + setupStandardTestWithReplication(uniqueTableName, 1, 10); + + // Capture time range for both runs (ensures checkpoint lookup will match) + long fromTime = 0L; + long toTime = System.currentTimeMillis(); + + // First run: Sync should succeed and create checkpoint entries for all mappers + Job job1 = runSyncTool(uniqueTableName, "--from-time", String.valueOf(fromTime), "--to-time", + String.valueOf(toTime)); + SyncCountersResult counters1 = getSyncCounters(job1); + + // Validate first run succeeded + assertTrue("First run should succeed", job1.isSuccessful()); + assertEquals("Should process 10 source rows", 10, counters1.sourceRowsProcessed); + assertEquals("Should process 10 target rows", 10, counters1.targetRowsProcessed); + + // Query checkpoint table to get all mapper entries + List allCheckpointEntries = + queryCheckpointTable(sourceConnection, uniqueTableName, targetZkQuorum); + + // Separate mapper and chunk entries using utility method + SeparatedCheckpointEntries separated = separateMapperAndChunkEntries(allCheckpointEntries); + List mapperEntries = separated.mappers; + List allChunks = separated.chunks; + + assertTrue("Should have at least 3 mapper entries after first run", mapperEntries.size() >= 3); + + // Select 3/4th of chunks from each mapper to delete (simulating partial rerun) + // We repro the partial run via deleting some entries from checkpoint table and re-running the + // tool. + List chunksToDelete = selectChunksToDeleteFromMappers( + sourceConnection, uniqueTableName, targetZkQuorum, fromTime, toTime, mapperEntries, 0.75); + + // Delete all mappers and selected chunks using utility method + deleteCheckpointEntries(sourceConnection, uniqueTableName, targetZkQuorum, mapperEntries, + chunksToDelete); + + // Verify mapper entries were deleted + List checkpointEntriesAfterDelete = + queryCheckpointTable(sourceConnection, uniqueTableName, targetZkQuorum); + SeparatedCheckpointEntries separatedAfterDelete = + separateMapperAndChunkEntries(checkpointEntriesAfterDelete); + + assertEquals("Should have 0 mapper entries after deleting all mappers", 0, + separatedAfterDelete.mappers.size()); + assertEquals("Should have remaining chunk entries after deletion", + allChunks.size() - chunksToDelete.size(), separatedAfterDelete.chunks.size()); + + // Drop target table to cause mapper failures during second run. + // Use HBase Admin directly because Phoenix DROP TABLE IF EXISTS via targetConnection + // may silently no-op in the shared-JVM mini-cluster due to metadata cache issues. + Admin targetAdmin = CLUSTERS.getHBaseCluster2().getConnection().getAdmin(); + TableName hbaseTableName = TableName.valueOf(uniqueTableName); + if (targetAdmin.tableExists(hbaseTableName)) { + targetAdmin.disableTable(hbaseTableName); + targetAdmin.deleteTable(hbaseTableName); + } + LOGGER.info("Dropped target table to cause mapper failures"); + + // Second run: Job should fail (exit code != 0) because target table is missing + Configuration conf = new Configuration(CLUSTERS.getHBaseCluster1().getConfiguration()); + String[] args = new String[] { "--table-name", uniqueTableName, "--target-cluster", + targetZkQuorum, "--run-foreground", "--from-time", String.valueOf(fromTime), "--to-time", + String.valueOf(toTime) }; + + PhoenixSyncTableTool tool = new PhoenixSyncTableTool(); + tool.setConf(conf); + int exitCode = tool.run(args); + + // Job should fail + assertTrue("Second run should fail with non-zero exit code due to missing target table", + exitCode != 0); + LOGGER.info("Second run failed as expected with exit code: {}", exitCode); + + // Remaining chunk entries that we dint delete should still persist despite job failure + List checkpointEntriesAfterFailedRun = + queryCheckpointTable(sourceConnection, uniqueTableName, targetZkQuorum); + SeparatedCheckpointEntries separatedAfterFailedRun = + separateMapperAndChunkEntries(checkpointEntriesAfterFailedRun); + + // After the failed run: + // - No mapper entries should exist (we deleted them all, and the job failed before creating new + // ones) + // - Only the remaining chunk entries (1/4th) should persist + assertEquals("Should have 0 mapper entries after failed run", 0, + separatedAfterFailedRun.mappers.size()); + assertEquals("Remaining chunk entries should persist after failed run", + allChunks.size() - chunksToDelete.size(), separatedAfterFailedRun.chunks.size()); + } + + /** + * Helper class to hold separated mapper and chunk entries. + */ + private static class SeparatedCheckpointEntries { + final List mappers; + final List chunks; + + SeparatedCheckpointEntries(List mappers, + List chunks) { + this.mappers = mappers; + this.chunks = chunks; + } + } + + /** + * Helper class to hold aggregated counters from checkpoint chunk entries. + */ + private static class CheckpointAggregateCounters { + final long sourceRowsProcessed; + final long targetRowsProcessed; + final long chunksVerified; + final long chunksMismatched; + + CheckpointAggregateCounters(long sourceRowsProcessed, long targetRowsProcessed, + long chunksVerified, long chunksMismatched) { + this.sourceRowsProcessed = sourceRowsProcessed; + this.targetRowsProcessed = targetRowsProcessed; + this.chunksVerified = chunksVerified; + this.chunksMismatched = chunksMismatched; + } + } + + /** + * Separates checkpoint entries into mapper and chunk entries. + */ + private SeparatedCheckpointEntries + separateMapperAndChunkEntries(List entries) { + List mappers = new ArrayList<>(); + List chunks = new ArrayList<>(); + + for (PhoenixSyncTableOutputRow entry : entries) { + if (PhoenixSyncTableOutputRow.Type.MAPPER_REGION.equals(entry.getType())) { + mappers.add(entry); + } else if (PhoenixSyncTableOutputRow.Type.CHUNK.equals(entry.getType())) { + chunks.add(entry); + } + } + + return new SeparatedCheckpointEntries(mappers, chunks); + } + + /** + * Calculates aggregate counters from checkpoint CHUNK entries. This aggregates the rows processed + * and chunk counts from all chunk entries in the checkpoint table. + * @param entries List of checkpoint entries (both mappers and chunks) + * @return Aggregated counters from chunk entries + */ + private CheckpointAggregateCounters + calculateAggregateCountersFromCheckpoint(List entries) { + long sourceRowsProcessed = 0; + long targetRowsProcessed = 0; + long chunksVerified = 0; + long chunksMismatched = 0; + + for (PhoenixSyncTableOutputRow entry : entries) { + if (PhoenixSyncTableOutputRow.Type.CHUNK.equals(entry.getType())) { + sourceRowsProcessed += entry.getSourceRowsProcessed(); + targetRowsProcessed += entry.getTargetRowsProcessed(); + if (PhoenixSyncTableOutputRow.Status.VERIFIED.equals(entry.getStatus())) { + chunksVerified++; + } else if (PhoenixSyncTableOutputRow.Status.MISMATCHED.equals(entry.getStatus())) { + chunksMismatched++; + } + } + } + + return new CheckpointAggregateCounters(sourceRowsProcessed, targetRowsProcessed, chunksVerified, + chunksMismatched); + } + + private List findChunksBelongingToMapper(Connection conn, + String tableName, String targetCluster, long fromTime, long toTime, + PhoenixSyncTableOutputRow mapper) throws SQLException { + PhoenixSyncTableOutputRepository repository = new PhoenixSyncTableOutputRepository(conn); + return repository.getProcessedChunks(tableName, targetCluster, fromTime, toTime, + mapper.getStartRowKey(), mapper.getEndRowKey()); + } + + /** + * Selects a percentage of chunks to delete from each mapper region. This is used to simulate + * partial rerun scenarios where some checkpoint entries are missing. SyncTableRepository uses + * overlap-based boundary checking, so chunks that span across mapper boundaries may be returned + * by multiple mappers. We use a Set to track unique chunks by their start row key to avoid + * duplicates. + */ + private List selectChunksToDeleteFromMappers(Connection conn, + String tableName, String targetCluster, long fromTime, long toTime, + List mappers, double deletionFraction) throws SQLException { + // Use a map to track unique chunks by start row key to avoid duplicates + Map uniqueChunksToDelete = new LinkedHashMap<>(); + + for (PhoenixSyncTableOutputRow mapper : mappers) { + List mapperChunks = + findChunksBelongingToMapper(conn, tableName, targetCluster, fromTime, toTime, mapper); + + int chunksToDeleteCount = (int) Math.ceil(mapperChunks.size() * deletionFraction); + for (int i = 0; i < chunksToDeleteCount && i < mapperChunks.size(); i++) { + PhoenixSyncTableOutputRow chunk = mapperChunks.get(i); + // Use start row key as unique identifier (convert to string for map key) + String key = + chunk.getStartRowKey() == null ? "NULL" : Bytes.toStringBinary(chunk.getStartRowKey()); + uniqueChunksToDelete.put(key, chunk); + } + } + + return new ArrayList<>(uniqueChunksToDelete.values()); + } + + /** + * Deletes mapper and chunk checkpoint entries to simulate partial rerun scenarios. + * @param conn Connection to use + * @param tableName Table name + * @param targetZkQuorum Target cluster ZK quorum + * @param mappersToDelete List of mapper entries to delete + * @param chunksToDelete List of chunk entries to delete + * @return Total number of entries deleted + */ + private int deleteCheckpointEntries(Connection conn, String tableName, String targetZkQuorum, + List mappersToDelete, List chunksToDelete) + throws SQLException { + int deletedCount = 0; + + // Delete mapper entries + for (PhoenixSyncTableOutputRow mapper : mappersToDelete) { + deletedCount += deleteSingleCheckpointEntry(conn, tableName, targetZkQuorum, + PhoenixSyncTableOutputRow.Type.MAPPER_REGION, mapper.getStartRowKey(), false); + } + + // Delete chunk entries + for (PhoenixSyncTableOutputRow chunk : chunksToDelete) { + deletedCount += deleteSingleCheckpointEntry(conn, tableName, targetZkQuorum, + PhoenixSyncTableOutputRow.Type.CHUNK, chunk.getStartRowKey(), false); + } + + conn.commit(); + return deletedCount; + } + + /** + * Initiates merge of adjacent regions in a table. Merges happen asynchronously in background. + */ + private void mergeAdjacentRegions(Connection conn, String tableName, int mergeCount) { + try { + PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class); + PTable table = pconn.getTable(tableName); + TableName hbaseTableName = TableName.valueOf(table.getPhysicalName().getBytes()); + try (Admin admin = pconn.getQueryServices().getAdmin()) { + // Get current regions + List regions = admin.getRegions(hbaseTableName); + LOGGER.info("Table {} has {} regions before merge", tableName, regions.size()); + + // Merge adjacent region pairs + int mergedCount = 0; + for (int i = 0; i < regions.size() - 1 && mergedCount < mergeCount; i++) { + try { + RegionInfo region1 = regions.get(i); + RegionInfo region2 = regions.get(i + 1); + + LOGGER.info("Initiating merge of regions {} and {}", region1.getEncodedName(), + region2.getEncodedName()); + // Merge regions asynchronously + admin.mergeRegionsAsync(region1.getEncodedNameAsBytes(), + region2.getEncodedNameAsBytes(), false); + mergedCount++; + i++; // Skip next region since it's being merged + } catch (Exception e) { + LOGGER.warn("Failed to merge regions: {}", e.getMessage()); + } + } + + LOGGER.info("Initiated {} region merges for table {}", mergedCount, tableName); + // Wait a bit for merges to start processing + Thread.sleep(1000); + // Get updated region count + List regionsAfter = admin.getRegions(hbaseTableName); + LOGGER.info("Table {} has {} regions after merge attempts", tableName, regionsAfter.size()); + } + } catch (Exception e) { + LOGGER.error("Error during region merge for table {}: {}", tableName, e.getMessage(), e); + } + } + + private void createTableOnBothClusters(Connection sourceConn, Connection targetConn, + String tableName) throws SQLException { + // For 10 rows: split source at 3, 5, 7 creating 4 regions + String sourceDdl = buildStandardTableDdl(tableName, true, "3, 5, 7"); + executeTableCreation(sourceConn, sourceDdl); + + // For target: different split points (2, 4, 6, 8) creating 5 regions + String targetDdl = buildStandardTableDdl(tableName, false, "2, 4, 6, 8"); + executeTableCreation(targetConn, targetDdl); + } + + /** + * Builds DDL for standard test table with common schema. + */ + private String buildStandardTableDdl(String tableName, boolean withReplication, + String splitPoints) { + return String.format( + "CREATE TABLE IF NOT EXISTS %s (\n" + " ID INTEGER NOT NULL PRIMARY KEY,\n" + + " NAME VARCHAR(50),\n" + " NAME_VALUE BIGINT,\n" + " UPDATED_DATE TIMESTAMP\n" + + ") %sUPDATE_CACHE_FREQUENCY=0\n" + "SPLIT ON (%s)", + tableName, withReplication ? "REPLICATION_SCOPE=1, " : "REPLICATION_SCOPE=0,", splitPoints); + } + + /** + * Executes table creation DDL. + */ + private void executeTableCreation(Connection conn, String ddl) throws SQLException { + conn.createStatement().execute(ddl); + conn.commit(); + } + + private void insertTestData(Connection conn, String tableName, int startId, int endId) + throws SQLException { + insertTestData(conn, tableName, startId, endId, System.currentTimeMillis()); + } + + /** + * Waits for HBase replication to complete by polling target cluster. + */ + private void waitForReplication(Connection targetConn, String tableName, int expectedRows) + throws Exception { + long startTime = System.currentTimeMillis(); + // Use NO_INDEX hint to force a full data table scan + String countQuery = "SELECT /*+ NO_INDEX */ COUNT(*) FROM " + tableName; + + while ( + System.currentTimeMillis() - startTime + < (long) PhoenixSyncTableToolIT.REPLICATION_WAIT_TIMEOUT_MS + ) { + ResultSet rs = targetConn.createStatement().executeQuery(countQuery); + rs.next(); + int count = rs.getInt(1); + rs.close(); + + if (count == expectedRows) { + return; + } + } + + fail("Replication timeout: expected " + expectedRows + " rows on target"); + } + + /** + * Verifies that source and target have identical data. + */ + private void verifyDataIdentical(Connection sourceConn, Connection targetConn, String tableName) + throws SQLException { + String query = "SELECT ID, NAME, NAME_VALUE FROM " + tableName + " ORDER BY ID"; + List sourceRows = queryAllRows(sourceConn, query); + List targetRows = queryAllRows(targetConn, query); + + assertEquals("Row counts should match", sourceRows.size(), targetRows.size()); + + for (int i = 0; i < sourceRows.size(); i++) { + assertEquals("Row " + i + " should be identical", sourceRows.get(i), targetRows.get(i)); + } + } + + private void introduceAndVerifyTargetDifferences(String tableName) throws SQLException { + upsertRowsOnTarget(targetConnection, tableName, new int[] { 2, 5, 8 }, + new String[] { "MODIFIED_NAME_2", "MODIFIED_NAME_5", "MODIFIED_NAME_8" }); + + List sourceRows = queryAllRows(sourceConnection, + "SELECT ID, NAME, NAME_VALUE FROM " + tableName + " ORDER BY ID"); + List targetRows = queryAllRows(targetConnection, + "SELECT ID, NAME, NAME_VALUE FROM " + tableName + " ORDER BY ID"); + assertEquals("Row count should be the same", sourceRows.size(), targetRows.size()); + assertNotEquals("Row values should differ after introducing differences", sourceRows, + targetRows); + } + + /** + * Upserts multiple rows on target cluster with specified names. + */ + private void upsertRowsOnTarget(Connection conn, String tableName, int[] ids, String[] names) + throws SQLException { + String upsert = "UPSERT INTO " + tableName + " (ID, NAME) VALUES (?, ?)"; + PreparedStatement stmt = conn.prepareStatement(upsert); + + for (int i = 0; i < ids.length; i++) { + stmt.setInt(1, ids[i]); + stmt.setString(2, names[i]); + stmt.executeUpdate(); + } + + conn.commit(); + } + + /** + * Queries all rows from a table. + */ + private List queryAllRows(Connection conn, String query) throws SQLException { + List rows = new ArrayList<>(); + + try (Statement stmt = conn.createStatement(); ResultSet rs = stmt.executeQuery(query)) { + + while (rs.next()) { + TestRow row = new TestRow(); + row.id = rs.getInt("ID"); + row.name = rs.getString("NAME"); + row.name_value = rs.getLong("NAME_VALUE"); + rows.add(row); + } + } + + return rows; + } + + /** + * Drops a table if it exists. + */ + private void dropTableIfExists(Connection conn, String tableName) { + try { + conn.createStatement().execute("DROP TABLE IF EXISTS " + tableName); + conn.commit(); + } catch (SQLException e) { + LOGGER.warn("Failed to drop table {}: {}", tableName, e.getMessage()); + } + } + + /** + * Creates a multi-tenant salted table with 4 salt buckets on both clusters. + */ + private void createMultiTenantSaltedTableOnBothClusters(Connection sourceConn, + Connection targetConn, String tableName) throws SQLException { + String sourceDdl = buildMultiTenantTableDdl(tableName, true); + executeTableCreation(sourceConn, sourceDdl); + + String targetDdl = buildMultiTenantTableDdl(tableName, false); + executeTableCreation(targetConn, targetDdl); + } + + /** + * Builds DDL for multi-tenant salted table. + */ + private String buildMultiTenantTableDdl(String tableName, boolean withReplication) { + return String.format( + "CREATE TABLE IF NOT EXISTS %s (\n" + " TENANT_ID VARCHAR(15) NOT NULL,\n" + + " ID INTEGER NOT NULL,\n" + " NAME VARCHAR(50),\n" + " NAME_VALUE BIGINT,\n" + + " UPDATED_DATE TIMESTAMP,\n" + " CONSTRAINT PK PRIMARY KEY (TENANT_ID, ID)\n" + + ") MULTI_TENANT=true, SALT_BUCKETS=4, %sUPDATE_CACHE_FREQUENCY=0", + tableName, withReplication ? "REPLICATION_SCOPE=1, " : ""); + } + + /** + * Gets a tenant-specific connection. + */ + private Connection getTenantConnection(Connection baseConnection, String tenantId) + throws SQLException { + String jdbcUrl = baseConnection.unwrap(PhoenixConnection.class).getURL(); + String tenantJdbcUrl = jdbcUrl + ";TenantId=" + tenantId; + return DriverManager.getConnection(tenantJdbcUrl); + } + + /** + * Executes an operation with tenant connections and ensures they are closed. + */ + @FunctionalInterface + private interface TenantConnectionConsumer { + void accept(Connection sourceConn, Connection targetConn) throws SQLException; + } + + private void withTenantConnections(String tenantId, TenantConnectionConsumer consumer) + throws SQLException { + try (Connection tenantSourceConn = getTenantConnection(sourceConnection, tenantId); + Connection tenantTargetConn = getTenantConnection(targetConnection, tenantId)) { + consumer.accept(tenantSourceConn, tenantTargetConn); + } + } + + /** + * Inserts test data for a multi-tenant table using tenant-specific connection. + */ + private void insertMultiTenantTestData(Connection tenantConn, String tableName, int startId, + int endId) throws SQLException { + insertTestData(tenantConn, tableName, startId, endId, System.currentTimeMillis()); + } + + /** + * Introduces differences in the target cluster for multi-tenant table. + */ + private void introduceMultiTenantTargetDifferences(Connection tenantConn, String tableName) + throws SQLException { + upsertRowsOnTarget(tenantConn, tableName, new int[] { 3, 7, 9 }, + new String[] { "MODIFIED_NAME_3", "MODIFIED_NAME_7", "MODIFIED_NAME_9" }); + } + + /** + * Inserts test data with a specific timestamp for time-range testing. Converts range to list and + * delegates to core method. + */ + private void insertTestData(Connection conn, String tableName, int startId, int endId, + long timestamp) throws SQLException { + List ids = new ArrayList<>(); + for (int i = startId; i <= endId; i++) { + ids.add(i); + } + insertTestData(conn, tableName, ids, timestamp); + } + + /** + * Core method: Inserts test data for specific list of IDs with given timestamp. All other + * insertTestData overloads delegate to this method. + */ + private void insertTestData(Connection conn, String tableName, List ids, long timestamp) + throws SQLException { + if (ids == null || ids.isEmpty()) { + return; + } + String upsert = + "UPSERT INTO " + tableName + " (ID, NAME, NAME_VALUE, UPDATED_DATE) VALUES (?, ?, ?, ?)"; + PreparedStatement stmt = conn.prepareStatement(upsert); + Timestamp ts = new Timestamp(timestamp); + for (int id : ids) { + stmt.setInt(1, id); + stmt.setString(2, "NAME_" + id); + stmt.setLong(3, (long) id); + stmt.setTimestamp(4, ts); + stmt.executeUpdate(); + } + conn.commit(); + } + + /** + * Inserts test data for specific list of IDs with current timestamp. + */ + private void insertTestData(Connection conn, String tableName, List ids) + throws SQLException { + insertTestData(conn, tableName, ids, System.currentTimeMillis()); + } + + /** + * Deletes multiple rows from target cluster to create mismatches. This method accepts variable + * number of row IDs to delete. + */ + private void deleteRowsOnTarget(Connection conn, String tableName, int... rowIds) + throws SQLException { + String delete = "DELETE FROM " + tableName + " WHERE ID = ?"; + PreparedStatement stmt = conn.prepareStatement(delete); + + for (int id : rowIds) { + stmt.setInt(1, id); + stmt.executeUpdate(); + } + conn.commit(); + } + + private void deleteHBaseRows(HBaseTestingUtility cluster, String tableName, int rowsToDelete) + throws Exception { + Table table = cluster.getConnection().getTable(TableName.valueOf(tableName)); + ResultScanner scanner = table.getScanner(new Scan()); + List deletes = new ArrayList<>(); + Result result; + int rowsDeleted = 0; + while ((result = scanner.next()) != null && rowsDeleted < rowsToDelete) { + deletes.add(new Delete(result.getRow())); + rowsDeleted++; + } + scanner.close(); + if (!deletes.isEmpty()) { + table.delete(deletes); + } + table.close(); + } + + private int getRowCount(Connection conn, String tableName) throws SQLException { + String countQuery = "SELECT COUNT(*) FROM " + tableName; + Statement stmt = conn.createStatement(); + ResultSet rs = stmt.executeQuery(countQuery); + rs.next(); + int count = rs.getInt(1); + rs.close(); + stmt.close(); + return count; + } + + /** + * Creates an index on both source and target clusters. Note: Indexes inherit replication settings + * from their parent table. + */ + private void createIndexOnBothClusters(Connection sourceConn, Connection targetConn, + String tableName, String indexName) throws SQLException { + // Create index on source (inherits replication from data table) + String indexDdl = String.format( + "CREATE INDEX IF NOT EXISTS %s ON %s (NAME) INCLUDE (NAME_VALUE)", indexName, tableName); + + sourceConn.createStatement().execute(indexDdl); + sourceConn.commit(); + + // Create same index on target + targetConn.createStatement().execute(indexDdl); + targetConn.commit(); + } + + /** + * Creates a local index on both source and target clusters. Note: Local indexes are stored in the + * same regions as the data table and inherit replication settings from their parent table. + */ + private void createLocalIndexOnBothClusters(Connection sourceConn, Connection targetConn, + String tableName, String indexName) throws SQLException { + String indexDdl = + String.format("CREATE LOCAL INDEX IF NOT EXISTS %s ON %s (NAME) INCLUDE (NAME_VALUE)", + indexName, tableName); + + sourceConn.createStatement().execute(indexDdl); + sourceConn.commit(); + + // Create same local index on target + targetConn.createStatement().execute(indexDdl); + targetConn.commit(); + } + + /** + * Attempts to split a table at the specified row ID using HBase Admin API. Ignores errors if the + * split fails (e.g., region in transition). + */ + private void splitTableAt(Connection conn, String tableName, int splitId) { + try { + PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class); + PTable table = pconn.getTable(tableName); + TableName hbaseTableName = TableName.valueOf(table.getPhysicalName().getBytes()); + + byte[] splitPoint = PInteger.INSTANCE.toBytes(splitId); + + // Attempt to split the region at the specified row key + try (Admin admin = pconn.getQueryServices().getAdmin()) { + admin.split(hbaseTableName, splitPoint); + LOGGER.info("Split initiated for table {} at split point {} (bytes: {})", tableName, + splitId, Bytes.toStringBinary(splitPoint)); + } + Thread.sleep(1500); + } catch (Exception e) { + // Ignore split failures - they don't affect the test's main goal + LOGGER.warn("Failed to split table {} at split point {}: {}", tableName, splitId, + e.getMessage()); + } + } + + /** + * Attempts to split a table at multiple split points using HBase Admin API. Ignores errors if any + * split fails (e.g., region in transition). + */ + private void splitTableAt(Connection conn, String tableName, List splitIds) { + if (splitIds == null || splitIds.isEmpty()) { + return; + } + for (int splitId : splitIds) { + splitTableAt(conn, tableName, splitId); + } + } + + /** + * Queries the checkpoint table for entries matching the given table and target cluster. Retrieves + * all columns for comprehensive validation. + */ + private List queryCheckpointTable(Connection conn, String tableName, + String targetCluster) throws SQLException { + List entries = new ArrayList<>(); + String query = "SELECT TABLE_NAME, TARGET_CLUSTER, TYPE, FROM_TIME, TO_TIME, IS_DRY_RUN, " + + "START_ROW_KEY, END_ROW_KEY, EXECUTION_START_TIME, EXECUTION_END_TIME, " + + "STATUS, COUNTERS FROM PHOENIX_SYNC_TABLE_CHECKPOINT " + + "WHERE TABLE_NAME = ? AND TARGET_CLUSTER = ?"; + + PreparedStatement stmt = conn.prepareStatement(query); + stmt.setString(1, tableName); + stmt.setString(2, targetCluster); + ResultSet rs = stmt.executeQuery(); + + while (rs.next()) { + String typeStr = rs.getString("TYPE"); + String statusStr = rs.getString("STATUS"); + + PhoenixSyncTableOutputRow entry = new PhoenixSyncTableOutputRow.Builder() + .setTableName(rs.getString("TABLE_NAME")).setTargetCluster(rs.getString("TARGET_CLUSTER")) + .setType(typeStr != null ? PhoenixSyncTableOutputRow.Type.valueOf(typeStr) : null) + .setFromTime(rs.getLong("FROM_TIME")).setToTime(rs.getLong("TO_TIME")) + .setIsDryRun(rs.getBoolean("IS_DRY_RUN")).setStartRowKey(rs.getBytes("START_ROW_KEY")) + .setEndRowKey(rs.getBytes("END_ROW_KEY")) + .setExecutionStartTime(rs.getTimestamp("EXECUTION_START_TIME")) + .setExecutionEndTime(rs.getTimestamp("EXECUTION_END_TIME")) + .setStatus(statusStr != null ? PhoenixSyncTableOutputRow.Status.valueOf(statusStr) : null) + .setCounters(rs.getString("COUNTERS")).build(); + entries.add(entry); + } + + rs.close(); + return entries; + } + + /** + * Unified method to delete a single checkpoint entry by start row key and optional type. Handles + * NULL/empty start keys for first region boundaries. + * @param conn Connection to use + * @param tableName Table name + * @param targetCluster Target cluster ZK quorum + * @param type Entry type (MAPPER_REGION or CHUNK), or null to delete regardless of type + * @param startRowKey Start row key to match + * @param autoCommit Whether to commit after delete + * @return Number of rows deleted + */ + private int deleteSingleCheckpointEntry(Connection conn, String tableName, String targetCluster, + PhoenixSyncTableOutputRow.Type type, byte[] startRowKey, boolean autoCommit) + throws SQLException { + StringBuilder deleteBuilder = new StringBuilder( + "DELETE FROM PHOENIX_SYNC_TABLE_CHECKPOINT WHERE TABLE_NAME = ? AND TARGET_CLUSTER = ?"); + + // Add TYPE filter if provided + if (type != null) { + deleteBuilder.append(" AND TYPE = ?"); + } + + // Add START_ROW_KEY filter (handle NULL/empty keys) + boolean isNullOrEmptyKey = (startRowKey == null || startRowKey.length == 0); + if (isNullOrEmptyKey) { + // Phoenix stores empty byte arrays as NULL in VARBINARY columns + deleteBuilder.append(" AND START_ROW_KEY IS NULL"); + } else { + deleteBuilder.append(" AND START_ROW_KEY = ?"); + } + + PreparedStatement stmt = conn.prepareStatement(deleteBuilder.toString()); + int paramIndex = 1; + stmt.setString(paramIndex++, tableName); + stmt.setString(paramIndex++, targetCluster); + + if (type != null) { + stmt.setString(paramIndex++, type.name()); + } + + if (!isNullOrEmptyKey) { + stmt.setBytes(paramIndex, startRowKey); + } + + int deleted = stmt.executeUpdate(); + if (autoCommit) { + conn.commit(); + } + return deleted; + } + + /** + * Cleans up checkpoint table entries for a specific table and target cluster. + */ + private void cleanupCheckpointTable(Connection conn, String tableName, String targetCluster) { + try { + String delete = "DELETE FROM PHOENIX_SYNC_TABLE_CHECKPOINT " + + "WHERE TABLE_NAME = ? AND TARGET_CLUSTER = ?"; + PreparedStatement stmt = conn.prepareStatement(delete); + stmt.setString(1, tableName); + stmt.setString(2, targetCluster); + stmt.executeUpdate(); + conn.commit(); + } catch (SQLException e) { + LOGGER.warn("Failed to cleanup checkpoint table: {}", e.getMessage()); + } + } + + /** + * Standard test setup: creates table, inserts data, waits for replication, and verifies. This + * consolidates the repetitive setup pattern used in most tests. + */ + private void setupStandardTestWithReplication(String tableName, int startId, int endId) + throws Exception { + createTableOnBothClusters(sourceConnection, targetConnection, tableName); + insertTestData(sourceConnection, tableName, startId, endId); + int expectedRows = endId - startId + 1; + waitForReplication(targetConnection, tableName, expectedRows); + verifyDataIdentical(sourceConnection, targetConnection, tableName); + } + + /** + * Runs the PhoenixSyncTableTool with standard configuration. Uses chunk size of 1 byte by default + * to create chunks of ~1 row each. Returns the completed Job for counter verification. + */ + private Job runSyncTool(String tableName, String... additionalArgs) throws Exception { + return runSyncToolWithChunkSize(tableName, 1, additionalArgs); + } + + /** + * Runs the PhoenixSyncTableTool with 1KB chunk size for testing multiple rows per chunk. Returns + * the completed Job for counter verification. + */ + private Job runSyncToolWithLargeChunks(String tableName, String... additionalArgs) + throws Exception { + return runSyncToolWithChunkSize(tableName, 1024, additionalArgs); + } + + /** + * Runs the PhoenixSyncTableTool with specified chunk size. Returns the completed Job for counter + * verification. + */ + private Job runSyncToolWithChunkSize(String tableName, int chunkSize, String... additionalArgs) + throws Exception { + Configuration conf = new Configuration(CLUSTERS.getHBaseCluster1().getConfiguration()); + return runSyncToolWithChunkSize(tableName, chunkSize, conf, additionalArgs); + } + + /** + * Runs the PhoenixSyncTableTool with specified chunk size and custom configuration. Allows + * passing pre-configured Configuration object for tests that need specific settings (e.g., paging + * enabled, custom timeouts). + * @param tableName Table name to sync + * @param chunkSize Chunk size in bytes + * @param conf Pre-configured Configuration object + * @param additionalArgs Additional command-line arguments + * @return Completed Job for counter verification + */ + private Job runSyncToolWithChunkSize(String tableName, int chunkSize, Configuration conf, + String... additionalArgs) throws Exception { + // Build args list: start with common args, then add additional ones + List argsList = new ArrayList<>(); + argsList.add("--table-name"); + argsList.add(tableName); + argsList.add("--target-cluster"); + argsList.add(targetZkQuorum); + argsList.add("--run-foreground"); + argsList.add("--chunk-size"); + argsList.add(String.valueOf(chunkSize)); + + // Add any additional args (like --tenant-id, --from-time, etc.) + List additionalArgsList = Arrays.asList(additionalArgs); + argsList.addAll(additionalArgsList); + + // If --to-time is not explicitly provided in additionalArgs, add current time + // This is needed because the default is now (current time - 1 hour) which won't + // capture data inserted immediately before running the sync tool + if (!additionalArgsList.contains("--to-time")) { + argsList.add("--to-time"); + argsList.add(String.valueOf(System.currentTimeMillis())); + } + + String[] args = argsList.toArray(new String[0]); + + PhoenixSyncTableTool tool = new PhoenixSyncTableTool(); + tool.setConf(conf); + int exitCode = tool.run(args); + Job job = tool.getJob(); + + assertNotNull("Job should not be null", job); + assertEquals("Tool should complete successfully", 0, exitCode); + + return job; + } + + /** + * Extracts and returns sync counters from a completed job. + */ + private static class SyncCountersResult { + public final long sourceRowsProcessed; + public final long targetRowsProcessed; + public final long chunksMismatched; + public final long chunksVerified; + + SyncCountersResult(Counters counters) { + this.sourceRowsProcessed = + counters.findCounter(SyncCounters.SOURCE_ROWS_PROCESSED).getValue(); + this.targetRowsProcessed = + counters.findCounter(SyncCounters.TARGET_ROWS_PROCESSED).getValue(); + this.chunksMismatched = counters.findCounter(SyncCounters.CHUNKS_MISMATCHED).getValue(); + this.chunksVerified = counters.findCounter(SyncCounters.CHUNKS_VERIFIED).getValue(); + } + + public void logCounters(String testName) { + LOGGER.info("{}: source rows={}, target rows={}, chunks mismatched={}, chunks verified={}", + testName, sourceRowsProcessed, targetRowsProcessed, chunksMismatched, chunksVerified); + } + } + + /** + * Gets sync counters from job and logs them. + */ + private SyncCountersResult getSyncCounters(Job job) throws IOException { + Counters counters = job.getCounters(); + SyncCountersResult result = new SyncCountersResult(counters); + result.logCounters(testName.getMethodName()); + return result; + } + + private void validateSyncCounters(SyncCountersResult counters, long expectedSourceRows, + long expectedTargetRows, long expectedChunksVerified, long expectedChunksMismatched) { + assertEquals("Should process expected source rows", expectedSourceRows, + counters.sourceRowsProcessed); + assertEquals("Should process expected target rows", expectedTargetRows, + counters.targetRowsProcessed); + assertEquals("Should have expected verified chunks", expectedChunksVerified, + counters.chunksVerified); + assertEquals("Should have expected mismatched chunks", expectedChunksMismatched, + counters.chunksMismatched); + } + + /** + * Validates sync counters with exact source/target rows and minimum chunk thresholds. Use this + * when chunk counts may vary but should be at least certain values. + */ + private void validateSyncCountersExactSourceTarget(SyncCountersResult counters, + long expectedSourceRows, long expectedTargetRows, long minChunksVerified, + long minChunksMismatched) { + assertEquals("Should process expected source rows", expectedSourceRows, + counters.sourceRowsProcessed); + assertEquals("Should process expected target rows", expectedTargetRows, + counters.targetRowsProcessed); + assertTrue(String.format("Should have at least %d verified chunks, actual: %d", + minChunksVerified, counters.chunksVerified), counters.chunksVerified >= minChunksVerified); + assertTrue(String.format("Should have at least %d mismatched chunks, actual: %d", + minChunksMismatched, counters.chunksMismatched), + counters.chunksMismatched >= minChunksMismatched); + } + + /** + * Validates that a checkpoint table has entries with proper structure. + */ + private void validateCheckpointEntries(List entries, + String expectedTableName, String expectedTargetCluster, int expectedSourceRows, + int expectedTargetRows, int expectedChunkVerified, int expectedChunkMismatched, + int expectedMapperRegion, int expectedMapperMismatched) { + int mapperRegionCount = 0; + int chunkCount = 0; + int mismatchedEntry = 0; + int sourceRowsProcessed = 0; + int targetRowsProcessed = 0; + for (PhoenixSyncTableOutputRow entry : entries) { + // Validate primary key columns + assertEquals("TABLE_NAME should match", expectedTableName, entry.getTableName()); + assertEquals("TARGET_CLUSTER should match", expectedTargetCluster, entry.getTargetCluster()); + assertNotNull("TYPE should not be null", entry.getType()); + assertTrue("TYPE should be MAPPER_REGION or CHUNK", + PhoenixSyncTableOutputRow.Type.MAPPER_REGION.equals(entry.getType()) + || PhoenixSyncTableOutputRow.Type.CHUNK.equals(entry.getType())); + + // Validate time range + assertTrue("FROM_TIME should be >= 0", entry.getFromTime() >= 0); + assertTrue("TO_TIME should be > FROM_TIME", entry.getToTime() > entry.getFromTime()); + + // Validate execution timestamps + assertNotNull("EXECUTION_START_TIME should not be null", entry.getExecutionStartTime()); + assertNotNull("EXECUTION_END_TIME should not be null", entry.getExecutionEndTime()); + assertTrue("EXECUTION_END_TIME should be >= EXECUTION_START_TIME", + entry.getExecutionEndTime().getTime() >= entry.getExecutionStartTime().getTime()); + + // Validate status + assertNotNull("STATUS should not be null", entry.getStatus()); + assertTrue("STATUS should be VERIFIED or MISMATCHED", + PhoenixSyncTableOutputRow.Status.VERIFIED.equals(entry.getStatus()) + || PhoenixSyncTableOutputRow.Status.MISMATCHED.equals(entry.getStatus())); + + if (PhoenixSyncTableOutputRow.Status.MISMATCHED.equals(entry.getStatus())) { + mismatchedEntry++; + } + + // Count entry types + if (PhoenixSyncTableOutputRow.Type.MAPPER_REGION.equals(entry.getType())) { + mapperRegionCount++; + sourceRowsProcessed += (int) entry.getSourceRowsProcessed(); + targetRowsProcessed += (int) entry.getTargetRowsProcessed(); + } else if (PhoenixSyncTableOutputRow.Type.CHUNK.equals(entry.getType())) { + chunkCount++; + assertNotNull("COUNTERS should not be null for CHUNK entries", entry.getCounters()); + } + } + + assertEquals(String.format("Should have %d MAPPER_REGION entry", expectedMapperRegion), + expectedMapperMismatched, expectedMapperRegion, mapperRegionCount); + assertEquals( + String.format("Should have %d CHUNK entry", expectedChunkVerified + expectedChunkMismatched), + expectedChunkVerified + expectedChunkMismatched, chunkCount); + assertEquals( + String.format("Should have %d MISMATCHED entry", + expectedMapperMismatched + expectedChunkMismatched), + expectedMapperMismatched + expectedChunkMismatched, mismatchedEntry); + assertEquals(String.format("Should have %d Source rows processed", expectedSourceRows), + expectedSourceRows, sourceRowsProcessed); + assertEquals(String.format("Should have %d Target rows processed", expectedTargetRows), + expectedTargetRows, targetRowsProcessed); + } + + private void disableReplication(Connection conn, String tableName) throws Exception { + PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class); + PTable table = pconn.getTable(tableName); + TableName hbaseTableName = TableName.valueOf(table.getPhysicalName().getBytes()); + + try (Admin admin = pconn.getQueryServices().getAdmin()) { + // Disable table first + admin.disableTable(hbaseTableName); + + // Modify table descriptor to disable replication + admin.modifyTable(TableDescriptorBuilder.newBuilder(admin.getDescriptor(hbaseTableName)) + .setRegionReplication(1).build()); + + // Re-enable table + admin.enableTable(hbaseTableName); + } + } + + /** + * Data class to hold test table data + */ + private static class TestRow { + int id; + String name; + long name_value; + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (!(o instanceof TestRow)) return false; + TestRow other = (TestRow) o; + return id == other.id && Objects.equals(name, other.name) && name_value == other.name_value; + } + } +} 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; diff --git a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/PhoenixNoOpSingleRecordReaderTest.java b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/PhoenixNoOpSingleRecordReaderTest.java new file mode 100644 index 00000000000..63933a4445e --- /dev/null +++ b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/PhoenixNoOpSingleRecordReaderTest.java @@ -0,0 +1,72 @@ +/* + * 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.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapreduce.lib.db.DBWritable; +import org.junit.Before; +import org.junit.Test; + +public class PhoenixNoOpSingleRecordReaderTest { + + private PhoenixNoOpSingleRecordReader reader; + + @Before + public void setup() { + reader = new PhoenixNoOpSingleRecordReader(); + } + + @Test + public void testNextKeyValueReturnsTrueExactlyOnce() { + assertTrue("First call should return true", reader.nextKeyValue()); + assertFalse("Second call should return false", reader.nextKeyValue()); + assertFalse("Third call should still return false", reader.nextKeyValue()); + } + + @Test + public void testGetCurrentKeyReturnsNullWritable() { + NullWritable key = reader.getCurrentKey(); + assertNotNull(key); + assertEquals(NullWritable.get(), key); + } + + @Test + public void testGetCurrentValueReturnsNullDBWritable() { + DBWritable value = reader.getCurrentValue(); + assertNotNull(value); + } + + @Test + public void testProgressReflectsRecordConsumption() { + assertEquals("Progress should be 0.0 before consuming record", 0.0f, reader.getProgress(), + 0.0f); + reader.nextKeyValue(); + assertEquals("Progress should be 1.0 after consuming record", 1.0f, reader.getProgress(), 0.0f); + } + + @Test + public void testInitializeAndCloseDoNotThrow() { + reader.initialize(null, null); + reader.close(); + } +} diff --git a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormatTest.java b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormatTest.java new file mode 100644 index 00000000000..15e643feaf0 --- /dev/null +++ b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/PhoenixSyncTableInputFormatTest.java @@ -0,0 +1,283 @@ +/* + * 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.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.phoenix.query.KeyRange; +import org.junit.Test; + +/** + * Unit tests for PhoenixSyncTableInputFormat. Tests various scenarios of filtering completed splits + */ +public class PhoenixSyncTableInputFormatTest { + + private PhoenixSyncTableInputFormat inputFormat = new PhoenixSyncTableInputFormat(); + + /** + * Helper method to create a PhoenixInputSplit with given key range boundaries. + */ + private PhoenixInputSplit createSplit(byte[] start, byte[] end) { + Scan scan = new Scan(); + scan.withStartRow(start, true); + scan.withStopRow(end, false); + return new PhoenixInputSplit(Collections.singletonList(scan)); + } + + /** + * Helper method to create a KeyRange with given boundaries. + */ + private KeyRange createKeyRange(byte[] start, byte[] end) { + return KeyRange.getKeyRange(start, true, end, false); + } + + @Test + public void testNoCompletedRegions() { + List allSplits = new ArrayList<>(); + allSplits.add(createSplit(Bytes.toBytes("a"), Bytes.toBytes("d"))); + allSplits.add(createSplit(Bytes.toBytes("d"), Bytes.toBytes("g"))); + allSplits.add(createSplit(Bytes.toBytes("g"), Bytes.toBytes("j"))); + + List completedRegions = new ArrayList<>(); + + List result = inputFormat.filterCompletedSplits(allSplits, completedRegions); + + assertEquals("All splits should be unprocessed when no completed regions", 3, result.size()); + assertEquals(allSplits, result); + } + + @Test + public void testAllRegionsCompleted() { + List allSplits = new ArrayList<>(); + allSplits.add(createSplit(Bytes.toBytes("a"), Bytes.toBytes("d"))); + allSplits.add(createSplit(Bytes.toBytes("d"), Bytes.toBytes("g"))); + allSplits.add(createSplit(Bytes.toBytes("g"), Bytes.toBytes("j"))); + + List completedRegions = new ArrayList<>(); + completedRegions.add(createKeyRange(Bytes.toBytes("a"), Bytes.toBytes("d"))); + completedRegions.add(createKeyRange(Bytes.toBytes("d"), Bytes.toBytes("g"))); + completedRegions.add(createKeyRange(Bytes.toBytes("g"), Bytes.toBytes("j"))); + + List result = inputFormat.filterCompletedSplits(allSplits, completedRegions); + + assertEquals("No splits should be unprocessed when all regions completed", 0, result.size()); + } + + @Test + public void testPartiallyCompletedRegions() { + // Scenario: Some regions completed, some pending + // Splits: [a,d), [d,g), [g,j) + // Completed: [a,d), [g,j) + // Expected unprocessed: [d,g) + List allSplits = new ArrayList<>(); + allSplits.add(createSplit(Bytes.toBytes("a"), Bytes.toBytes("d"))); + allSplits.add(createSplit(Bytes.toBytes("d"), Bytes.toBytes("g"))); + allSplits.add(createSplit(Bytes.toBytes("g"), Bytes.toBytes("j"))); + + List completedRegions = new ArrayList<>(); + completedRegions.add(createKeyRange(Bytes.toBytes("a"), Bytes.toBytes("d"))); + completedRegions.add(createKeyRange(Bytes.toBytes("g"), Bytes.toBytes("j"))); + + List result = inputFormat.filterCompletedSplits(allSplits, completedRegions); + + assertEquals("Only middle split should be unprocessed", 1, result.size()); + PhoenixInputSplit unprocessed = (PhoenixInputSplit) result.get(0); + assertTrue("Should be [d,g) split", + Bytes.equals(Bytes.toBytes("d"), unprocessed.getKeyRange().getLowerRange())); + assertTrue("Should be [d,g) split", + Bytes.equals(Bytes.toBytes("g"), unprocessed.getKeyRange().getUpperRange())); + } + + @Test + public void testSplitExtendsAcrossCompleted() { + // Scenario: Split extends beyond completed region on both sides + // Split: [a,k) + // Completed: [c,g) + // Expected unprocessed: [a,k) (completed is inside split, not fully contained) + List allSplits = new ArrayList<>(); + allSplits.add(createSplit(Bytes.toBytes("a"), Bytes.toBytes("k"))); + + List completedRegions = new ArrayList<>(); + completedRegions.add(createKeyRange(Bytes.toBytes("c"), Bytes.toBytes("g"))); + + List result = inputFormat.filterCompletedSplits(allSplits, completedRegions); + + assertEquals("Split should be unprocessed when completed is inside split", 1, result.size()); + } + + @Test + public void testExactMatchSplitAndCompleted() { + // Scenario: Split exactly matches completed region + // Split: [a,d) + // Completed: [a,d) + // Expected unprocessed: none + List allSplits = new ArrayList<>(); + allSplits.add(createSplit(Bytes.toBytes("a"), Bytes.toBytes("d"))); + + List completedRegions = new ArrayList<>(); + completedRegions.add(createKeyRange(Bytes.toBytes("a"), Bytes.toBytes("d"))); + + List result = inputFormat.filterCompletedSplits(allSplits, completedRegions); + + assertEquals("Split should be filtered out when it exactly matches completed region", 0, + result.size()); + } + + @Test + public void testLastRegionWithEmptyEndRow() { + // Scenario: Last region with empty end row that partially overlaps with a middle split + // Splits: [a,d), [d,g), [g,[]) + // Completed: [f,[]) - fully contains [g,[]) and partially overlaps [d,g) + // Expected unprocessed: [a,d), [d,g) - partial overlap means split is NOT filtered + List allSplits = new ArrayList<>(); + allSplits.add(createSplit(Bytes.toBytes("a"), Bytes.toBytes("d"))); + allSplits.add(createSplit(Bytes.toBytes("d"), Bytes.toBytes("g"))); + allSplits.add(createSplit(Bytes.toBytes("g"), HConstants.EMPTY_END_ROW)); + + List completedRegions = new ArrayList<>(); + completedRegions.add(createKeyRange(Bytes.toBytes("f"), HConstants.EMPTY_END_ROW)); + + List result = inputFormat.filterCompletedSplits(allSplits, completedRegions); + + assertEquals( + "First two splits should be unprocessed (partial overlap keeps split), last should be filtered", + 2, result.size()); + PhoenixInputSplit first = (PhoenixInputSplit) result.get(0); + PhoenixInputSplit second = (PhoenixInputSplit) result.get(1); + assertTrue("First should be [a,d) split", + Bytes.equals(Bytes.toBytes("a"), first.getKeyRange().getLowerRange())); + assertTrue("Second should be [d,g) split", + Bytes.equals(Bytes.toBytes("d"), second.getKeyRange().getLowerRange())); + } + + @Test + public void testCompletedRegionCoversMultipleSplits() { + // Scenario: One completed region covers multiple splits + // Splits: [a,c), [c,e), [e,g) + // Completed: [a,g) + // Expected unprocessed: none + List allSplits = new ArrayList<>(); + allSplits.add(createSplit(Bytes.toBytes("a"), Bytes.toBytes("c"))); + allSplits.add(createSplit(Bytes.toBytes("c"), Bytes.toBytes("e"))); + allSplits.add(createSplit(Bytes.toBytes("e"), Bytes.toBytes("g"))); + + List completedRegions = new ArrayList<>(); + completedRegions.add(createKeyRange(Bytes.toBytes("a"), Bytes.toBytes("g"))); + + List result = inputFormat.filterCompletedSplits(allSplits, completedRegions); + + assertEquals("All splits should be filtered when covered by one large completed region", 0, + result.size()); + } + + @Test + public void testInterleavedCompletedAndUnprocessed() { + // Scenario: Completed and unprocessed regions interleaved + // Splits: [a,c), [c,e), [e,g), [g,i), [i,k) + // Completed: [a,c), [e,g), [i,k) + // Expected unprocessed: [c,e), [g,i) + List allSplits = new ArrayList<>(); + allSplits.add(createSplit(Bytes.toBytes("a"), Bytes.toBytes("c"))); + allSplits.add(createSplit(Bytes.toBytes("c"), Bytes.toBytes("e"))); + allSplits.add(createSplit(Bytes.toBytes("e"), Bytes.toBytes("g"))); + allSplits.add(createSplit(Bytes.toBytes("g"), Bytes.toBytes("i"))); + allSplits.add(createSplit(Bytes.toBytes("i"), Bytes.toBytes("k"))); + + List completedRegions = new ArrayList<>(); + completedRegions.add(createKeyRange(Bytes.toBytes("a"), Bytes.toBytes("c"))); + completedRegions.add(createKeyRange(Bytes.toBytes("e"), Bytes.toBytes("g"))); + completedRegions.add(createKeyRange(Bytes.toBytes("i"), Bytes.toBytes("k"))); + + List result = inputFormat.filterCompletedSplits(allSplits, completedRegions); + + assertEquals("Should have 2 unprocessed splits", 2, result.size()); + PhoenixInputSplit split1 = (PhoenixInputSplit) result.get(0); + PhoenixInputSplit split2 = (PhoenixInputSplit) result.get(1); + + assertTrue("First unprocessed should be [c,e)", + Bytes.equals(Bytes.toBytes("c"), split1.getKeyRange().getLowerRange())); + assertTrue("Second unprocessed should be [g,i)", + Bytes.equals(Bytes.toBytes("g"), split2.getKeyRange().getLowerRange())); + } + + @Test + public void testEmptyStartRow() { + // Scenario: First region with empty start row + // Splits: [[],c), [c,f) + // Completed: [[],c) + // Expected unprocessed: [c,f) + List allSplits = new ArrayList<>(); + allSplits.add(createSplit(HConstants.EMPTY_START_ROW, Bytes.toBytes("c"))); + allSplits.add(createSplit(Bytes.toBytes("c"), Bytes.toBytes("f"))); + + List completedRegions = new ArrayList<>(); + completedRegions.add(createKeyRange(HConstants.EMPTY_START_ROW, Bytes.toBytes("c"))); + + List result = inputFormat.filterCompletedSplits(allSplits, completedRegions); + + assertEquals("Second split should be unprocessed", 1, result.size()); + PhoenixInputSplit unprocessed = (PhoenixInputSplit) result.get(0); + assertTrue("Should be [c,f) split", + Bytes.equals(Bytes.toBytes("c"), unprocessed.getKeyRange().getLowerRange())); + } + + @Test + public void testUnsortedInputSplits() { + // Scenario: Verify that input splits are sorted before processing + // Splits (unsorted): [g,j), [a,d), [d,g) + // Completed: [a,d) + // Expected unprocessed: [d,g), [g,j) (after sorting) + List allSplits = new ArrayList<>(); + allSplits.add(createSplit(Bytes.toBytes("g"), Bytes.toBytes("j"))); // out of order + allSplits.add(createSplit(Bytes.toBytes("a"), Bytes.toBytes("d"))); + allSplits.add(createSplit(Bytes.toBytes("d"), Bytes.toBytes("g"))); + + List completedRegions = new ArrayList<>(); + completedRegions.add(createKeyRange(Bytes.toBytes("a"), Bytes.toBytes("d"))); + + List result = inputFormat.filterCompletedSplits(allSplits, completedRegions); + + assertEquals("Should have 2 unprocessed splits after sorting", 2, result.size()); + // Verify sorted order + PhoenixInputSplit split1 = (PhoenixInputSplit) result.get(0); + PhoenixInputSplit split2 = (PhoenixInputSplit) result.get(1); + + assertTrue("First should be [d,g)", + Bytes.compareTo(split1.getKeyRange().getLowerRange(), split2.getKeyRange().getLowerRange()) + < 0); + } + + @SuppressWarnings("rawtypes") + @Test + public void testCreateRecordReaderReturnsNoOpReader() { + RecordReader reader = inputFormat.createRecordReader(null, null); + assertNotNull("createRecordReader should never return null", reader); + assertTrue("Should return a PhoenixNoOpSingleRecordReader", + reader instanceof PhoenixNoOpSingleRecordReader); + } +} diff --git a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapperTest.java b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapperTest.java new file mode 100644 index 00000000000..28024f698dd --- /dev/null +++ b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/PhoenixSyncTableMapperTest.java @@ -0,0 +1,514 @@ +/* + * 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.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Pair; +import org.junit.Before; +import org.junit.Test; + +/** + * Unit tests for PhoenixSyncTableMapper. + */ +public class PhoenixSyncTableMapperTest { + + private PhoenixSyncTableMapper mapper; + + @Before + public void setup() throws Exception { + mapper = new PhoenixSyncTableMapper(); + } + + /** + * Helper method to create a PhoenixSyncTableOutputRow with start and end keys. + */ + private PhoenixSyncTableOutputRow createChunk(byte[] startKey, byte[] endKey) { + return new PhoenixSyncTableOutputRow.Builder().setStartRowKey(startKey).setEndRowKey(endKey) + .build(); + } + + /** + * Helper method to verify a gap/range matches expected values. + */ + private void assertGap(Pair gap, byte[] expectedStart, byte[] expectedEnd) { + assertArrayEquals("Gap start key mismatch", expectedStart, gap.getFirst()); + assertArrayEquals("Gap end key mismatch", expectedEnd, gap.getSecond()); + } + + @Test + public void testNullProcessedChunks() { + // Scenario: No processed chunks (null), entire region should be unprocessed + byte[] regionStart = Bytes.toBytes("a"); + byte[] regionEnd = Bytes.toBytes("z"); + + List> result = + mapper.calculateUnprocessedRanges(regionStart, regionEnd, null); + + assertEquals("Should have 1 unprocessed range when no chunks processed", 1, result.size()); + assertGap(result.get(0), regionStart, regionEnd); + } + + @Test + public void testEmptyProcessedChunks() { + // Scenario: Empty processed chunks list, entire region should be unprocessed + byte[] regionStart = Bytes.toBytes("a"); + byte[] regionEnd = Bytes.toBytes("z"); + List processedChunks = new ArrayList<>(); + + List> result = + mapper.calculateUnprocessedRanges(regionStart, regionEnd, processedChunks); + + assertEquals("Should have 1 unprocessed range when chunks list is empty", 1, result.size()); + assertGap(result.get(0), regionStart, regionEnd); + } + + @Test + public void testFullyCoveredRegion() { + // Scenario: Entire region covered by processed chunks, no gaps + // Region: [a, z) + // Chunks: [a, d], [d, g], [g, z] + // Expected: No unprocessed ranges + byte[] regionStart = Bytes.toBytes("a"); + byte[] regionEnd = Bytes.toBytes("z"); + List processedChunks = new ArrayList<>(); + processedChunks.add(createChunk(Bytes.toBytes("a"), Bytes.toBytes("d"))); + processedChunks.add(createChunk(Bytes.toBytes("d"), Bytes.toBytes("g"))); + processedChunks.add(createChunk(Bytes.toBytes("g"), Bytes.toBytes("z"))); + + List> result = + mapper.calculateUnprocessedRanges(regionStart, regionEnd, processedChunks); + + assertEquals("Should have no unprocessed ranges when region fully covered", 0, result.size()); + } + + @Test + public void testGapAtStart() { + // Scenario: Gap at the start of the region + // Region: [a, z) + // Chunks: [e, z] + // Expected: [a, e) + byte[] regionStart = Bytes.toBytes("a"); + byte[] regionEnd = Bytes.toBytes("z"); + List processedChunks = new ArrayList<>(); + processedChunks.add(createChunk(Bytes.toBytes("e"), Bytes.toBytes("z"))); + + List> result = + mapper.calculateUnprocessedRanges(regionStart, regionEnd, processedChunks); + + assertEquals("Should have 1 unprocessed range at start", 1, result.size()); + assertGap(result.get(0), Bytes.toBytes("a"), Bytes.toBytes("e")); + } + + @Test + public void testGapAtEnd() { + // Scenario: Gap at the end of the region + // Region: [a, z) + // Chunks: [a, e] + // Expected: [e, z) + byte[] regionStart = Bytes.toBytes("a"); + byte[] regionEnd = Bytes.toBytes("z"); + List processedChunks = new ArrayList<>(); + processedChunks.add(createChunk(Bytes.toBytes("a"), Bytes.toBytes("e"))); + + List> result = + mapper.calculateUnprocessedRanges(regionStart, regionEnd, processedChunks); + + assertEquals("Should have 1 unprocessed range at end", 1, result.size()); + assertGap(result.get(0), Bytes.toBytes("e"), Bytes.toBytes("z")); + } + + @Test + public void testGapInMiddle() { + // Scenario: Gap in the middle between two chunks + // Region: [a, z) + // Chunks: [a, d], [g, z] + // Expected: [d, g) + byte[] regionStart = Bytes.toBytes("a"); + byte[] regionEnd = Bytes.toBytes("z"); + List processedChunks = new ArrayList<>(); + processedChunks.add(createChunk(Bytes.toBytes("a"), Bytes.toBytes("d"))); + processedChunks.add(createChunk(Bytes.toBytes("g"), Bytes.toBytes("z"))); + + List> result = + mapper.calculateUnprocessedRanges(regionStart, regionEnd, processedChunks); + + assertEquals("Should have 1 unprocessed range in middle", 1, result.size()); + assertGap(result.get(0), Bytes.toBytes("d"), Bytes.toBytes("g")); + } + + @Test + public void testMultipleGaps() { + // Scenario: Multiple gaps throughout the region + // Region: [a, z) + // Chunks: [b, d], [f, h], [j, l] + // Expected: [a, b), [d, f), [h, j), [l, z) + byte[] regionStart = Bytes.toBytes("a"); + byte[] regionEnd = Bytes.toBytes("z"); + List processedChunks = new ArrayList<>(); + processedChunks.add(createChunk(Bytes.toBytes("b"), Bytes.toBytes("d"))); + processedChunks.add(createChunk(Bytes.toBytes("f"), Bytes.toBytes("h"))); + processedChunks.add(createChunk(Bytes.toBytes("j"), Bytes.toBytes("l"))); + + List> result = + mapper.calculateUnprocessedRanges(regionStart, regionEnd, processedChunks); + + assertEquals("Should have 4 unprocessed ranges", 4, result.size()); + assertGap(result.get(0), Bytes.toBytes("a"), Bytes.toBytes("b")); + assertGap(result.get(1), Bytes.toBytes("d"), Bytes.toBytes("f")); + assertGap(result.get(2), Bytes.toBytes("h"), Bytes.toBytes("j")); + assertGap(result.get(3), Bytes.toBytes("l"), Bytes.toBytes("z")); + } + + @Test + public void testChunkStartsBeforeRegion() { + // Scenario: First chunk starts before mapper region (clipping needed) + // Region: [d, z) + // Chunks: [a, g], [g, z] + // Expected: No gaps (chunk clipped to [d, g]) + byte[] regionStart = Bytes.toBytes("d"); + byte[] regionEnd = Bytes.toBytes("z"); + List processedChunks = new ArrayList<>(); + processedChunks.add(createChunk(Bytes.toBytes("a"), Bytes.toBytes("g"))); + processedChunks.add(createChunk(Bytes.toBytes("g"), Bytes.toBytes("z"))); + + List> result = + mapper.calculateUnprocessedRanges(regionStart, regionEnd, processedChunks); + + assertEquals("Should have no unprocessed ranges after clipping", 0, result.size()); + } + + @Test + public void testChunkEndsAfterRegion() { + // Scenario: Last chunk ends after mapper region (clipping needed) + // Region: [a, m) + // Chunks: [a, g], [g, z] + // Expected: No gaps (last chunk clipped to [g, m)) + byte[] regionStart = Bytes.toBytes("a"); + byte[] regionEnd = Bytes.toBytes("m"); + List processedChunks = new ArrayList<>(); + processedChunks.add(createChunk(Bytes.toBytes("a"), Bytes.toBytes("g"))); + processedChunks.add(createChunk(Bytes.toBytes("g"), Bytes.toBytes("z"))); + + List> result = + mapper.calculateUnprocessedRanges(regionStart, regionEnd, processedChunks); + + assertEquals("Should have no unprocessed ranges after clipping", 0, result.size()); + } + + @Test + public void testChunkBothSidesOutsideRegion() { + // Scenario: Chunk starts before and ends after region + // Region: [d, m) + // Chunks: [a, z] + // Expected: No gaps (chunk covers entire region) + byte[] regionStart = Bytes.toBytes("d"); + byte[] regionEnd = Bytes.toBytes("m"); + List processedChunks = new ArrayList<>(); + processedChunks.add(createChunk(Bytes.toBytes("a"), Bytes.toBytes("z"))); + + List> result = + mapper.calculateUnprocessedRanges(regionStart, regionEnd, processedChunks); + + assertEquals("Should have no unprocessed ranges when chunk covers entire region", 0, + result.size()); + } + + @Test + public void testFirstRegionWithEmptyStartKey() { + // Scenario: First region of table with empty start key [] + // Region: [[], d) + // Chunks: [a, d] + // Expected: [[], a) gap at start, no gap at end since chunk ends at region boundary + byte[] regionStart = HConstants.EMPTY_START_ROW; + byte[] regionEnd = Bytes.toBytes("d"); + List processedChunks = new ArrayList<>(); + processedChunks.add(createChunk(Bytes.toBytes("a"), Bytes.toBytes("d"))); + + List> result = + mapper.calculateUnprocessedRanges(regionStart, regionEnd, processedChunks); + + assertEquals("Should have 1 unprocessed range at start", 1, result.size()); + assertGap(result.get(0), HConstants.EMPTY_START_ROW, Bytes.toBytes("a")); + } + + @Test + public void testFirstRegionWithGapAtStart() { + // Scenario: First region with gap at start + // Region: [[], d) + // Chunks: [b, d] + // Expected: [[], b) + byte[] regionStart = HConstants.EMPTY_START_ROW; + byte[] regionEnd = Bytes.toBytes("d"); + List processedChunks = new ArrayList<>(); + processedChunks.add(createChunk(Bytes.toBytes("b"), Bytes.toBytes("d"))); + + List> result = + mapper.calculateUnprocessedRanges(regionStart, regionEnd, processedChunks); + + assertEquals("Should have 1 unprocessed range at start of first region", 1, result.size()); + assertGap(result.get(0), HConstants.EMPTY_START_ROW, Bytes.toBytes("b")); + } + + @Test + public void testLastRegionWithEmptyEndKey() { + // Scenario: Last region of table with empty end key [] + // Region: [v, []) + // Chunks: [v, z] + // Expected: [z, []) + byte[] regionStart = Bytes.toBytes("v"); + byte[] regionEnd = HConstants.EMPTY_END_ROW; + List processedChunks = new ArrayList<>(); + processedChunks.add(createChunk(Bytes.toBytes("v"), Bytes.toBytes("z"))); + + List> result = + mapper.calculateUnprocessedRanges(regionStart, regionEnd, processedChunks); + + assertEquals("Should have 1 unprocessed range at end of last region", 1, result.size()); + assertGap(result.get(0), Bytes.toBytes("z"), HConstants.EMPTY_END_ROW); + } + + @Test + public void testLastRegionFullyCovered() { + // Scenario: Last region fully covered + // Region: [v, []) + // Chunks: [v, x], [x, []) + // Expected: No gaps (but will add [[], []) due to isEndRegionOfTable logic) + byte[] regionStart = Bytes.toBytes("v"); + byte[] regionEnd = HConstants.EMPTY_END_ROW; + List processedChunks = new ArrayList<>(); + processedChunks.add(createChunk(Bytes.toBytes("v"), Bytes.toBytes("x"))); + processedChunks.add(createChunk(Bytes.toBytes("x"), HConstants.EMPTY_END_ROW)); + + List> result = + mapper.calculateUnprocessedRanges(regionStart, regionEnd, processedChunks); + + // Due to isEndRegionOfTable check, we always add remaining range + assertEquals("Last region should have 1 range added", 1, result.size()); + assertGap(result.get(0), HConstants.EMPTY_END_ROW, HConstants.EMPTY_END_ROW); + } + + @Test + public void testSingleRegionTable() { + // Scenario: Single region table (entire table) + // Region: [[], []) + // Chunks: [a, m] + // Expected: [[], a), [m, []) + byte[] regionStart = HConstants.EMPTY_START_ROW; + byte[] regionEnd = HConstants.EMPTY_END_ROW; + List processedChunks = new ArrayList<>(); + processedChunks.add(createChunk(Bytes.toBytes("a"), Bytes.toBytes("m"))); + + List> result = + mapper.calculateUnprocessedRanges(regionStart, regionEnd, processedChunks); + + assertEquals("Should have 2 unprocessed ranges for single region table", 2, result.size()); + assertGap(result.get(0), HConstants.EMPTY_START_ROW, Bytes.toBytes("a")); + assertGap(result.get(1), Bytes.toBytes("m"), HConstants.EMPTY_END_ROW); + } + + @Test + public void testSingleChunkInMiddleOfRegion() { + // Scenario: Single chunk in middle of region + // Region: [a, z) + // Chunks: [e, m] + // Expected: [a, e), [m, z) + byte[] regionStart = Bytes.toBytes("a"); + byte[] regionEnd = Bytes.toBytes("z"); + List processedChunks = new ArrayList<>(); + processedChunks.add(createChunk(Bytes.toBytes("e"), Bytes.toBytes("m"))); + + List> result = + mapper.calculateUnprocessedRanges(regionStart, regionEnd, processedChunks); + + assertEquals("Should have 2 unprocessed ranges", 2, result.size()); + assertGap(result.get(0), Bytes.toBytes("a"), Bytes.toBytes("e")); + assertGap(result.get(1), Bytes.toBytes("m"), Bytes.toBytes("z")); + } + + @Test + public void testAdjacentChunksWithNoGaps() { + // Scenario: Perfectly adjacent chunks with no gaps + // Region: [a, z) + // Chunks: [a, c], [c, f], [f, j], [j, z] + // Expected: No gaps + byte[] regionStart = Bytes.toBytes("a"); + byte[] regionEnd = Bytes.toBytes("z"); + List processedChunks = new ArrayList<>(); + processedChunks.add(createChunk(Bytes.toBytes("a"), Bytes.toBytes("c"))); + processedChunks.add(createChunk(Bytes.toBytes("c"), Bytes.toBytes("f"))); + processedChunks.add(createChunk(Bytes.toBytes("f"), Bytes.toBytes("j"))); + processedChunks.add(createChunk(Bytes.toBytes("j"), Bytes.toBytes("z"))); + + List> result = + mapper.calculateUnprocessedRanges(regionStart, regionEnd, processedChunks); + + assertEquals("Should have no unprocessed ranges for perfectly adjacent chunks", 0, + result.size()); + } + + @Test + public void testSmallGapsBetweenManyChunks() { + // Scenario: Many chunks with small gaps between them + // Region: [a, z) + // Chunks: [a, b], [c, d], [e, f], [g, h] + // Expected: [b, c), [d, e), [f, g), [h, z) + byte[] regionStart = Bytes.toBytes("a"); + byte[] regionEnd = Bytes.toBytes("z"); + List processedChunks = new ArrayList<>(); + processedChunks.add(createChunk(Bytes.toBytes("a"), Bytes.toBytes("b"))); + processedChunks.add(createChunk(Bytes.toBytes("c"), Bytes.toBytes("d"))); + processedChunks.add(createChunk(Bytes.toBytes("e"), Bytes.toBytes("f"))); + processedChunks.add(createChunk(Bytes.toBytes("g"), Bytes.toBytes("h"))); + + List> result = + mapper.calculateUnprocessedRanges(regionStart, regionEnd, processedChunks); + + assertEquals("Should have 4 unprocessed ranges", 4, result.size()); + assertGap(result.get(0), Bytes.toBytes("b"), Bytes.toBytes("c")); + assertGap(result.get(1), Bytes.toBytes("d"), Bytes.toBytes("e")); + assertGap(result.get(2), Bytes.toBytes("f"), Bytes.toBytes("g")); + assertGap(result.get(3), Bytes.toBytes("h"), Bytes.toBytes("z")); + } + + @Test + public void testChunkExactlyMatchesRegion() { + // Scenario: Single chunk exactly matches region boundaries + // Region: [a, z) + // Chunks: [a, z] + // Expected: No gaps + byte[] regionStart = Bytes.toBytes("a"); + byte[] regionEnd = Bytes.toBytes("z"); + List processedChunks = new ArrayList<>(); + processedChunks.add(createChunk(Bytes.toBytes("a"), Bytes.toBytes("z"))); + + List> result = + mapper.calculateUnprocessedRanges(regionStart, regionEnd, processedChunks); + + assertEquals("Should have no unprocessed ranges when chunk matches region", 0, result.size()); + } + + @Test + public void testRegionBoundaryChangeScenario() { + // Scenario: Region boundaries changed after split/merge + // New Region: [d, r) + // Old processed chunks: [a, g], [j, m], [s, v] + // First chunk [a, g] starts before region, clipped to [d, g] + // Second chunk [j, m] is within region + // Third chunk [s, v] starts after region end 'r', effectiveStart becomes 's' + // scanPos is updated to 's', making the final gap [m, s) + // Expected: [g, j), [m, s) + byte[] regionStart = Bytes.toBytes("d"); + byte[] regionEnd = Bytes.toBytes("r"); + List processedChunks = new ArrayList<>(); + processedChunks.add(createChunk(Bytes.toBytes("a"), Bytes.toBytes("g"))); + processedChunks.add(createChunk(Bytes.toBytes("j"), Bytes.toBytes("m"))); + processedChunks.add(createChunk(Bytes.toBytes("s"), Bytes.toBytes("v"))); + + List> result = + mapper.calculateUnprocessedRanges(regionStart, regionEnd, processedChunks); + + assertEquals("Should have 2 unprocessed ranges after region boundary change", 2, result.size()); + assertGap(result.get(0), Bytes.toBytes("g"), Bytes.toBytes("j")); + assertGap(result.get(1), Bytes.toBytes("m"), Bytes.toBytes("s")); + } + + @Test + public void testComplexMultiByteKeys() { + // Scenario: Using multi-byte keys (realistic scenario) + // Region: [\x01\x00, \x05\x00) + // Chunks: [\x01\x00, \x02\x00], [\x03\x00, \x04\x00] + // Expected: [\x02\x00, \x03\x00), [\x04\x00, \x05\x00) + byte[] regionStart = new byte[] { 0x01, 0x00 }; + byte[] regionEnd = new byte[] { 0x05, 0x00 }; + List processedChunks = new ArrayList<>(); + processedChunks.add(createChunk(new byte[] { 0x01, 0x00 }, new byte[] { 0x02, 0x00 })); + processedChunks.add(createChunk(new byte[] { 0x03, 0x00 }, new byte[] { 0x04, 0x00 })); + + List> result = + mapper.calculateUnprocessedRanges(regionStart, regionEnd, processedChunks); + + assertEquals("Should have 2 unprocessed ranges with multi-byte keys", 2, result.size()); + assertGap(result.get(0), new byte[] { 0x02, 0x00 }, new byte[] { 0x03, 0x00 }); + assertGap(result.get(1), new byte[] { 0x04, 0x00 }, new byte[] { 0x05, 0x00 }); + } + + // Tests for shouldStartKeyBeInclusive method + + @Test + public void testShouldStartKeyBeInclusiveWithNullMapperStart() { + // Null mapper region start should return true (first region) + assertTrue(mapper.shouldStartKeyBeInclusive(null, new ArrayList<>())); + } + + @Test + public void testShouldStartKeyBeInclusiveWithEmptyMapperStart() { + // Empty mapper region start should return true (first region) + assertTrue(mapper.shouldStartKeyBeInclusive(HConstants.EMPTY_START_ROW, new ArrayList<>())); + } + + @Test + public void testShouldStartKeyBeInclusiveWithNullChunks() { + // Null processed chunks should return true + assertTrue(mapper.shouldStartKeyBeInclusive(Bytes.toBytes("a"), null)); + } + + @Test + public void testShouldStartKeyBeInclusiveWithEmptyChunks() { + // Empty processed chunks should return true + assertTrue(mapper.shouldStartKeyBeInclusive(Bytes.toBytes("a"), new ArrayList<>())); + } + + @Test + public void testShouldStartKeyBeInclusiveWhenFirstChunkAfterMapperStart() { + // Mapper: [a, ...) Chunks: [c, ...] + // First chunk starts AFTER mapper start -> return true (gap at beginning) + byte[] mapperStart = Bytes.toBytes("a"); + List chunks = new ArrayList<>(); + chunks.add(createChunk(Bytes.toBytes("c"), Bytes.toBytes("f"))); + assertTrue(mapper.shouldStartKeyBeInclusive(mapperStart, chunks)); + } + + @Test + public void testShouldStartKeyBeInclusiveWhenFirstChunkAtMapperStart() { + // Mapper: [a, ...) Chunks: [a, ...] + // First chunk starts AT mapper start -> return false (no gap) + byte[] mapperStart = Bytes.toBytes("a"); + List chunks = new ArrayList<>(); + chunks.add(createChunk(Bytes.toBytes("a"), Bytes.toBytes("f"))); + assertFalse(mapper.shouldStartKeyBeInclusive(mapperStart, chunks)); + } + + @Test + public void testShouldStartKeyBeInclusiveWhenFirstChunkBeforeMapperStart() { + // Mapper: [d, ...) Chunks: [a, ...] + // First chunk starts BEFORE mapper start -> return false (no gap, chunk overlaps start) + byte[] mapperStart = Bytes.toBytes("d"); + List chunks = new ArrayList<>(); + chunks.add(createChunk(Bytes.toBytes("a"), Bytes.toBytes("g"))); + assertFalse(mapper.shouldStartKeyBeInclusive(mapperStart, chunks)); + } + +} diff --git a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepositoryTest.java b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepositoryTest.java new file mode 100644 index 00000000000..40262b3a1e0 --- /dev/null +++ b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/PhoenixSyncTableOutputRepositoryTest.java @@ -0,0 +1,560 @@ +/* + * 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.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNotSame; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.Timestamp; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.phoenix.mapreduce.PhoenixSyncTableOutputRow.Status; +import org.apache.phoenix.mapreduce.PhoenixSyncTableOutputRow.Type; +import org.apache.phoenix.query.BaseTest; +import org.apache.phoenix.util.ReadOnlyProps; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.phoenix.thirdparty.com.google.common.collect.Maps; + +/** + * Unit tests for PhoenixSyncTableOutputRepository and PhoenixSyncTableOutputRow. Tests checkpoint + * table operations and data model functionality. + */ +public class PhoenixSyncTableOutputRepositoryTest extends BaseTest { + + private Connection connection; + private PhoenixSyncTableOutputRepository repository; + private String targetCluster; + + @BeforeClass + public static synchronized void doSetup() throws Exception { + Map props = Maps.newHashMapWithExpectedSize(1); + setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator())); + } + + @AfterClass + public static synchronized void freeResources() throws Exception { + BaseTest.freeResourcesIfBeyondThreshold(); + } + + @Before + public void setup() throws Exception { + Properties props = new Properties(); + connection = DriverManager.getConnection(getUrl(), props); + repository = new PhoenixSyncTableOutputRepository(connection); + repository.createSyncCheckpointTableIfNotExists(); + targetCluster = "target-zk1,target-zk2:2181:/hbase"; + } + + @After + public void tearDown() throws Exception { + if (connection != null) { + try { + connection.createStatement().execute("DROP TABLE IF EXISTS " + + PhoenixSyncTableOutputRepository.SYNC_TABLE_CHECKPOINT_TABLE_NAME); + connection.commit(); + connection.close(); + } catch (SQLException e) { + // Ignore cleanup errors + } + } + } + + @Test + public void testCreateSyncCheckpointTableIfNotExists() throws Exception { + // Table was already created in @Before, verify it exists + String query = + "SELECT COUNT(*) FROM " + PhoenixSyncTableOutputRepository.SYNC_TABLE_CHECKPOINT_TABLE_NAME; + try (Statement stmt = connection.createStatement(); ResultSet rs = stmt.executeQuery(query)) { + assertTrue("Table should exist and be queryable", rs.next()); + } + } + + @Test + public void testCreateSyncCheckpointTableIdempotent() throws Exception { + // Create again - should not throw exception + repository.createSyncCheckpointTableIfNotExists(); + + // Verify table still exists + String query = + "SELECT COUNT(*) FROM " + PhoenixSyncTableOutputRepository.SYNC_TABLE_CHECKPOINT_TABLE_NAME; + try (Statement stmt = connection.createStatement(); ResultSet rs = stmt.executeQuery(query)) { + assertTrue("Table should still exist after second create", rs.next()); + } + } + + @Test + public void testCheckpointMapperRegionVerified() throws Exception { + String tableName = generateUniqueName(); + byte[] startKey = Bytes.toBytes("row1"); + byte[] endKey = Bytes.toBytes("row100"); + Timestamp timestamp = new Timestamp(System.currentTimeMillis()); + + repository.checkpointSyncTableResult(tableName, targetCluster, Type.MAPPER_REGION, 0L, 1000L, + false, startKey, endKey, Status.VERIFIED, timestamp, timestamp, + "SOURCE_ROWS_PROCESSED=10,TARGET_ROWS_PROCESSED=10"); + + List results = + repository.getProcessedMapperRegions(tableName, targetCluster, 0L, 1000L); + + assertEquals(1, results.size()); + assertArrayEquals(startKey, results.get(0).getStartRowKey()); + assertArrayEquals(endKey, results.get(0).getEndRowKey()); + } + + @Test + public void testCheckpointChunkVerified() throws Exception { + String tableName = generateUniqueName(); + byte[] chunkStart = Bytes.toBytes("row10"); + byte[] chunkEnd = Bytes.toBytes("row20"); + byte[] mapperStart = Bytes.toBytes("row1"); + byte[] mapperEnd = Bytes.toBytes("row100"); + Timestamp timestamp = new Timestamp(System.currentTimeMillis()); + + repository.checkpointSyncTableResult(tableName, targetCluster, Type.CHUNK, 0L, 1000L, false, + chunkStart, chunkEnd, Status.VERIFIED, timestamp, timestamp, null); + + List results = + repository.getProcessedChunks(tableName, targetCluster, 0L, 1000L, mapperStart, mapperEnd); + + assertTrue("Should find chunk within mapper region", results.size() > 0); + } + + @Test + public void testCheckpointWithEmptyStartKey() throws Exception { + String tableName = generateUniqueName(); + byte[] startKey = new byte[0]; + byte[] endKey = Bytes.toBytes("row100"); + Timestamp timestamp = new Timestamp(System.currentTimeMillis()); + + // Pass empty byte array as start key (first region) + repository.checkpointSyncTableResult(tableName, targetCluster, Type.MAPPER_REGION, 0L, 1000L, + false, startKey, endKey, Status.VERIFIED, timestamp, timestamp, null); + + List results = + repository.getProcessedMapperRegions(tableName, targetCluster, 0L, 1000L); + + assertEquals(1, results.size()); + // Phoenix returns null for empty byte arrays in primary key columns + byte[] retrievedStartKey = results.get(0).getStartRowKey(); + assertTrue("Start key should be null or empty for first region", + retrievedStartKey == null || retrievedStartKey.length == 0); + } + + @Test + public void testCheckpointWithNullEndKey() throws Exception { + String tableName = generateUniqueName(); + byte[] startKey = Bytes.toBytes("row1"); + Timestamp timestamp = new Timestamp(System.currentTimeMillis()); + + repository.checkpointSyncTableResult(tableName, targetCluster, Type.MAPPER_REGION, 0L, 1000L, + false, startKey, null, Status.VERIFIED, timestamp, timestamp, null); + + List results = + repository.getProcessedMapperRegions(tableName, targetCluster, 0L, 1000L); + + assertEquals(1, results.size()); + } + + @Test + public void testCheckpointWithCounters() throws Exception { + String tableName = generateUniqueName(); + byte[] startKey = Bytes.toBytes("row1"); + byte[] endKey = Bytes.toBytes("row100"); + Timestamp timestamp = new Timestamp(System.currentTimeMillis()); + String counters = "SOURCE_ROWS_PROCESSED=100,TARGET_ROWS_PROCESSED=95"; + + repository.checkpointSyncTableResult(tableName, targetCluster, Type.MAPPER_REGION, 0L, 1000L, + false, startKey, endKey, Status.MISMATCHED, timestamp, timestamp, counters); + + // Verify by querying directly + String query = "SELECT COUNTERS FROM " + + PhoenixSyncTableOutputRepository.SYNC_TABLE_CHECKPOINT_TABLE_NAME + " WHERE TABLE_NAME = ?"; + try (java.sql.PreparedStatement ps = connection.prepareStatement(query)) { + ps.setString(1, tableName); + try (ResultSet rs = ps.executeQuery()) { + assertTrue(rs.next()); + assertEquals(counters, rs.getString("COUNTERS")); + } + } + } + + @Test + public void testCheckpointValidationNullTableName() throws Exception { + byte[] startKey = Bytes.toBytes("row1"); + Timestamp timestamp = new Timestamp(System.currentTimeMillis()); + + try { + repository.checkpointSyncTableResult(null, targetCluster, Type.MAPPER_REGION, 0L, 1000L, + false, startKey, startKey, Status.VERIFIED, timestamp, timestamp, null); + fail("Should throw IllegalArgumentException for null tableName"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("TableName cannot be null")); + } + } + + @Test + public void testCheckpointValidationNullTargetCluster() throws Exception { + String tableName = generateUniqueName(); + byte[] startKey = Bytes.toBytes("row1"); + Timestamp timestamp = new Timestamp(System.currentTimeMillis()); + + try { + repository.checkpointSyncTableResult(tableName, null, Type.MAPPER_REGION, 0L, 1000L, false, + startKey, startKey, Status.VERIFIED, timestamp, timestamp, null); + fail("Should throw IllegalArgumentException for null targetCluster"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("TargetCluster cannot be null")); + } + } + + @Test + public void testCheckpointValidationNullType() throws Exception { + String tableName = generateUniqueName(); + byte[] startKey = Bytes.toBytes("row1"); + Timestamp timestamp = new Timestamp(System.currentTimeMillis()); + + try { + repository.checkpointSyncTableResult(tableName, targetCluster, null, 0L, 1000L, false, + startKey, startKey, Status.VERIFIED, timestamp, timestamp, null); + fail("Should throw IllegalArgumentException for null type"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Type cannot be null")); + } + } + + @Test + public void testCheckpointValidationNullTimeRange() throws Exception { + String tableName = generateUniqueName(); + byte[] startKey = Bytes.toBytes("row1"); + Timestamp timestamp = new Timestamp(System.currentTimeMillis()); + + try { + repository.checkpointSyncTableResult(tableName, targetCluster, Type.MAPPER_REGION, null, + 1000L, false, startKey, startKey, Status.VERIFIED, timestamp, timestamp, null); + fail("Should throw IllegalArgumentException for null fromTime"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("FromTime and ToTime cannot be null")); + } + } + + @Test + public void testCheckpointUpsertBehavior() throws Exception { + String tableName = generateUniqueName(); + byte[] startKey = Bytes.toBytes("row1"); + byte[] endKey = Bytes.toBytes("row100"); + Timestamp timestamp1 = new Timestamp(System.currentTimeMillis()); + Timestamp timestamp2 = new Timestamp(System.currentTimeMillis() + 1000); + + // Insert first time + repository.checkpointSyncTableResult(tableName, targetCluster, Type.MAPPER_REGION, 0L, 1000L, + false, startKey, endKey, Status.VERIFIED, timestamp1, timestamp1, "counter=1"); + + // Upsert with same PK but different values + repository.checkpointSyncTableResult(tableName, targetCluster, Type.MAPPER_REGION, 0L, 1000L, + false, startKey, endKey, Status.MISMATCHED, timestamp2, timestamp2, "counter=2"); + + // Verify only one row exists with updated values + List results = + repository.getProcessedMapperRegions(tableName, targetCluster, 0L, 1000L); + + assertEquals("Should have only one row after upsert", 1, results.size()); + } + + @Test + public void testGetProcessedMapperRegionsEmpty() throws Exception { + String tableName = generateUniqueName(); + + List results = + repository.getProcessedMapperRegions(tableName, targetCluster, 0L, 1000L); + + assertEquals(0, results.size()); + } + + @Test + public void testGetProcessedMapperRegionsBoth() throws Exception { + String tableName = generateUniqueName(); + byte[] startKey1 = Bytes.toBytes("row1"); + byte[] endKey1 = Bytes.toBytes("row50"); + byte[] startKey2 = Bytes.toBytes("row50"); + byte[] endKey2 = Bytes.toBytes("row100"); + Timestamp timestamp = new Timestamp(System.currentTimeMillis()); + + repository.checkpointSyncTableResult(tableName, targetCluster, Type.MAPPER_REGION, 0L, 1000L, + false, startKey1, endKey1, Status.VERIFIED, timestamp, timestamp, null); + + repository.checkpointSyncTableResult(tableName, targetCluster, Type.MAPPER_REGION, 0L, 1000L, + false, startKey2, endKey2, Status.MISMATCHED, timestamp, timestamp, null); + + List results = + repository.getProcessedMapperRegions(tableName, targetCluster, 0L, 1000L); + + assertEquals(2, results.size()); + } + + @Test + public void testGetProcessedMapperRegionsFiltersChunks() throws Exception { + String tableName = generateUniqueName(); + byte[] startKey = Bytes.toBytes("row1"); + byte[] endKey = Bytes.toBytes("row100"); + Timestamp timestamp = new Timestamp(System.currentTimeMillis()); + + // Insert mapper region + repository.checkpointSyncTableResult(tableName, targetCluster, Type.MAPPER_REGION, 0L, 1000L, + false, startKey, endKey, Status.VERIFIED, timestamp, timestamp, null); + + // Insert chunk + repository.checkpointSyncTableResult(tableName, targetCluster, Type.CHUNK, 0L, 1000L, false, + startKey, endKey, Status.VERIFIED, timestamp, timestamp, null); + + List results = + repository.getProcessedMapperRegions(tableName, targetCluster, 0L, 1000L); + + assertEquals("Should only return MAPPER_REGION entries", 1, results.size()); + } + + @Test + public void testGetProcessedMapperRegionsFiltersTimeRange() throws Exception { + String tableName = generateUniqueName(); + byte[] startKey = Bytes.toBytes("row1"); + byte[] endKey = Bytes.toBytes("row100"); + Timestamp timestamp = new Timestamp(System.currentTimeMillis()); + + // Insert with time range 0-1000 + repository.checkpointSyncTableResult(tableName, targetCluster, Type.MAPPER_REGION, 0L, 1000L, + false, startKey, endKey, Status.VERIFIED, timestamp, timestamp, null); + + // Query with different time range + List results = + repository.getProcessedMapperRegions(tableName, targetCluster, 2000L, 3000L); + + assertEquals("Should not find entry with different time range", 0, results.size()); + } + + @Test + public void testGetProcessedChunksEmpty() throws Exception { + String tableName = generateUniqueName(); + byte[] mapperStart = Bytes.toBytes("row1"); + byte[] mapperEnd = Bytes.toBytes("row100"); + + List results = + repository.getProcessedChunks(tableName, targetCluster, 0L, 1000L, mapperStart, mapperEnd); + + assertEquals(0, results.size()); + } + + @Test + public void testGetProcessedChunksBothStatuses() throws Exception { + String tableName = generateUniqueName(); + byte[] chunk1Start = Bytes.toBytes("row10"); + byte[] chunk1End = Bytes.toBytes("row20"); + byte[] chunk2Start = Bytes.toBytes("row30"); + byte[] chunk2End = Bytes.toBytes("row40"); + byte[] mapperStart = Bytes.toBytes("row1"); + byte[] mapperEnd = Bytes.toBytes("row99"); + Timestamp timestamp = new Timestamp(System.currentTimeMillis()); + + repository.checkpointSyncTableResult(tableName, targetCluster, Type.CHUNK, 0L, 1000L, false, + chunk1Start, chunk1End, Status.VERIFIED, timestamp, timestamp, null); + + repository.checkpointSyncTableResult(tableName, targetCluster, Type.CHUNK, 0L, 1000L, false, + chunk2Start, chunk2End, Status.MISMATCHED, timestamp, timestamp, null); + + List results = + repository.getProcessedChunks(tableName, targetCluster, 0L, 1000L, mapperStart, mapperEnd); + + assertEquals(2, results.size()); + } + + @Test + public void testGetProcessedChunksFiltersMapperRegions() throws Exception { + String tableName = generateUniqueName(); + byte[] startKey = Bytes.toBytes("row10"); + byte[] endKey = Bytes.toBytes("row20"); + byte[] mapperStart = Bytes.toBytes("row1"); + byte[] mapperEnd = Bytes.toBytes("row100"); + Timestamp timestamp = new Timestamp(System.currentTimeMillis()); + + // Insert mapper region + repository.checkpointSyncTableResult(tableName, targetCluster, Type.MAPPER_REGION, 0L, 1000L, + false, startKey, endKey, Status.VERIFIED, timestamp, timestamp, null); + + // Insert chunk + repository.checkpointSyncTableResult(tableName, targetCluster, Type.CHUNK, 0L, 1000L, false, + startKey, endKey, Status.VERIFIED, timestamp, timestamp, null); + + List results = + repository.getProcessedChunks(tableName, targetCluster, 0L, 1000L, mapperStart, mapperEnd); + + assertEquals("Should only return CHUNK entries", 1, results.size()); + } + + @Test + public void testGetProcessedChunksWithNoBoundaries() throws Exception { + String tableName = generateUniqueName(); + byte[] chunkStart = Bytes.toBytes("row50"); + byte[] chunkEnd = Bytes.toBytes("row60"); + Timestamp timestamp = new Timestamp(System.currentTimeMillis()); + + repository.checkpointSyncTableResult(tableName, targetCluster, Type.CHUNK, 0L, 1000L, false, + chunkStart, chunkEnd, Status.VERIFIED, timestamp, timestamp, null); + + // Query with no boundaries (entire table) + List results = + repository.getProcessedChunks(tableName, targetCluster, 0L, 1000L, null, null); + + assertEquals(1, results.size()); + } + + @Test + public void testBuilderAllFields() throws Exception { + String tableName = generateUniqueName(); + byte[] startKey = Bytes.toBytes("start"); + byte[] endKey = Bytes.toBytes("end"); + Timestamp timestamp = new Timestamp(System.currentTimeMillis()); + + PhoenixSyncTableOutputRow row = new PhoenixSyncTableOutputRow.Builder().setTableName(tableName) + .setTargetCluster(targetCluster).setType(Type.CHUNK).setFromTime(0L).setToTime(1000L) + .setIsDryRun(false).setStartRowKey(startKey).setEndRowKey(endKey) + .setExecutionStartTime(timestamp).setExecutionEndTime(timestamp).setStatus(Status.VERIFIED) + .setCounters("SOURCE_ROWS_PROCESSED=100").build(); + + assertEquals(tableName, row.getTableName()); + assertEquals(targetCluster, row.getTargetCluster()); + assertEquals(Type.CHUNK, row.getType()); + assertEquals(Long.valueOf(0L), row.getFromTime()); + assertEquals(Long.valueOf(1000L), row.getToTime()); + assertArrayEquals(startKey, row.getStartRowKey()); + assertArrayEquals(endKey, row.getEndRowKey()); + assertEquals(Status.VERIFIED, row.getStatus()); + } + + @Test + public void testBuilderMinimalFields() throws Exception { + byte[] startKey = Bytes.toBytes("start"); + + PhoenixSyncTableOutputRow row = + new PhoenixSyncTableOutputRow.Builder().setStartRowKey(startKey).build(); + + assertNotNull(row); + assertArrayEquals(startKey, row.getStartRowKey()); + } + + @Test + public void testGetStartRowKeyDefensiveCopy() throws Exception { + byte[] startKey = Bytes.toBytes("start"); + + PhoenixSyncTableOutputRow row = + new PhoenixSyncTableOutputRow.Builder().setStartRowKey(startKey).build(); + + byte[] retrieved = row.getStartRowKey(); + assertNotSame("Should return a copy, not the original", startKey, retrieved); + + // Modify retrieved array + retrieved[0] = (byte) 0xFF; + + // Get again and verify it's unchanged + byte[] retrievedAgain = row.getStartRowKey(); + assertFalse("Internal array should not be modified", retrievedAgain[0] == (byte) 0xFF); + } + + @Test + public void testEqualsDifferentObjectSameValues() throws Exception { + byte[] startKey = Bytes.toBytes("start"); + byte[] endKey = Bytes.toBytes("end"); + + PhoenixSyncTableOutputRow row1 = + new PhoenixSyncTableOutputRow.Builder().setTableName("table1").setTargetCluster(targetCluster) + .setType(Type.CHUNK).setStartRowKey(startKey).setEndRowKey(endKey).build(); + + PhoenixSyncTableOutputRow row2 = + new PhoenixSyncTableOutputRow.Builder().setTableName("table1").setTargetCluster(targetCluster) + .setType(Type.CHUNK).setStartRowKey(startKey).setEndRowKey(endKey).build(); + + assertTrue(row1.equals(row2)); + assertEquals(row1.hashCode(), row2.hashCode()); + } + + @Test + public void testEqualsDifferentValues() throws Exception { + PhoenixSyncTableOutputRow row1 = new PhoenixSyncTableOutputRow.Builder().setTableName("table1") + .setStartRowKey(Bytes.toBytes("start1")).build(); + + PhoenixSyncTableOutputRow row2 = new PhoenixSyncTableOutputRow.Builder().setTableName("table2") + .setStartRowKey(Bytes.toBytes("start2")).build(); + + assertFalse(row1.equals(row2)); + } + + @Test + public void testEqualsWithByteArrays() throws Exception { + byte[] startKey1 = Bytes.toBytes("start"); + byte[] startKey2 = Bytes.toBytes("start"); // Same content, different object + + PhoenixSyncTableOutputRow row1 = + new PhoenixSyncTableOutputRow.Builder().setStartRowKey(startKey1).build(); + + PhoenixSyncTableOutputRow row2 = + new PhoenixSyncTableOutputRow.Builder().setStartRowKey(startKey2).build(); + + assertTrue("Byte arrays with same content should be equal", row1.equals(row2)); + } + + @Test + public void testParseCounterValueSingle() throws Exception { + PhoenixSyncTableOutputRow row = new PhoenixSyncTableOutputRow.Builder() + .setStartRowKey(Bytes.toBytes("start")).setCounters("SOURCE_ROWS_PROCESSED=100").build(); + + assertEquals(100L, row.getSourceRowsProcessed()); + } + + @Test + public void testParseCounterValueMultiple() throws Exception { + PhoenixSyncTableOutputRow row = new PhoenixSyncTableOutputRow.Builder() + .setStartRowKey(Bytes.toBytes("start")) + .setCounters("SOURCE_ROWS_PROCESSED=100,TARGET_ROWS_PROCESSED=95,CHUNKS_VERIFIED=10").build(); + + assertEquals(100L, row.getSourceRowsProcessed()); + assertEquals(95L, row.getTargetRowsProcessed()); + } + + @Test + public void testParseCounterValueNull() throws Exception { + PhoenixSyncTableOutputRow row = new PhoenixSyncTableOutputRow.Builder() + .setStartRowKey(Bytes.toBytes("start")).setCounters(null).build(); + + assertEquals(0L, row.getSourceRowsProcessed()); + assertEquals(0L, row.getTargetRowsProcessed()); + } +} diff --git a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/bulkload/PhoenixSyncTableToolTest.java b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/bulkload/PhoenixSyncTableToolTest.java new file mode 100644 index 00000000000..e799e0e96ea --- /dev/null +++ b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/bulkload/PhoenixSyncTableToolTest.java @@ -0,0 +1,448 @@ +/* + * 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.bulkload; + +import static org.apache.phoenix.mapreduce.util.PhoenixMapReduceUtil.INVALID_TIME_RANGE_EXCEPTION_MESSAGE; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.phoenix.mapreduce.PhoenixSyncTableTool; +import org.apache.phoenix.query.BaseTest; +import org.apache.phoenix.query.QueryServices; +import org.apache.phoenix.query.QueryServicesOptions; +import org.apache.phoenix.util.EnvironmentEdgeManager; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import org.apache.phoenix.thirdparty.org.apache.commons.cli.CommandLine; + +/** + * Unit tests for PhoenixSyncTableTool command-line parsing and validation. + */ +public class PhoenixSyncTableToolTest extends BaseTest { + + PhoenixSyncTableTool tool; + private String tableName; + private String targetCluster; + private String schema; + private String tenantId; + + @Rule + public ExpectedException exceptionRule = ExpectedException.none(); + + @Before + public void setup() { + tool = new PhoenixSyncTableTool(); + Configuration conf = HBaseConfiguration.create(); + tool.setConf(conf); + tool.initializeConfiguration(); + schema = generateUniqueName(); + tableName = generateUniqueName(); + targetCluster = "target-zk1,target-zk2:2181:/hbase"; + tenantId = generateUniqueName(); + } + + @Test + public void testParseOptionsTimeRangeBothTimesProvided() throws Exception { + Long startTime = 10L; + Long endTime = 15L; + String[] args = getArgValues(schema, tableName, targetCluster, tenantId, startTime, endTime); + CommandLine cmdLine = tool.parseOptions(args); + tool.populateSyncTableToolAttributes(cmdLine); + assertEquals(startTime, tool.getStartTime()); + assertEquals(endTime, tool.getEndTime()); + } + + @Test + public void testParseOptionsTimeRangeBothTimesNull() throws Exception { + String[] args = getArgValues(schema, tableName, targetCluster, tenantId, null, null); + CommandLine cmdLine = tool.parseOptions(args); + tool.populateSyncTableToolAttributes(cmdLine); + assertEquals(Long.valueOf(0L), tool.getStartTime()); + // endTime should default to current time - 1 hour + Long endTime = tool.getEndTime(); + assertNotNull(endTime); + long expectedEndTime = EnvironmentEdgeManager.currentTimeMillis() - (60 * 60 * 1000); + assertTrue("End time should be close to (current time - 1 hour)", + Math.abs(endTime - expectedEndTime) < 10000); + } + + @Test + public void testParseOptionsTimeRangeStartTimeOnlyProvided() throws Exception { + Long startTime = 10L; + String[] args = getArgValues(schema, tableName, targetCluster, tenantId, startTime, null); + CommandLine cmdLine = tool.parseOptions(args); + tool.populateSyncTableToolAttributes(cmdLine); + assertEquals(startTime, tool.getStartTime()); + // endTime should default to current time - 1 hour + Long endTime = tool.getEndTime(); + assertNotNull(endTime); + long expectedEndTime = EnvironmentEdgeManager.currentTimeMillis() - (60 * 60 * 1000); + assertTrue("End time should be close to (current time - 1 hour)", + Math.abs(endTime - expectedEndTime) < 10000); + } + + @Test + public void testParseOptionsTimeRangeEndTimeOnlyProvided() throws Exception { + Long endTime = 15L; + String[] args = getArgValues(schema, tableName, targetCluster, tenantId, null, endTime); + CommandLine cmdLine = tool.parseOptions(args); + tool.populateSyncTableToolAttributes(cmdLine); + assertEquals(Long.valueOf(0L), tool.getStartTime()); + assertEquals(endTime, tool.getEndTime()); + } + + @Test + public void testParseOptionsTimeRangeStartTimeInFuture() throws Exception { + Long startTime = EnvironmentEdgeManager.currentTimeMillis() + 100000; + Long endTime = EnvironmentEdgeManager.currentTimeMillis() + 200000; + String[] args = getArgValues(schema, tableName, targetCluster, tenantId, startTime, endTime); + CommandLine cmdLine = tool.parseOptions(args); + exceptionRule.expect(RuntimeException.class); + exceptionRule.expectMessage(INVALID_TIME_RANGE_EXCEPTION_MESSAGE); + tool.populateSyncTableToolAttributes(cmdLine); + } + + @Test + public void testParseOptionsTimeRangeEndTimeInFuture() throws Exception { + Long startTime = EnvironmentEdgeManager.currentTimeMillis(); + Long endTime = EnvironmentEdgeManager.currentTimeMillis() + 100000; + String[] args = getArgValues(schema, tableName, targetCluster, tenantId, startTime, endTime); + CommandLine cmdLine = tool.parseOptions(args); + exceptionRule.expect(RuntimeException.class); + exceptionRule.expectMessage(INVALID_TIME_RANGE_EXCEPTION_MESSAGE); + tool.populateSyncTableToolAttributes(cmdLine); + } + + @Test + public void testParseOptionsTimeRangeStartTimeNullEndTimeInFuture() throws Exception { + Long endTime = EnvironmentEdgeManager.currentTimeMillis() + 100000; + String[] args = getArgValues(schema, tableName, targetCluster, tenantId, null, endTime); + CommandLine cmdLine = tool.parseOptions(args); + exceptionRule.expect(RuntimeException.class); + exceptionRule.expectMessage(INVALID_TIME_RANGE_EXCEPTION_MESSAGE); + tool.populateSyncTableToolAttributes(cmdLine); + } + + @Test + public void testParseOptionsTimeRangeStartTimeEqualEndTime() throws Exception { + Long startTime = 10L; + Long endTime = 10L; + String[] args = getArgValues(schema, tableName, targetCluster, tenantId, startTime, endTime); + CommandLine cmdLine = tool.parseOptions(args); + exceptionRule.expect(RuntimeException.class); + exceptionRule.expectMessage(INVALID_TIME_RANGE_EXCEPTION_MESSAGE); + tool.populateSyncTableToolAttributes(cmdLine); + } + + @Test + public void testParseOptionsTimeRangeStartTimeGreaterThanEndTime() throws Exception { + Long startTime = 15L; + Long endTime = 10L; + String[] args = getArgValues(schema, tableName, targetCluster, tenantId, startTime, endTime); + CommandLine cmdLine = tool.parseOptions(args); + exceptionRule.expect(RuntimeException.class); + exceptionRule.expectMessage(INVALID_TIME_RANGE_EXCEPTION_MESSAGE); + tool.populateSyncTableToolAttributes(cmdLine); + } + + @Test + public void testParseOptionsWithSchema() throws Exception { + Long startTime = 1L; + Long endTime = 10L; + String[] args = getArgValues(schema, tableName, targetCluster, tenantId, startTime, endTime); + CommandLine cmdLine = tool.parseOptions(args); + tool.populateSyncTableToolAttributes(cmdLine); + assertEquals(startTime, tool.getStartTime()); + assertEquals(endTime, tool.getEndTime()); + assertEquals(schema, tool.getSchemaName()); + } + + @Test + public void testParseOptionsWithoutSchema() throws Exception { + Long startTime = 1L; + Long endTime = 10L; + String[] args = getArgValues(null, tableName, targetCluster, tenantId, startTime, endTime); + CommandLine cmdLine = tool.parseOptions(args); + tool.populateSyncTableToolAttributes(cmdLine); + assertEquals(startTime, tool.getStartTime()); + assertEquals(endTime, tool.getEndTime()); + assertNull(tool.getSchemaName()); + } + + @Test + public void testParseOptionsWithTenantId() throws Exception { + Long startTime = 1L; + Long endTime = 10L; + String[] args = getArgValues(schema, tableName, targetCluster, tenantId, startTime, endTime); + CommandLine cmdLine = tool.parseOptions(args); + tool.populateSyncTableToolAttributes(cmdLine); + assertEquals(startTime, tool.getStartTime()); + assertEquals(endTime, tool.getEndTime()); + assertEquals(tenantId, tool.getTenantId()); + } + + @Test + public void testParseOptionsWithoutTenantId() throws Exception { + Long startTime = 1L; + Long endTime = 10L; + String[] args = getArgValues(schema, tableName, targetCluster, null, startTime, endTime); + CommandLine cmdLine = tool.parseOptions(args); + tool.populateSyncTableToolAttributes(cmdLine); + assertEquals(startTime, tool.getStartTime()); + assertEquals(endTime, tool.getEndTime()); + assertNull(tool.getTenantId()); + } + + @Test + public void testParseOptionsWithCustomChunkSize() throws Exception { + Long startTime = 1L; + Long endTime = 10L; + Long chunkSize = 1048576L; // 1MB + String[] args = getArgValues(schema, tableName, targetCluster, tenantId, startTime, endTime, + chunkSize, false, false); + CommandLine cmdLine = tool.parseOptions(args); + tool.populateSyncTableToolAttributes(cmdLine); + assertEquals(startTime, tool.getStartTime()); + assertEquals(endTime, tool.getEndTime()); + assertEquals(chunkSize, tool.getChunkSizeBytes()); + } + + @Test + public void testParseOptionsWithoutChunkSize() throws Exception { + Long startTime = 1L; + Long endTime = 10L; + String[] args = getArgValues(schema, tableName, targetCluster, tenantId, startTime, endTime, + null, false, false); + CommandLine cmdLine = tool.parseOptions(args); + tool.populateSyncTableToolAttributes(cmdLine); + assertEquals(startTime, tool.getStartTime()); + assertEquals(endTime, tool.getEndTime()); + // Tool should use default chunk size (1GB) + assertNull(tool.getChunkSizeBytes()); + } + + @Test + public void testParseOptionsDryRunEnabled() throws Exception { + Long startTime = 1L; + Long endTime = 10L; + String[] args = getArgValues(schema, tableName, targetCluster, tenantId, startTime, endTime, + null, true, false); + CommandLine cmdLine = tool.parseOptions(args); + tool.populateSyncTableToolAttributes(cmdLine); + assertEquals(startTime, tool.getStartTime()); + assertEquals(endTime, tool.getEndTime()); + assertTrue(tool.isDryRun()); + } + + @Test + public void testParseOptionsDryRunDisabled() throws Exception { + Long startTime = 1L; + Long endTime = 10L; + String[] args = getArgValues(schema, tableName, targetCluster, tenantId, startTime, endTime, + null, false, false); + CommandLine cmdLine = tool.parseOptions(args); + tool.populateSyncTableToolAttributes(cmdLine); + assertEquals(startTime, tool.getStartTime()); + assertEquals(endTime, tool.getEndTime()); + assertFalse(tool.isDryRun()); + } + + @Test + public void testParseOptionsRunForeground() throws Exception { + Long startTime = 1L; + Long endTime = 10L; + String[] args = getArgValues(schema, tableName, targetCluster, tenantId, startTime, endTime, + null, false, true); + CommandLine cmdLine = tool.parseOptions(args); + tool.populateSyncTableToolAttributes(cmdLine); + assertEquals(startTime, tool.getStartTime()); + assertEquals(endTime, tool.getEndTime()); + assertTrue(tool.isForeground()); + } + + @Test + public void testParseOptionsRunBackground() throws Exception { + Long startTime = 1L; + Long endTime = 10L; + String[] args = getArgValues(schema, tableName, targetCluster, tenantId, startTime, endTime, + null, false, false); + CommandLine cmdLine = tool.parseOptions(args); + tool.populateSyncTableToolAttributes(cmdLine); + assertEquals(startTime, tool.getStartTime()); + assertEquals(endTime, tool.getEndTime()); + assertFalse(tool.isForeground()); + } + + @Test + public void testParseOptionsMissingTableName() throws Exception { + String[] args = new String[] { "--target-cluster", targetCluster }; + exceptionRule.expect(IllegalStateException.class); + exceptionRule.expectMessage("table-name is a mandatory parameter"); + tool.parseOptions(args); + } + + @Test + public void testParseOptionsMissingTargetCluster() throws Exception { + String[] args = new String[] { "--table-name", tableName }; + exceptionRule.expect(IllegalStateException.class); + exceptionRule.expectMessage("target-cluster is a mandatory parameter"); + tool.parseOptions(args); + } + + @Test + public void testDefaultTimeoutConfigurationValues() { + // Verify that default timeout configuration keys exist and can be retrieved + Configuration conf = HBaseConfiguration.create(); + + // Test that we can retrieve default values from configuration + long queryTimeout = conf.getLong(QueryServices.SYNC_TABLE_QUERY_TIMEOUT_ATTRIB, + QueryServicesOptions.DEFAULT_SYNC_TABLE_QUERY_TIMEOUT); + long rpcTimeout = conf.getLong(QueryServices.SYNC_TABLE_RPC_TIMEOUT_ATTRIB, + QueryServicesOptions.DEFAULT_SYNC_TABLE_RPC_TIMEOUT); + long scannerTimeout = conf.getLong(QueryServices.SYNC_TABLE_CLIENT_SCANNER_TIMEOUT_ATTRIB, + QueryServicesOptions.DEFAULT_SYNC_TABLE_CLIENT_SCANNER_TIMEOUT); + int rpcRetries = conf.getInt(QueryServices.SYNC_TABLE_RPC_RETRIES_COUNTER, + QueryServicesOptions.DEFAULT_SYNC_TABLE_RPC_RETRIES_COUNTER); + + // When no custom values are set, configuration returns the defaults + assertEquals("Query timeout should return default when not configured", + QueryServicesOptions.DEFAULT_SYNC_TABLE_QUERY_TIMEOUT, queryTimeout); + assertEquals("RPC timeout should return default when not configured", + QueryServicesOptions.DEFAULT_SYNC_TABLE_RPC_TIMEOUT, rpcTimeout); + assertEquals("Scanner timeout should return default when not configured", + QueryServicesOptions.DEFAULT_SYNC_TABLE_CLIENT_SCANNER_TIMEOUT, scannerTimeout); + assertEquals("RPC retries should return default when not configured", + QueryServicesOptions.DEFAULT_SYNC_TABLE_RPC_RETRIES_COUNTER, rpcRetries); + } + + @Test + public void testCustomTimeoutConfigurationCanBeSet() { + // Verify that custom timeout values can be set in configuration + Configuration conf = HBaseConfiguration.create(); + long customQueryTimeout = 1200000L; // 20 minutes + long customRpcTimeout = 120000L; // 2 minutes + long customScannerTimeout = 360000L; // 6 minutes + int customRpcRetries = 10; + + // Set custom values + conf.setLong(QueryServices.SYNC_TABLE_QUERY_TIMEOUT_ATTRIB, customQueryTimeout); + conf.setLong(QueryServices.SYNC_TABLE_RPC_TIMEOUT_ATTRIB, customRpcTimeout); + conf.setLong(QueryServices.SYNC_TABLE_CLIENT_SCANNER_TIMEOUT_ATTRIB, customScannerTimeout); + conf.setInt(QueryServices.SYNC_TABLE_RPC_RETRIES_COUNTER, customRpcRetries); + + // Verify custom values can be retrieved + assertEquals("Should retrieve custom query timeout", customQueryTimeout, + conf.getLong(QueryServices.SYNC_TABLE_QUERY_TIMEOUT_ATTRIB, -1)); + assertEquals("Should retrieve custom RPC timeout", customRpcTimeout, + conf.getLong(QueryServices.SYNC_TABLE_RPC_TIMEOUT_ATTRIB, -1)); + assertEquals("Should retrieve custom scanner timeout", customScannerTimeout, + conf.getLong(QueryServices.SYNC_TABLE_CLIENT_SCANNER_TIMEOUT_ATTRIB, -1)); + assertEquals("Should retrieve custom RPC retries", customRpcRetries, + conf.getInt(QueryServices.SYNC_TABLE_RPC_RETRIES_COUNTER, -1)); + } + + @Test + public void testParseOptionsWithNegativeChunkSize() throws Exception { + Long startTime = 1L; + Long endTime = 10L; + Long negativeChunkSize = -1048576L; + String[] args = getArgValues(schema, tableName, targetCluster, tenantId, startTime, endTime, + negativeChunkSize, false, false); + CommandLine cmdLine = tool.parseOptions(args); + exceptionRule.expect(IllegalArgumentException.class); + exceptionRule.expectMessage("Chunk size must be a positive value"); + tool.populateSyncTableToolAttributes(cmdLine); + } + + @Test + public void testParseOptionsWithBothMandatoryOptionsMissing() throws Exception { + String[] args = new String[] {}; + exceptionRule.expect(IllegalStateException.class); + exceptionRule.expectMessage("table-name is a mandatory parameter"); + tool.parseOptions(args); + } + + /** + * Creates argument array for PhoenixSyncTableTool + */ + private static String[] getArgValues(String schema, String tableName, String targetCluster, + String tenantId, Long startTime, Long endTime) { + return getArgValues(schema, tableName, targetCluster, tenantId, startTime, endTime, null, false, + false); + } + + /** + * Creates argument array with all optional parameters + */ + private static String[] getArgValues(String schema, String tableName, String targetCluster, + String tenantId, Long startTime, Long endTime, Long chunkSize, boolean dryRun, + boolean runForeground) { + List args = new ArrayList<>(); + + if (schema != null) { + args.add("--schema"); + args.add(schema); + } + + args.add("--table-name"); + args.add(tableName); + + args.add("--target-cluster"); + args.add(targetCluster); + + if (tenantId != null) { + args.add("--tenant-id"); + args.add(tenantId); + } + + if (startTime != null) { + args.add("--from-time"); + args.add(String.valueOf(startTime)); + } + + if (endTime != null) { + args.add("--to-time"); + args.add(String.valueOf(endTime)); + } + + if (chunkSize != null) { + args.add("--chunk-size"); + args.add(String.valueOf(chunkSize)); + } + + if (dryRun) { + args.add("--dry-run"); + } + + if (runForeground) { + args.add("--run-foreground"); + } + + return args.toArray(new String[0]); + } +} diff --git a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtilTest.java b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtilTest.java index a1e9b265b85..104756ad3d2 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtilTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtilTest.java @@ -337,4 +337,55 @@ public void testIndexToolSourceConfig() { sourceTable = PhoenixConfigurationUtil.getIndexToolSourceTable(conf); Assert.assertEquals(sourceTable, SourceTable.DATA_TABLE_SOURCE); } + + @Test + public void testSyncTableConfiguration() { + final Configuration conf = new Configuration(); + + // Test table name + String testTableName = "TEST_SYNC_TABLE"; + PhoenixConfigurationUtil.setPhoenixSyncTableName(conf, testTableName); + assertEquals("Should retrieve set table name", testTableName, + PhoenixConfigurationUtil.getPhoenixSyncTableName(conf)); + + // Test target ZK quorum + String testZkQuorum = "testhost1:2181,testhost2:2181:/hbase"; + PhoenixConfigurationUtil.setPhoenixSyncTableTargetZkQuorum(conf, testZkQuorum); + assertEquals("Should retrieve set target ZK quorum", testZkQuorum, + PhoenixConfigurationUtil.getPhoenixSyncTableTargetZkQuorum(conf)); + + // Test from time + Long testFromTime = 1000000L; + PhoenixConfigurationUtil.setPhoenixSyncTableFromTime(conf, testFromTime); + assertEquals("Should retrieve set from time", testFromTime, + PhoenixConfigurationUtil.getPhoenixSyncTableFromTime(conf)); + + // Test to time + Long testToTime = 2000000L; + PhoenixConfigurationUtil.setPhoenixSyncTableToTime(conf, testToTime); + assertEquals("Should retrieve set to time", testToTime, + PhoenixConfigurationUtil.getPhoenixSyncTableToTime(conf)); + + // Test dry run flag + PhoenixConfigurationUtil.setPhoenixSyncTableDryRun(conf, true); + assertEquals("Should retrieve dry run as true", true, + PhoenixConfigurationUtil.getPhoenixSyncTableDryRun(conf)); + + PhoenixConfigurationUtil.setPhoenixSyncTableDryRun(conf, false); + assertEquals("Should retrieve dry run as false", false, + PhoenixConfigurationUtil.getPhoenixSyncTableDryRun(conf)); + + // Test chunk size bytes + Long testChunkSize = 5000000L; // 5MB + PhoenixConfigurationUtil.setPhoenixSyncTableChunkSizeBytes(conf, testChunkSize); + assertEquals("Should retrieve set chunk size", testChunkSize.longValue(), + PhoenixConfigurationUtil.getPhoenixSyncTableChunkSizeBytes(conf)); + + // Test default chunk size (1GB) + final Configuration freshConf = new Configuration(); + long defaultChunkSize = 1024L * 1024L * 1024L; // 1GB + assertEquals("Should return default chunk size when not set", defaultChunkSize, + PhoenixConfigurationUtil.getPhoenixSyncTableChunkSizeBytes(freshConf)); + } + } diff --git a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/util/SHA256DigestUtilTest.java b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/util/SHA256DigestUtilTest.java new file mode 100644 index 00000000000..417e0e0f9ab --- /dev/null +++ b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/util/SHA256DigestUtilTest.java @@ -0,0 +1,362 @@ +/* + * 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.util; + +import java.io.IOException; +import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.phoenix.util.SHA256DigestUtil; +import org.bouncycastle.crypto.digests.SHA256Digest; +import org.junit.Assert; +import org.junit.Test; + +/** + * Unit tests for SHA256DigestUtil. Tests digest state serialization, deserialization, and hash + * finalization. + */ +public class SHA256DigestUtilTest { + + @Test + public void testEncodeDigestStateBasic() { + SHA256Digest digest = new SHA256Digest(); + digest.update("test".getBytes(), 0, 4); + + byte[] encoded = SHA256DigestUtil.encodeDigestState(digest); + + Assert.assertNotNull("Encoded state should not be null", encoded); + Assert.assertTrue("Encoded state should have length prefix + state data", + encoded.length > Bytes.SIZEOF_INT); + } + + @Test + public void testEncodeDigestStateWithMultipleUpdates() { + SHA256Digest digest = new SHA256Digest(); + digest.update("hello".getBytes(), 0, 5); + digest.update(" ".getBytes(), 0, 1); + digest.update("world".getBytes(), 0, 5); + + byte[] encoded = SHA256DigestUtil.encodeDigestState(digest); + + Assert.assertNotNull("Encoded state should not be null", encoded); + // Extract length prefix + ByteBuffer buffer = ByteBuffer.wrap(encoded); + int stateLength = buffer.getInt(); + Assert.assertTrue("State length should be positive", stateLength > 0); + Assert.assertEquals("Encoded length should match length prefix + state", + Bytes.SIZEOF_INT + stateLength, encoded.length); + } + + @Test + public void testDecodeDigestStateBasic() throws IOException { + SHA256Digest original = new SHA256Digest(); + original.update("test".getBytes(), 0, 4); + + byte[] encoded = SHA256DigestUtil.encodeDigestState(original); + SHA256Digest decoded = SHA256DigestUtil.decodeDigestState(encoded); + + Assert.assertNotNull("Decoded digest should not be null", decoded); + + // Verify by finalizing both and comparing checksums + byte[] originalHash = SHA256DigestUtil.finalizeDigestToChecksum(original); + byte[] decodedHash = SHA256DigestUtil.finalizeDigestToChecksum(decoded); + + Assert.assertArrayEquals("Original and decoded digest should produce same hash", originalHash, + decodedHash); + } + + @Test + public void testDecodeDigestStateEmptyDigest() throws IOException { + SHA256Digest original = new SHA256Digest(); + + byte[] encoded = SHA256DigestUtil.encodeDigestState(original); + SHA256Digest decoded = SHA256DigestUtil.decodeDigestState(encoded); + + Assert.assertNotNull("Decoded digest should not be null", decoded); + + byte[] originalHash = SHA256DigestUtil.finalizeDigestToChecksum(original); + byte[] decodedHash = SHA256DigestUtil.finalizeDigestToChecksum(decoded); + + Assert.assertArrayEquals("Empty digest should produce consistent hash", originalHash, + decodedHash); + } + + @Test + public void testDecodeDigestStateNullInput() { + try { + SHA256DigestUtil.decodeDigestState(null); + Assert.fail("Should throw IllegalArgumentException for null input"); + } catch (IllegalArgumentException e) { + Assert.assertTrue("Error message should mention null", e.getMessage().contains("null")); + } catch (IOException e) { + Assert.fail("Should throw IllegalArgumentException, not IOException"); + } + } + + @Test + public void testDecodeDigestStateEmptyByteArray() { + try { + SHA256DigestUtil.decodeDigestState(new byte[0]); + Assert.fail("Should throw IOException for empty byte array"); + } catch (IOException e) { + // Expected - empty array can't contain a valid 4-byte length prefix + } + } + + @Test + public void testDecodeDigestStateTooShort() { + // Only 3 bytes - less than the 4-byte length prefix + byte[] tooShort = new byte[] { 0x01, 0x02, 0x03 }; + + try { + SHA256DigestUtil.decodeDigestState(tooShort); + Assert.fail("Should throw IOException for too short byte array"); + } catch (IOException e) { + // Expected + } + } + + @Test + public void testDecodeDigestStateMaliciousLargeLength() { + // Create a byte array with malicious large length prefix + ByteBuffer buffer = ByteBuffer.allocate(Bytes.SIZEOF_INT); + buffer.putInt(SHA256DigestUtil.MAX_SHA256_DIGEST_STATE_SIZE + 1); + + try { + SHA256DigestUtil.decodeDigestState(buffer.array()); + Assert.fail( + "Should throw IllegalArgumentException for state size exceeding MAX_SHA256_DIGEST_STATE_SIZE"); + } catch (IllegalArgumentException e) { + Assert.assertTrue("Error message should mention invalid state length", + e.getMessage().contains("Invalid SHA256 state length")); + Assert.assertTrue("Error message should show expected max size", + e.getMessage().contains(String.valueOf(SHA256DigestUtil.MAX_SHA256_DIGEST_STATE_SIZE))); + } catch (IOException e) { + Assert.fail("Should throw IllegalArgumentException for security check failure"); + } + } + + @Test + public void testDecodeDigestStateNegativeLength() { + // Create a byte array with negative length prefix + ByteBuffer buffer = ByteBuffer.allocate(Bytes.SIZEOF_INT); + buffer.putInt(-1); + + try { + SHA256DigestUtil.decodeDigestState(buffer.array()); + Assert.fail("Should throw exception for negative length"); + } catch (Exception e) { + // Expected - either IllegalArgumentException or IOException + } + } + + @Test + public void testDecodeDigestStateLengthMismatch() { + // Create encoded state with length that doesn't match actual data + SHA256Digest digest = new SHA256Digest(); + digest.update("test".getBytes(), 0, 4); + byte[] encoded = SHA256DigestUtil.encodeDigestState(digest); + + // Corrupt the length prefix to be larger than actual state + ByteBuffer buffer = ByteBuffer.wrap(encoded); + buffer.putInt(encoded.length); // Set length larger than actual state size + + try { + SHA256DigestUtil.decodeDigestState(encoded); + Assert.fail("Should throw IOException for length mismatch"); + } catch (IOException e) { + // Expected + } + } + + @Test + public void testFinalizeDigestToChecksumFromEncodedState() throws IOException { + SHA256Digest digest = new SHA256Digest(); + digest.update("test data".getBytes(), 0, 9); + + byte[] encoded = SHA256DigestUtil.encodeDigestState(digest); + byte[] checksum = SHA256DigestUtil.finalizeDigestToChecksum(encoded); + + Assert.assertNotNull("Checksum should not be null", checksum); + Assert.assertEquals("SHA-256 checksum should be 32 bytes", 32, checksum.length); + } + + @Test + public void testFinalizeDigestToChecksumFromDigest() { + SHA256Digest digest = new SHA256Digest(); + digest.update("test data".getBytes(), 0, 9); + + byte[] checksum = SHA256DigestUtil.finalizeDigestToChecksum(digest); + + Assert.assertNotNull("Checksum should not be null", checksum); + Assert.assertEquals("SHA-256 checksum should be 32 bytes", 32, checksum.length); + } + + @Test + public void testFinalizeDigestProducesDeterministicHash() { + SHA256Digest digest1 = new SHA256Digest(); + digest1.update("same input".getBytes(), 0, 10); + + SHA256Digest digest2 = new SHA256Digest(); + digest2.update("same input".getBytes(), 0, 10); + + byte[] hash1 = SHA256DigestUtil.finalizeDigestToChecksum(digest1); + byte[] hash2 = SHA256DigestUtil.finalizeDigestToChecksum(digest2); + + Assert.assertArrayEquals("Same input should produce same hash", hash1, hash2); + } + + @Test + public void testFinalizeDigestProducesDifferentHashForDifferentInput() { + SHA256Digest digest1 = new SHA256Digest(); + digest1.update("input1".getBytes(), 0, 6); + + SHA256Digest digest2 = new SHA256Digest(); + digest2.update("input2".getBytes(), 0, 6); + + byte[] hash1 = SHA256DigestUtil.finalizeDigestToChecksum(digest1); + byte[] hash2 = SHA256DigestUtil.finalizeDigestToChecksum(digest2); + + Assert.assertNotEquals("Different inputs should produce different hashes", + Bytes.toStringBinary(hash1), Bytes.toStringBinary(hash2)); + } + + @Test + public void testRoundTripEncodeDecode() throws IOException { + SHA256Digest original = new SHA256Digest(); + original.update("round trip test".getBytes(), 0, 15); + + // Encode + byte[] encoded = SHA256DigestUtil.encodeDigestState(original); + + // Decode + SHA256Digest decoded = SHA256DigestUtil.decodeDigestState(encoded); + + // Continue hashing with both + original.update(" continued".getBytes(), 0, 10); + decoded.update(" continued".getBytes(), 0, 10); + + // Finalize both + byte[] originalHash = SHA256DigestUtil.finalizeDigestToChecksum(original); + byte[] decodedHash = SHA256DigestUtil.finalizeDigestToChecksum(decoded); + + Assert.assertArrayEquals("Round-trip encode/decode should preserve digest state", originalHash, + decodedHash); + } + + @Test + public void testCrossRegionHashContinuation() throws IOException { + // Simulate cross-region hashing scenario + // Region 1: Hash first part + SHA256Digest region1Digest = new SHA256Digest(); + region1Digest.update("data from region 1".getBytes(), 0, 18); + + // Save state + byte[] savedState = SHA256DigestUtil.encodeDigestState(region1Digest); + + // Region 2: Restore state and continue + SHA256Digest region2Digest = SHA256DigestUtil.decodeDigestState(savedState); + region2Digest.update(" and region 2".getBytes(), 0, 13); + + // Compare with continuous hashing + SHA256Digest continuousDigest = new SHA256Digest(); + continuousDigest.update("data from region 1 and region 2".getBytes(), 0, 31); + + byte[] region2Hash = SHA256DigestUtil.finalizeDigestToChecksum(region2Digest); + byte[] continuousHash = SHA256DigestUtil.finalizeDigestToChecksum(continuousDigest); + + Assert.assertArrayEquals("Cross-region hashing should match continuous hashing", continuousHash, + region2Hash); + } + + @Test + public void testEncodedStateSizeWithinLimits() { + SHA256Digest digest = new SHA256Digest(); + // Hash large data + for (int i = 0; i < 1000; i++) { + digest.update("test data chunk".getBytes(), 0, 15); + } + + byte[] encoded = SHA256DigestUtil.encodeDigestState(digest); + + Assert.assertTrue("Encoded state should be within MAX_SHA256_DIGEST_STATE_SIZE limit", + encoded.length <= Bytes.SIZEOF_INT + SHA256DigestUtil.MAX_SHA256_DIGEST_STATE_SIZE); + } + + @Test + public void testEmptyDigestFinalization() { + SHA256Digest emptyDigest = new SHA256Digest(); + + byte[] hash = SHA256DigestUtil.finalizeDigestToChecksum(emptyDigest); + + Assert.assertNotNull("Empty digest hash should not be null", hash); + Assert.assertEquals("SHA-256 hash should be 32 bytes", 32, hash.length); + } + + @Test + public void testLargeDataHashing() { + SHA256Digest digest = new SHA256Digest(); + + // Hash 1MB of data + byte[] chunk = new byte[1024]; + for (int i = 0; i < 1024; i++) { + digest.update(chunk, 0, chunk.length); + } + + byte[] hash = SHA256DigestUtil.finalizeDigestToChecksum(digest); + + Assert.assertNotNull("Hash of large data should not be null", hash); + Assert.assertEquals("SHA-256 hash should always be 32 bytes", 32, hash.length); + } + + @Test + public void testStateSizeConstant() { + // Verify the constant is reasonable for SHA-256 state + Assert.assertTrue("MAX_SHA256_DIGEST_STATE_SIZE should be at least 96 bytes", true); + Assert.assertTrue("MAX_SHA256_DIGEST_STATE_SIZE should not be excessively large", true); + } + + @Test + public void testEncodedStateLengthPrefixFormat() { + SHA256Digest digest = new SHA256Digest(); + digest.update("test".getBytes(), 0, 4); + + byte[] encoded = SHA256DigestUtil.encodeDigestState(digest); + + // Extract and verify length prefix + ByteBuffer buffer = ByteBuffer.wrap(encoded); + int lengthPrefix = buffer.getInt(); + + Assert.assertEquals("Length prefix should match actual state size", lengthPrefix, + encoded.length - Bytes.SIZEOF_INT); + Assert.assertTrue("Length prefix should be positive", lengthPrefix > 0); + } + + @Test + public void testBinaryDataHashing() { + SHA256Digest digest = new SHA256Digest(); + + // Test with binary data (not just text) + byte[] binaryData = new byte[] { 0x00, 0x01, 0x02, (byte) 0xFF, (byte) 0xFE, (byte) 0xFD }; + digest.update(binaryData, 0, binaryData.length); + + byte[] hash = SHA256DigestUtil.finalizeDigestToChecksum(digest); + + Assert.assertNotNull("Hash of binary data should not be null", hash); + Assert.assertEquals("SHA-256 hash should be 32 bytes", 32, hash.length); + } +}