diff --git a/Jenkinsfile b/Jenkinsfile
index 46a06283885..49a44d3766f 100644
--- a/Jenkinsfile
+++ b/Jenkinsfile
@@ -66,7 +66,7 @@ pipeline {
stage('BuildAndTest') {
options {
- timeout(time: 5, unit: 'HOURS')
+ timeout(time: 9, unit: 'HOURS')
}
steps {
dir("HBASE_${HBASE_PROFILE}") {
diff --git a/Jenkinsfile.yetus b/Jenkinsfile.yetus
index a0e9c43f654..7e99f45b056 100644
--- a/Jenkinsfile.yetus
+++ b/Jenkinsfile.yetus
@@ -37,7 +37,7 @@ pipeline {
}
options {
- timeout(time: 5, unit: 'HOURS')
+ timeout(time: 9, unit: 'HOURS')
}
steps {
diff --git a/phoenix-core-client/src/main/antlr3/PhoenixSQL.g b/phoenix-core-client/src/main/antlr3/PhoenixSQL.g
index 2324c78d673..9eccd4df007 100644
--- a/phoenix-core-client/src/main/antlr3/PhoenixSQL.g
+++ b/phoenix-core-client/src/main/antlr3/PhoenixSQL.g
@@ -85,6 +85,8 @@ tokens
PRE='pre';
POST='post';
CHANGE='change';
+ IDX_MUTATIONS='idx_mutations';
+ DATA_ROW_STATE='data_row_state';
LATEST='latest';
ALL='all';
INDEX='index';
@@ -161,6 +163,9 @@ tokens
REGIONS = 'regions';
NOVERIFY = 'noverify';
RETURNING = 'returning';
+ CONSISTENCY = 'consistency';
+ EVENTUAL = 'eventual';
+ STRONG = 'strong';
}
@@ -209,6 +214,7 @@ import org.apache.phoenix.query.QueryConstants;
import org.apache.phoenix.schema.SortOrder;
import org.apache.phoenix.schema.IllegalDataException;
import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.types.IndexConsistency;
import org.apache.phoenix.schema.PTableType;
import org.apache.phoenix.schema.PTable.IndexType;
import org.apache.phoenix.schema.PTable.CDCChangeScope;
@@ -585,7 +591,7 @@ cdc_change_scopes returns [Set ret]
;
cdc_change_scope returns [CDCChangeScope ret]
- : v=(PRE | POST | CHANGE)
+ : v=(PRE | POST | CHANGE | IDX_MUTATIONS | DATA_ROW_STATE)
{
ret = CDCChangeScope.valueOf(v.getText().toUpperCase());
}
@@ -699,8 +705,8 @@ drop_cdc_node returns [DropCDCStatement ret]
// Parse a alter index statement
alter_index_node returns [AlterIndexStatement ret]
: ALTER INDEX (IF ex=EXISTS)? i=index_name ON t=from_table_name
- ((s=(USABLE | UNUSABLE | REBUILD (isRebuildAll=ALL)? | DISABLE | ACTIVE)) (async=ASYNC)? ((SET?)p=fam_properties)?)
- {ret = factory.alterIndex(factory.namedTable(null, TableName.create(t.getSchemaName(), i.getName())), t.getTableName(), ex!=null, PIndexState.valueOf(SchemaUtil.normalizeIdentifier(s.getText())), isRebuildAll!=null, async!=null, p); }
+ ((s=(USABLE | UNUSABLE | REBUILD (isRebuildAll=ALL)? | DISABLE | ACTIVE)) (async=ASYNC)? ((SET?)p=fam_properties)? | (CONSISTENCY EQ c=(STRONG | EVENTUAL)))
+ {ret = factory.alterIndex(factory.namedTable(null, TableName.create(t.getSchemaName(), i.getName())), t.getTableName(), ex!=null, s!=null ? PIndexState.valueOf(SchemaUtil.normalizeIdentifier(s.getText())) : null, isRebuildAll!=null, async!=null, p, c!=null ? IndexConsistency.valueOf(SchemaUtil.normalizeIdentifier(c.getText())) : null); }
;
// Parse a trace statement.
@@ -1318,6 +1324,9 @@ identifier returns [String ret]
parseNoReserved returns [String ret]
: n=NAME { $ret = n.getText(); }
+ | CONSISTENCY { $ret = "CONSISTENCY"; }
+ | EVENTUAL { $ret = "EVENTUAL"; }
+ | STRONG { $ret = "STRONG"; }
;
case_statement returns [ParseNode ret]
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/compile/JoinCompiler.java b/phoenix-core-client/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
index ad31baeaf84..8a091fbdc06 100644
--- a/phoenix-core-client/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
@@ -1544,7 +1544,8 @@ public static PTable joinProjectedTables(PTable left, PTable right, JoinType typ
.setDisableWAL(PTable.DEFAULT_DISABLE_WAL).setMultiTenant(left.isMultiTenant())
.setStoreNulls(left.getStoreNulls()).setViewType(left.getViewType())
.setViewIndexIdType(left.getviewIndexIdType()).setViewIndexId(left.getViewIndexId())
- .setIndexType(left.getIndexType()).setTransactionProvider(left.getTransactionProvider())
+ .setIndexType(left.getIndexType()).setIndexConsistency(left.getIndexConsistency())
+ .setTransactionProvider(left.getTransactionProvider())
.setUpdateCacheFrequency(left.getUpdateCacheFrequency())
.setNamespaceMapped(left.isNamespaceMapped())
.setAutoPartitionSeqName(left.getAutoPartitionSeqName())
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/MetaDataProtocol.java b/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/MetaDataProtocol.java
index cc7120f71da..cc5f3144767 100644
--- a/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/MetaDataProtocol.java
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/MetaDataProtocol.java
@@ -91,7 +91,7 @@ public abstract class MetaDataProtocol extends MetaDataService {
public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_16_0 = MIN_TABLE_TIMESTAMP + 33;
public static final long MIN_SYSTEM_TABLE_TIMESTAMP_5_1_0 = MIN_SYSTEM_TABLE_TIMESTAMP_4_16_0;
public static final long MIN_SYSTEM_TABLE_TIMESTAMP_5_2_0 = MIN_TABLE_TIMESTAMP + 38;
- public static final long MIN_SYSTEM_TABLE_TIMESTAMP_5_3_0 = MIN_TABLE_TIMESTAMP + 42;
+ public static final long MIN_SYSTEM_TABLE_TIMESTAMP_5_3_0 = MIN_TABLE_TIMESTAMP + 44;
// MIN_SYSTEM_TABLE_TIMESTAMP needs to be set to the max of all the MIN_SYSTEM_TABLE_TIMESTAMP_*
// constants
public static final long MIN_SYSTEM_TABLE_TIMESTAMP = MIN_SYSTEM_TABLE_TIMESTAMP_5_3_0;
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/hbase/index/metrics/MetricsIndexCDCConsumerSource.java b/phoenix-core-client/src/main/java/org/apache/phoenix/hbase/index/metrics/MetricsIndexCDCConsumerSource.java
new file mode 100644
index 00000000000..c278c1fa4ef
--- /dev/null
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/hbase/index/metrics/MetricsIndexCDCConsumerSource.java
@@ -0,0 +1,103 @@
+/*
+ * 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.hbase.index.metrics;
+
+import org.apache.hadoop.hbase.metrics.BaseSource;
+
+/**
+ * IndexCDCConsumer metrics for eventually consistent index updates.
+ */
+public interface MetricsIndexCDCConsumerSource extends BaseSource {
+
+ String METRICS_NAME = "IndexCDCConsumer";
+ String METRICS_CONTEXT = "phoenix";
+ String METRICS_DESCRIPTION = "Metrics about the Phoenix Index CDC Consumer";
+ String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME;
+
+ String CDC_BATCH_PROCESS_TIME = "cdcBatchProcessTime";
+ String CDC_BATCH_PROCESS_TIME_DESC =
+ "Histogram for the end-to-end time in milliseconds for processing one CDC batch";
+
+ String CDC_MUTATION_GENERATE_TIME = "cdcMutationGenerateTime";
+ String CDC_MUTATION_GENERATE_TIME_DESC =
+ "Histogram for the time in milliseconds to generate index mutations from data row states";
+
+ String CDC_MUTATION_APPLY_TIME = "cdcMutationApplyTime";
+ String CDC_MUTATION_APPLY_TIME_DESC =
+ "Histogram for the time in milliseconds to apply index mutations to index tables";
+
+ String CDC_BATCH_COUNT = "cdcBatchCount";
+ String CDC_BATCH_COUNT_DESC = "The number of CDC batches processed";
+
+ String CDC_MUTATION_COUNT = "cdcMutationCount";
+ String CDC_MUTATION_COUNT_DESC = "The number of individual index mutations applied through CDC";
+
+ String CDC_BATCH_FAILURE_COUNT = "cdcBatchFailureCount";
+ String CDC_BATCH_FAILURE_COUNT_DESC = "The number of CDC batch processing failures";
+
+ String CDC_INDEX_UPDATE_LAG = "cdcIndexUpdateLag";
+ String CDC_INDEX_UPDATE_LAG_DESC =
+ "Histogram for the lag in milliseconds between current time and the last processed CDC event";
+
+ /**
+ * Updates the CDC batch processing time histogram.
+ * @param dataTableName physical data table name
+ * @param t time taken in milliseconds
+ */
+ void updateCdcBatchProcessTime(String dataTableName, long t);
+
+ /**
+ * Updates the CDC mutation generation time histogram.
+ * @param dataTableName physical data table name
+ * @param t time taken in milliseconds
+ */
+ void updateCdcMutationGenerateTime(String dataTableName, long t);
+
+ /**
+ * Updates the CDC mutation apply time histogram.
+ * @param dataTableName physical data table name
+ * @param t time taken in milliseconds
+ */
+ void updateCdcMutationApplyTime(String dataTableName, long t);
+
+ /**
+ * Increments the CDC batch count.
+ * @param dataTableName physical data table name
+ */
+ void incrementCdcBatchCount(String dataTableName);
+
+ /**
+ * Increments the CDC mutation count by the given amount.
+ * @param dataTableName physical data table name
+ * @param count number of mutations applied
+ */
+ void incrementCdcMutationCount(String dataTableName, long count);
+
+ /**
+ * Increments the CDC batch failure count.
+ * @param dataTableName physical data table name
+ */
+ void incrementCdcBatchFailureCount(String dataTableName);
+
+ /**
+ * Updates the CDC lag histogram.
+ * @param dataTableName physical data table name
+ * @param lag lag in milliseconds between current time and last processed event
+ */
+ void updateCdcLag(String dataTableName, long lag);
+}
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/hbase/index/metrics/MetricsIndexCDCConsumerSourceImpl.java b/phoenix-core-client/src/main/java/org/apache/phoenix/hbase/index/metrics/MetricsIndexCDCConsumerSourceImpl.java
new file mode 100644
index 00000000000..71c63826339
--- /dev/null
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/hbase/index/metrics/MetricsIndexCDCConsumerSourceImpl.java
@@ -0,0 +1,120 @@
+/*
+ * 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.hbase.index.metrics;
+
+import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.metrics2.MetricHistogram;
+import org.apache.hadoop.metrics2.lib.MutableFastCounter;
+
+/**
+ * Implementation for tracking IndexCDCConsumer metrics.
+ */
+public class MetricsIndexCDCConsumerSourceImpl extends BaseSourceImpl
+ implements MetricsIndexCDCConsumerSource {
+
+ private final MetricHistogram cdcBatchProcessTimeHisto;
+ private final MetricHistogram cdcMutationGenerateTimeHisto;
+ private final MetricHistogram cdcMutationApplyTimeHisto;
+ private final MutableFastCounter cdcBatchCounter;
+ private final MutableFastCounter cdcMutationCounter;
+ private final MutableFastCounter cdcBatchFailureCounter;
+ private final MetricHistogram cdcIndexUpdateLagHisto;
+
+ public MetricsIndexCDCConsumerSourceImpl() {
+ this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);
+ }
+
+ public MetricsIndexCDCConsumerSourceImpl(String metricsName, String metricsDescription,
+ String metricsContext, String metricsJmxContext) {
+ super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
+
+ cdcBatchProcessTimeHisto =
+ getMetricsRegistry().newHistogram(CDC_BATCH_PROCESS_TIME, CDC_BATCH_PROCESS_TIME_DESC);
+ cdcMutationGenerateTimeHisto = getMetricsRegistry().newHistogram(CDC_MUTATION_GENERATE_TIME,
+ CDC_MUTATION_GENERATE_TIME_DESC);
+ cdcMutationApplyTimeHisto =
+ getMetricsRegistry().newHistogram(CDC_MUTATION_APPLY_TIME, CDC_MUTATION_APPLY_TIME_DESC);
+ cdcBatchCounter = getMetricsRegistry().newCounter(CDC_BATCH_COUNT, CDC_BATCH_COUNT_DESC, 0L);
+ cdcMutationCounter =
+ getMetricsRegistry().newCounter(CDC_MUTATION_COUNT, CDC_MUTATION_COUNT_DESC, 0L);
+ cdcBatchFailureCounter =
+ getMetricsRegistry().newCounter(CDC_BATCH_FAILURE_COUNT, CDC_BATCH_FAILURE_COUNT_DESC, 0L);
+ cdcIndexUpdateLagHisto =
+ getMetricsRegistry().newHistogram(CDC_INDEX_UPDATE_LAG, CDC_INDEX_UPDATE_LAG_DESC);
+ }
+
+ @Override
+ public void updateCdcBatchProcessTime(String dataTableName, long t) {
+ incrementTableSpecificHistogram(CDC_BATCH_PROCESS_TIME, dataTableName, t);
+ cdcBatchProcessTimeHisto.add(t);
+ }
+
+ @Override
+ public void updateCdcMutationGenerateTime(String dataTableName, long t) {
+ incrementTableSpecificHistogram(CDC_MUTATION_GENERATE_TIME, dataTableName, t);
+ cdcMutationGenerateTimeHisto.add(t);
+ }
+
+ @Override
+ public void updateCdcMutationApplyTime(String dataTableName, long t) {
+ incrementTableSpecificHistogram(CDC_MUTATION_APPLY_TIME, dataTableName, t);
+ cdcMutationApplyTimeHisto.add(t);
+ }
+
+ @Override
+ public void incrementCdcBatchCount(String dataTableName) {
+ incrementTableSpecificCounter(CDC_BATCH_COUNT, dataTableName);
+ cdcBatchCounter.incr();
+ }
+
+ @Override
+ public void incrementCdcMutationCount(String dataTableName, long count) {
+ MutableFastCounter tableCounter =
+ getMetricsRegistry().getCounter(getMetricName(CDC_MUTATION_COUNT, dataTableName), 0);
+ tableCounter.incr(count);
+ cdcMutationCounter.incr(count);
+ }
+
+ @Override
+ public void incrementCdcBatchFailureCount(String dataTableName) {
+ incrementTableSpecificCounter(CDC_BATCH_FAILURE_COUNT, dataTableName);
+ cdcBatchFailureCounter.incr();
+ }
+
+ @Override
+ public void updateCdcLag(String dataTableName, long lag) {
+ incrementTableSpecificHistogram(CDC_INDEX_UPDATE_LAG, dataTableName, lag);
+ cdcIndexUpdateLagHisto.add(lag);
+ }
+
+ private void incrementTableSpecificCounter(String baseName, String tableName) {
+ MutableFastCounter tableCounter =
+ getMetricsRegistry().getCounter(getMetricName(baseName, tableName), 0);
+ tableCounter.incr();
+ }
+
+ private void incrementTableSpecificHistogram(String baseName, String tableName, long t) {
+ MetricHistogram tableHistogram =
+ getMetricsRegistry().getHistogram(getMetricName(baseName, tableName));
+ tableHistogram.add(t);
+ }
+
+ private String getMetricName(String baseName, String tableName) {
+ return baseName + "." + tableName;
+ }
+}
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/hbase/index/metrics/MetricsIndexerSourceFactory.java b/phoenix-core-client/src/main/java/org/apache/phoenix/hbase/index/metrics/MetricsIndexerSourceFactory.java
index e579b2f836e..55b648c53a1 100644
--- a/phoenix-core-client/src/main/java/org/apache/phoenix/hbase/index/metrics/MetricsIndexerSourceFactory.java
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/hbase/index/metrics/MetricsIndexerSourceFactory.java
@@ -24,6 +24,7 @@ public class MetricsIndexerSourceFactory {
private static final MetricsIndexerSourceFactory INSTANCE = new MetricsIndexerSourceFactory();
private volatile MetricsIndexerSource indexerSource;
private GlobalIndexCheckerSource globalIndexCheckerSource;
+ private MetricsIndexCDCConsumerSource indexCDCConsumerSource;
private MetricsIndexerSourceFactory() {
}
@@ -45,4 +46,11 @@ public synchronized GlobalIndexCheckerSource getGlobalIndexCheckerSource() {
}
return INSTANCE.globalIndexCheckerSource;
}
+
+ public synchronized MetricsIndexCDCConsumerSource getIndexCDCConsumerSource() {
+ if (INSTANCE.indexCDCConsumerSource == null) {
+ INSTANCE.indexCDCConsumerSource = new MetricsIndexCDCConsumerSourceImpl();
+ }
+ return INSTANCE.indexCDCConsumerSource;
+ }
}
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/index/IndexMaintainer.java b/phoenix-core-client/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
index e2b33a17d7e..70b83d17950 100644
--- a/phoenix-core-client/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
@@ -108,6 +108,7 @@
import org.apache.phoenix.schema.tuple.BaseTuple;
import org.apache.phoenix.schema.tuple.MultiKeyValueTuple;
import org.apache.phoenix.schema.tuple.ValueGetterTuple;
+import org.apache.phoenix.schema.types.IndexConsistency;
import org.apache.phoenix.schema.types.PBoolean;
import org.apache.phoenix.schema.types.PDataType;
import org.apache.phoenix.schema.types.PVarbinaryEncoded;
@@ -477,10 +478,12 @@ public static IndexMaintainer getIndexMaintainer(List maintaine
private Expression indexWhere;
private Set indexWhereColumns;
private boolean isCDCIndex;
+ private IndexConsistency indexConsistency;
protected IndexMaintainer(RowKeySchema dataRowKeySchema, boolean isDataTableSalted) {
this.dataRowKeySchema = dataRowKeySchema;
this.isDataTableSalted = isDataTableSalted;
+ this.indexConsistency = null;
}
private IndexMaintainer(final PTable dataTable, final PTable index, PhoenixConnection connection)
@@ -501,6 +504,7 @@ private IndexMaintainer(final PTable dataTable, final PTable cdcTable, final PTa
this.isUncovered = index.getIndexType() == IndexType.UNCOVERED_GLOBAL;
this.encodingScheme = index.getEncodingScheme();
this.isCDCIndex = CDCUtil.isCDCIndex(index);
+ this.indexConsistency = index.getIndexConsistency();
// null check for b/w compatibility
this.encodingScheme = index.getEncodingScheme() == null
@@ -1933,6 +1937,7 @@ public PDataType getDataType() {
this.encodingScheme = QualifierEncodingScheme.NON_ENCODED_QUALIFIERS;
this.dataImmutableStorageScheme = ImmutableStorageScheme.ONE_CELL_PER_COLUMN;
this.dataEncodingScheme = QualifierEncodingScheme.NON_ENCODED_QUALIFIERS;
+ this.indexConsistency = null;
initCachedState();
}
@@ -2068,6 +2073,11 @@ public static IndexMaintainer fromProto(ServerCachingProtos.IndexMaintainer prot
} else {
maintainer.isCDCIndex = false;
}
+ if (proto.hasIndexConsistency()) {
+ maintainer.indexConsistency = IndexConsistency.valueOf(proto.getIndexConsistency());
+ } else {
+ maintainer.indexConsistency = null;
+ }
maintainer.nDataTableSaltBuckets =
proto.hasDataTableSaltBuckets() ? proto.getDataTableSaltBuckets() : -1;
maintainer.initCachedState();
@@ -2223,6 +2233,9 @@ public static ServerCachingProtos.IndexMaintainer toProto(IndexMaintainer mainta
}
}
builder.setIsCDCIndex(maintainer.isCDCIndex);
+ if (maintainer.indexConsistency != null) {
+ builder.setIndexConsistency(maintainer.indexConsistency.name());
+ }
if (maintainer.isDataTableSalted) {
builder.setDataTableSaltBuckets(maintainer.nDataTableSaltBuckets);
}
@@ -2609,6 +2622,10 @@ public Set getIndexedColumns() {
return indexedColumns;
}
+ public IndexConsistency getIndexConsistency() {
+ return indexConsistency;
+ }
+
public static class UDFParseNodeVisitor extends StatelessTraverseAllParseNodeVisitor {
private Map udfParseNodes;
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
index 3bc68c5ddb7..ed63485440f 100644
--- a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
@@ -213,6 +213,8 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
public static final byte[] VIEW_TYPE_BYTES = Bytes.toBytes(VIEW_TYPE);
public static final String INDEX_TYPE = "INDEX_TYPE";
public static final byte[] INDEX_TYPE_BYTES = Bytes.toBytes(INDEX_TYPE);
+ public static final String INDEX_CONSISTENCY = "INDEX_CONSISTENCY";
+ public static final byte[] INDEX_CONSISTENCY_BYTES = Bytes.toBytes(INDEX_CONSISTENCY);
public static final String LINK_TYPE = "LINK_TYPE";
public static final byte[] LINK_TYPE_BYTES = Bytes.toBytes(LINK_TYPE);
public static final String TASK_TYPE = "TASK_TYPE";
@@ -498,6 +500,16 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
"PARTITION_END_TIME IS NOT NULL "
+ "AND TO_NUMBER(CURRENT_TIME()) - TO_NUMBER(PHOENIX_ROW_TIMESTAMP()) >= %d";
+ public static final String SYSTEM_IDX_CDC_TRACKER_TABLE = "IDX_CDC_TRACKER";
+ public static final String SYSTEM_IDX_CDC_TRACKER_NAME =
+ SchemaUtil.getTableName(SYSTEM_CATALOG_SCHEMA, SYSTEM_IDX_CDC_TRACKER_TABLE);
+ public static final String LAST_TIMESTAMP = "LAST_TIMESTAMP";
+ public static final String OWNER_PARTITION_ID = "OWNER_PARTITION_ID";
+ public static final String TRACKER_STATUS = "STATUS";
+ public static final String TRACKER_STATUS_IN_PROGRESS = "i";
+ public static final String TRACKER_STATUS_COMPLETE = "c";
+ public static final int IDX_CDC_TRACKER_TTL = 10 * 24 * 60 * 60;
+
public static final String QUERY_ID = "QUERY_ID";
public static final String USER = "USER";
public static final String CLIENT_IP = "CLIENT_IP";
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
index 136d7a2d621..e8c89451b2c 100644
--- a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
@@ -220,6 +220,7 @@
import org.apache.phoenix.schema.tuple.MultiKeyValueTuple;
import org.apache.phoenix.schema.tuple.ResultTuple;
import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.IndexConsistency;
import org.apache.phoenix.schema.types.PDataType;
import org.apache.phoenix.schema.types.PLong;
import org.apache.phoenix.schema.types.PVarchar;
@@ -1623,9 +1624,10 @@ private static class ExecutableCreateIndexStatement extends CreateIndexStatement
public ExecutableCreateIndexStatement(NamedNode indexName, NamedTableNode dataTable,
IndexKeyConstraint ikConstraint, List includeColumns, List splits,
ListMultimap> props, boolean ifNotExists, IndexType indexType,
- boolean async, int bindCount, Map udfParseNodes, ParseNode where) {
+ boolean async, int bindCount, Map udfParseNodes, ParseNode where,
+ IndexConsistency indexConsistency) {
super(indexName, dataTable, ikConstraint, includeColumns, splits, props, ifNotExists,
- indexType, async, bindCount, udfParseNodes, where);
+ indexType, async, bindCount, udfParseNodes, where, indexConsistency);
}
@SuppressWarnings("unchecked")
@@ -1857,12 +1859,19 @@ public MutationState execute() throws SQLException {
private static class ExecutableAlterIndexStatement extends AlterIndexStatement
implements CompilableStatement {
- public ExecutableAlterIndexStatement(NamedTableNode indexTableNode, String dataTableName,
+ ExecutableAlterIndexStatement(NamedTableNode indexTableNode, String dataTableName,
boolean ifExists, PIndexState state, boolean isRebuildAll, boolean async,
ListMultimap> props) {
super(indexTableNode, dataTableName, ifExists, state, isRebuildAll, async, props);
}
+ ExecutableAlterIndexStatement(NamedTableNode indexTableNode, String dataTableName,
+ boolean ifExists, PIndexState state, boolean isRebuildAll, boolean async,
+ ListMultimap> props, IndexConsistency indexConsistency) {
+ super(indexTableNode, dataTableName, ifExists, state, isRebuildAll, async, props,
+ indexConsistency);
+ }
+
@SuppressWarnings("unchecked")
@Override
public MutationPlan compilePlan(final PhoenixStatement stmt, Sequence.ValueOp seqAction)
@@ -2248,7 +2257,8 @@ public CreateIndexStatement createIndex(NamedNode indexName, NamedTableNode data
ListMultimap> props, boolean ifNotExists, IndexType indexType,
boolean async, int bindCount, Map udfParseNodes, ParseNode where) {
return new ExecutableCreateIndexStatement(indexName, dataTable, ikConstraint, includeColumns,
- splits, props, ifNotExists, indexType, async, bindCount, udfParseNodes, where);
+ splits, props, ifNotExists, indexType, async, bindCount, udfParseNodes, where,
+ CreateIndexStatement.getIndexConsistency(props));
}
@Override
@@ -2305,6 +2315,14 @@ public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String data
isRebuildAll, async, props);
}
+ @Override
+ public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName,
+ boolean ifExists, PIndexState state, boolean isRebuildAll, boolean async,
+ ListMultimap> props, IndexConsistency indexConsistency) {
+ return new ExecutableAlterIndexStatement(indexTableNode, dataTableName, ifExists, state,
+ isRebuildAll, async, props, indexConsistency);
+ }
+
@Override
public TraceStatement trace(boolean isTraceOn, double samplingRate) {
return new ExecutableTraceStatement(isTraceOn, samplingRate);
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java b/phoenix-core-client/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java
index 7ef5a618286..2b12e32ae6a 100644
--- a/phoenix-core-client/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java
@@ -20,6 +20,7 @@
import org.apache.hadoop.hbase.util.Pair;
import org.apache.phoenix.schema.PIndexState;
import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.types.IndexConsistency;
import org.apache.phoenix.thirdparty.com.google.common.collect.ImmutableListMultimap;
import org.apache.phoenix.thirdparty.com.google.common.collect.ListMultimap;
@@ -31,16 +32,23 @@ public class AlterIndexStatement extends SingleTableStatement {
private boolean async;
private boolean isRebuildAll;
private ListMultimap> props;
+ private final IndexConsistency indexConsistency;
private static final PTableType tableType = PTableType.INDEX;
public AlterIndexStatement(NamedTableNode indexTableNode, String dataTableName, boolean ifExists,
PIndexState indexState, boolean isRebuildAll, boolean async) {
- this(indexTableNode, dataTableName, ifExists, indexState, isRebuildAll, async, null);
+ this(indexTableNode, dataTableName, ifExists, indexState, isRebuildAll, async, null, null);
}
public AlterIndexStatement(NamedTableNode indexTableNode, String dataTableName, boolean ifExists,
PIndexState indexState, boolean isRebuildAll, boolean async,
ListMultimap> props) {
+ this(indexTableNode, dataTableName, ifExists, indexState, isRebuildAll, async, props, null);
+ }
+
+ public AlterIndexStatement(NamedTableNode indexTableNode, String dataTableName, boolean ifExists,
+ PIndexState indexState, boolean isRebuildAll, boolean async,
+ ListMultimap> props, IndexConsistency indexConsistency) {
super(indexTableNode, 0);
this.dataTableName = dataTableName;
this.ifExists = ifExists;
@@ -48,6 +56,7 @@ public AlterIndexStatement(NamedTableNode indexTableNode, String dataTableName,
this.async = async;
this.isRebuildAll = isRebuildAll;
this.props = props == null ? ImmutableListMultimap.> of() : props;
+ this.indexConsistency = indexConsistency;
}
public String getTableName() {
@@ -82,4 +91,8 @@ public ListMultimap> getProps() {
public PTableType getTableType() {
return tableType;
}
+
+ public IndexConsistency getIndexConsistency() {
+ return indexConsistency;
+ }
}
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/parse/CreateIndexStatement.java b/phoenix-core-client/src/main/java/org/apache/phoenix/parse/CreateIndexStatement.java
index ec71fb10246..8cc7685a313 100644
--- a/phoenix-core-client/src/main/java/org/apache/phoenix/parse/CreateIndexStatement.java
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/parse/CreateIndexStatement.java
@@ -21,7 +21,9 @@
import java.util.List;
import java.util.Map;
import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.query.QueryConstants;
import org.apache.phoenix.schema.PTable.IndexType;
+import org.apache.phoenix.schema.types.IndexConsistency;
import org.apache.phoenix.thirdparty.com.google.common.collect.ArrayListMultimap;
import org.apache.phoenix.thirdparty.com.google.common.collect.ListMultimap;
@@ -37,11 +39,21 @@ public class CreateIndexStatement extends SingleTableStatement {
private final boolean async;
private final Map udfParseNodes;
private final ParseNode where;
+ private final IndexConsistency indexConsistency;
public CreateIndexStatement(NamedNode indexTableName, NamedTableNode dataTable,
IndexKeyConstraint indexKeyConstraint, List includeColumns, List splits,
ListMultimap> props, boolean ifNotExists, IndexType indexType,
boolean async, int bindCount, Map udfParseNodes, ParseNode where) {
+ this(indexTableName, dataTable, indexKeyConstraint, includeColumns, splits, props, ifNotExists,
+ indexType, async, bindCount, udfParseNodes, where, getIndexConsistency(props));
+ }
+
+ public CreateIndexStatement(NamedNode indexTableName, NamedTableNode dataTable,
+ IndexKeyConstraint indexKeyConstraint, List includeColumns, List splits,
+ ListMultimap> props, boolean ifNotExists, IndexType indexType,
+ boolean async, int bindCount, Map udfParseNodes, ParseNode where,
+ IndexConsistency indexConsistency) {
super(dataTable, bindCount);
this.indexTableName =
TableName.create(dataTable.getName().getSchemaName(), indexTableName.getName());
@@ -56,6 +68,7 @@ public CreateIndexStatement(NamedNode indexTableName, NamedTableNode dataTable,
this.async = async;
this.udfParseNodes = udfParseNodes;
this.where = where;
+ this.indexConsistency = indexConsistency;
}
public CreateIndexStatement(CreateIndexStatement createStmt,
@@ -71,6 +84,23 @@ public CreateIndexStatement(CreateIndexStatement createStmt,
this.async = createStmt.isAsync();
this.udfParseNodes = createStmt.getUdfParseNodes();
this.where = createStmt.where;
+ this.indexConsistency = createStmt.getIndexConsistency();
+ }
+
+ public static IndexConsistency
+ getIndexConsistency(ListMultimap> props) {
+ IndexConsistency indexConsistency = null;
+ if (props != null) {
+ for (Pair prop : props.get(QueryConstants.ALL_FAMILY_PROPERTIES_KEY)) {
+ if (prop != null && "CONSISTENCY".equalsIgnoreCase(prop.getFirst())) {
+ Object value = prop.getSecond();
+ indexConsistency =
+ value == null ? null : IndexConsistency.valueOf(value.toString().toUpperCase());
+ break;
+ }
+ }
+ }
+ return indexConsistency;
}
public IndexKeyConstraint getIndexConstraint() {
@@ -112,4 +142,8 @@ public Map getUdfParseNodes() {
public ParseNode getWhere() {
return where;
}
+
+ public IndexConsistency getIndexConsistency() {
+ return indexConsistency;
+ }
}
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java b/phoenix-core-client/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
index f3c373601c9..163b99f8735 100644
--- a/phoenix-core-client/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
@@ -49,6 +49,7 @@
import org.apache.phoenix.schema.SortOrder;
import org.apache.phoenix.schema.TypeMismatchException;
import org.apache.phoenix.schema.stats.StatisticsCollectionScope;
+import org.apache.phoenix.schema.types.IndexConsistency;
import org.apache.phoenix.schema.types.PBinary;
import org.apache.phoenix.schema.types.PDataType;
import org.apache.phoenix.schema.types.PLong;
@@ -470,6 +471,13 @@ public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String data
async, props);
}
+ public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName,
+ boolean ifExists, PIndexState state, boolean isRebuildAll, boolean async,
+ ListMultimap> props, IndexConsistency indexConsistency) {
+ return new AlterIndexStatement(indexTableNode, dataTableName, ifExists, state, isRebuildAll,
+ async, props, indexConsistency);
+ }
+
public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName,
boolean ifExists, PIndexState state) {
return new AlterIndexStatement(indexTableNode, dataTableName, ifExists, state, false, false);
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 8e867842bc4..0f19965030a 100644
--- a/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -60,6 +60,7 @@
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CDC_STREAM_STATUS_NAME;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_FUNCTION_HBASE_TABLE_NAME;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_IDX_CDC_TRACKER_NAME;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_MUTEX_COLUMN_NAME_BYTES;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_MUTEX_FAMILY_NAME_BYTES;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_MUTEX_NAME;
@@ -4101,6 +4102,10 @@ protected String getCDCStreamDDL() {
return ddl + ",TTL='" + ttlExpression + "'";
}
+ protected String getIdxCdcTrackerDDL() {
+ return setSystemDDLProperties(QueryConstants.CREATE_IDX_CDC_TRACKER_METADATA);
+ }
+
private String setSystemDDLProperties(String ddl) {
return String.format(ddl,
props.getInt(DEFAULT_SYSTEM_MAX_VERSIONS_ATTRIB,
@@ -4434,6 +4439,10 @@ private void createOtherSystemTables(PhoenixConnection metaConnection)
metaConnection.createStatement().executeUpdate(getCDCStreamDDL());
} catch (TableAlreadyExistsException ignore) {
}
+ try {
+ metaConnection.createStatement().executeUpdate(getIdxCdcTrackerDDL());
+ } catch (TableAlreadyExistsException ignore) {
+ }
}
/**
@@ -4755,22 +4764,22 @@ protected PhoenixConnection upgradeSystemCatalogIfRequired(PhoenixConnection met
}
if (currentServerSideTableTimeStamp < MIN_SYSTEM_TABLE_TIMESTAMP_5_3_0) {
metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
- MIN_SYSTEM_TABLE_TIMESTAMP_5_3_0 - 8,
+ MIN_SYSTEM_TABLE_TIMESTAMP_5_3_0 - 9,
PhoenixDatabaseMetaData.PHYSICAL_TABLE_NAME + " " + PVarchar.INSTANCE.getSqlTypeName());
metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
- MIN_SYSTEM_TABLE_TIMESTAMP_5_3_0 - 7,
+ MIN_SYSTEM_TABLE_TIMESTAMP_5_3_0 - 8,
PhoenixDatabaseMetaData.SCHEMA_VERSION + " " + PVarchar.INSTANCE.getSqlTypeName());
metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
- MIN_SYSTEM_TABLE_TIMESTAMP_5_3_0 - 6,
+ MIN_SYSTEM_TABLE_TIMESTAMP_5_3_0 - 7,
PhoenixDatabaseMetaData.EXTERNAL_SCHEMA_ID + " " + PVarchar.INSTANCE.getSqlTypeName());
metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
- MIN_SYSTEM_TABLE_TIMESTAMP_5_3_0 - 5,
+ MIN_SYSTEM_TABLE_TIMESTAMP_5_3_0 - 6,
PhoenixDatabaseMetaData.STREAMING_TOPIC_NAME + " " + PVarchar.INSTANCE.getSqlTypeName());
metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
- MIN_SYSTEM_TABLE_TIMESTAMP_5_3_0 - 4,
+ MIN_SYSTEM_TABLE_TIMESTAMP_5_3_0 - 5,
PhoenixDatabaseMetaData.INDEX_WHERE + " " + PVarchar.INSTANCE.getSqlTypeName());
metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
- MIN_SYSTEM_TABLE_TIMESTAMP_5_3_0 - 3,
+ MIN_SYSTEM_TABLE_TIMESTAMP_5_3_0 - 4,
PhoenixDatabaseMetaData.CDC_INCLUDE_TABLE + " " + PVarchar.INSTANCE.getSqlTypeName());
/**
@@ -4778,19 +4787,16 @@ protected PhoenixConnection upgradeSystemCatalogIfRequired(PhoenixConnection met
* PHOENIX_TTL Column. See PHOENIX-7023
*/
metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
- MIN_SYSTEM_TABLE_TIMESTAMP_5_3_0 - 2,
+ MIN_SYSTEM_TABLE_TIMESTAMP_5_3_0 - 3,
PhoenixDatabaseMetaData.TTL + " " + PVarchar.INSTANCE.getSqlTypeName());
metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
- MIN_SYSTEM_TABLE_TIMESTAMP_5_3_0 - 1,
+ MIN_SYSTEM_TABLE_TIMESTAMP_5_3_0 - 2,
PhoenixDatabaseMetaData.ROW_KEY_MATCHER + " " + PVarbinary.INSTANCE.getSqlTypeName());
metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
- MIN_SYSTEM_TABLE_TIMESTAMP_5_3_0,
+ MIN_SYSTEM_TABLE_TIMESTAMP_5_3_0 - 1,
PhoenixDatabaseMetaData.IS_STRICT_TTL + " " + PBoolean.INSTANCE.getSqlTypeName());
- // Values in PHOENIX_TTL column will not be used for further release as PHOENIX_TTL column is
- // being deprecated
- // and will be removed in later release. To copy copyDataFromPhoenixTTLtoTTL(metaConnection)
- // can be used but
- // as that feature was not fully built we are not moving old value to new column
+ metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+ MIN_SYSTEM_TABLE_TIMESTAMP_5_3_0, PhoenixDatabaseMetaData.INDEX_CONSISTENCY + " CHAR(1)");
// move TTL values stored in descriptor to SYSCAT TTL column.
moveTTLFromHBaseLevelTTLToPhoenixLevelTTL(metaConnection);
@@ -5009,6 +5015,7 @@ private PhoenixConnection upgradeOtherSystemTablesIfRequired(PhoenixConnection m
metaConnection = upgradeSystemMutex(metaConnection);
metaConnection = upgradeSystemCDCStreamStatus(metaConnection);
metaConnection = upgradeSystemCDCStream(metaConnection);
+ metaConnection = upgradeSystemIdxCdcTracker(metaConnection);
// As this is where the most time will be spent during an upgrade,
// especially when there are large number of views.
@@ -5331,6 +5338,15 @@ private PhoenixConnection upgradeSystemCDCStream(PhoenixConnection metaConnectio
return metaConnection;
}
+ private PhoenixConnection upgradeSystemIdxCdcTracker(PhoenixConnection metaConnection)
+ throws SQLException {
+ try {
+ metaConnection.createStatement().executeUpdate(getIdxCdcTrackerDDL());
+ } catch (TableAlreadyExistsException ignored) {
+ }
+ return metaConnection;
+ }
+
// Special method for adding the column qualifier column for 4.10.
private PhoenixConnection addColumnQualifierColumn(PhoenixConnection oldMetaConnection,
Long timestamp) throws SQLException {
@@ -6931,6 +6947,8 @@ public void deleteAllStreamMetadataForTable(java.sql.Connection conn, String tab
"DELETE FROM " + SYSTEM_CDC_STREAM_STATUS_NAME + " WHERE TABLE_NAME = ?";
String deleteStreamPartitionsQuery =
"DELETE FROM " + SYSTEM_CDC_STREAM_NAME + " WHERE TABLE_NAME = ?";
+ String deleteIdxCdcTrackerQuery =
+ "DELETE FROM " + SYSTEM_IDX_CDC_TRACKER_NAME + " WHERE TABLE_NAME = ?";
LOGGER.info("Deleting Stream Metadata for table {}", tableName);
try (PreparedStatement ps = conn.prepareStatement(deleteStreamStatusQuery)) {
ps.setString(1, tableName);
@@ -6942,6 +6960,11 @@ public void deleteAllStreamMetadataForTable(java.sql.Connection conn, String tab
ps.executeUpdate();
conn.commit();
}
+ try (PreparedStatement ps = conn.prepareStatement(deleteIdxCdcTrackerQuery)) {
+ ps.setString(1, tableName);
+ ps.executeUpdate();
+ conn.commit();
+ }
}
/**
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java b/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
index ad5df1e39e2..2e62e0117ff 100644
--- a/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
@@ -219,6 +219,10 @@ protected String getCDCStreamDDL() {
return setSystemDDLProperties(QueryConstants.CREATE_CDC_STREAM_METADATA);
}
+ protected String getIdxCdcTrackerDDL() {
+ return setSystemDDLProperties(QueryConstants.CREATE_IDX_CDC_TRACKER_METADATA);
+ }
+
private String setSystemDDLProperties(String ddl) {
return String.format(ddl,
props.getInt(DEFAULT_SYSTEM_MAX_VERSIONS_ATTRIB,
@@ -496,6 +500,10 @@ public void init(String url, Properties props) throws SQLException {
metaConnection.createStatement().executeUpdate(getCDCStreamDDL());
} catch (TableAlreadyExistsException ignore) {
}
+ try {
+ metaConnection.createStatement().executeUpdate(getIdxCdcTrackerDDL());
+ } catch (TableAlreadyExistsException ignore) {
+ }
} catch (SQLException e) {
sqlE = e;
} finally {
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryConstants.java b/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryConstants.java
index fe9ea6ef8bf..aa73c833145 100644
--- a/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryConstants.java
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryConstants.java
@@ -54,9 +54,11 @@
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POST_KEY;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IDX_CDC_TRACKER_TTL;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_STORAGE_SCHEME;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INCREMENT_BY;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_CONSISTENCY;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_TYPE;
@@ -73,6 +75,7 @@
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.KEY_SEQ;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LAST_DDL_TIMESTAMP;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LAST_STATS_UPDATE_TIME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LAST_TIMESTAMP;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LIMIT_REACHED_FLAG;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LOGICAL_PARENT_NAME;
@@ -88,6 +91,7 @@
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.NUM_PREC_RADIX;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.OLD_METADATA;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ORDINAL_POSITION;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.OWNER_PARTITION_ID;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PARENT_PARTITION_ID;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PARENT_PARTITION_START_TIME;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PARTITION_END_KEY;
@@ -133,6 +137,7 @@
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CDC_STREAM_TABLE;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_TABLE;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_FUNCTION_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_IDX_CDC_TRACKER_TABLE;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_LOG_TABLE;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_MUTEX_TABLE_NAME;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE;
@@ -150,6 +155,7 @@
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TASK_TS;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TASK_TYPE;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TENANT_ID;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TRACKER_STATUS;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TRANSACTIONAL;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TRANSACTION_PROVIDER;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TRANSFORM_FUNCTION;
@@ -376,12 +382,17 @@ enum JoinType {
String CDC_PRE_IMAGE = "pre_image";
String CDC_POST_IMAGE = "post_image";
String CDC_CHANGE_IMAGE = "change_image";
+ String CDC_IDX_MUTATIONS = "idx_mutations";
String CDC_UPSERT_EVENT_TYPE = "upsert";
String CDC_DELETE_EVENT_TYPE = "delete";
String SPLITS_FILE = "SPLITS_FILE";
String CDC_TTL_DELETE_EVENT_TYPE = "ttl_delete";
String CDC_IMAGE_CQ = "_CDC_IMG_";
byte[] CDC_IMAGE_CQ_BYTES = Bytes.toBytes(CDC_IMAGE_CQ);
+ String CDC_INDEX_PRE_MUTATIONS_CQ = "_IDX_PRE_";
+ byte[] CDC_INDEX_PRE_MUTATIONS_CQ_BYTES = Bytes.toBytes(CDC_INDEX_PRE_MUTATIONS_CQ);
+ String CDC_INDEX_POST_MUTATIONS_CQ = "_IDX_POST_";
+ byte[] CDC_INDEX_POST_MUTATIONS_CQ_BYTES = Bytes.toBytes(CDC_INDEX_POST_MUTATIONS_CQ);
/**
* We mark counter values 0 to 10 as reserved. Value 0 is used by
@@ -408,7 +419,7 @@ enum JoinType {
+ " BOOLEAN, \n" + SCHEMA_VERSION + " VARCHAR, \n" + EXTERNAL_SCHEMA_ID + " VARCHAR, \n"
+ STREAMING_TOPIC_NAME + " VARCHAR, \n" + INDEX_WHERE + " VARCHAR, \n" + CDC_INCLUDE_TABLE
+ " VARCHAR, \n" + TTL + " VARCHAR, \n" + ROW_KEY_MATCHER + " VARBINARY_ENCODED, \n"
- + IS_STRICT_TTL + " BOOLEAN, \n" +
+ + IS_STRICT_TTL + " BOOLEAN, \n" + INDEX_CONSISTENCY + " CHAR(1), \n" +
// Column metadata (will be null for table row)
DATA_TYPE + " INTEGER," + COLUMN_SIZE + " INTEGER," + DECIMAL_DIGITS + " INTEGER," + NULLABLE
+ " INTEGER," + ORDINAL_POSITION + " INTEGER," + SORT_ORDER + " INTEGER," + ARRAY_SIZE
@@ -586,4 +597,18 @@ enum JoinType {
+ PARENT_PARTITION_ID + "))\n" + HConstants.VERSIONS + "=%s,\n"
+ ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" + TRANSACTIONAL + "="
+ Boolean.FALSE + ",\n" + UPDATE_CACHE_FREQUENCY + "=" + "7200000";
+
+ String CREATE_IDX_CDC_TRACKER_METADATA =
+ "CREATE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_IDX_CDC_TRACKER_TABLE + "\"(\n" +
+ // PK columns
+ TABLE_NAME + " VARCHAR NOT NULL," + PARTITION_ID + " VARCHAR NOT NULL," + OWNER_PARTITION_ID
+ + " VARCHAR NOT NULL," +
+ // Non-PK columns
+ LAST_TIMESTAMP + " BIGINT," + TRACKER_STATUS + " CHAR(1),\n" + "CONSTRAINT "
+ + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TABLE_NAME + "," + PARTITION_ID + ","
+ + OWNER_PARTITION_ID + "))\n" + HConstants.VERSIONS + "=%s,\n"
+ + ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" + TRANSACTIONAL + "="
+ + Boolean.FALSE + ",\n" + UPDATE_CACHE_FREQUENCY + "=" + "7200000" + ",\n"
+ + ColumnFamilyDescriptorBuilder.TTL + "=" + IDX_CDC_TRACKER_TTL + ",\n"
+ + "\"phoenix.max.lookback.age.seconds\"=0";
}
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/DelegateTable.java b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/DelegateTable.java
index 44df43c929a..b41da687826 100644
--- a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/DelegateTable.java
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/DelegateTable.java
@@ -28,6 +28,7 @@
import org.apache.phoenix.index.IndexMaintainer;
import org.apache.phoenix.jdbc.PhoenixConnection;
import org.apache.phoenix.schema.transform.TransformMaintainer;
+import org.apache.phoenix.schema.types.IndexConsistency;
import org.apache.phoenix.schema.types.PDataType;
import org.apache.phoenix.transaction.TransactionFactory;
@@ -283,6 +284,11 @@ public IndexType getIndexType() {
return delegate.getIndexType();
}
+ @Override
+ public IndexConsistency getIndexConsistency() {
+ return delegate.getIndexConsistency();
+ }
+
private final PTable delegate;
public DelegateTable(PTable delegate) {
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index 93c9cdd39bb..530c5612ac2 100644
--- a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -57,6 +57,7 @@
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_STORAGE_SCHEME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_CONSISTENCY;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_TYPE;
@@ -276,6 +277,7 @@
import org.apache.phoenix.schema.task.SystemTaskParams;
import org.apache.phoenix.schema.task.Task;
import org.apache.phoenix.schema.transform.TransformClient;
+import org.apache.phoenix.schema.types.IndexConsistency;
import org.apache.phoenix.schema.types.PBson;
import org.apache.phoenix.schema.types.PDataType;
import org.apache.phoenix.schema.types.PDate;
@@ -344,9 +346,9 @@ public class MetaDataClient {
+ IMMUTABLE_STORAGE_SCHEME + "," + ENCODING_SCHEME + "," + USE_STATS_FOR_PARALLELIZATION + ","
+ VIEW_INDEX_ID_DATA_TYPE + "," + CHANGE_DETECTION_ENABLED + "," + PHYSICAL_TABLE_NAME + ","
+ SCHEMA_VERSION + "," + STREAMING_TOPIC_NAME + "," + INDEX_WHERE + "," + CDC_INCLUDE_TABLE
- + "," + TTL + "," + ROW_KEY_MATCHER + "," + IS_STRICT_TTL
+ + "," + TTL + "," + ROW_KEY_MATCHER + "," + IS_STRICT_TTL + "," + INDEX_CONSISTENCY
+ ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, "
- + "?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+ + "?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
private static final String CREATE_SCHEMA = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\""
+ SYSTEM_CATALOG_TABLE + "\"( " + TABLE_SCHEM + "," + TABLE_NAME + ") VALUES (?,?)";
@@ -398,6 +400,10 @@ public class MetaDataClient {
+ INDEX_STATE + "," + INDEX_DISABLE_TIMESTAMP + "," + ASYNC_REBUILD_TIMESTAMP + " "
+ PLong.INSTANCE.getSqlTypeName() + ") VALUES (?, ?, ?, ?, ?, ?)";
+ public static final String UPDATE_INDEX_CONSISTENCY =
+ "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " + TENANT_ID + ","
+ + TABLE_SCHEM + "," + TABLE_NAME + "," + INDEX_CONSISTENCY + ") VALUES (?, ?, ?, ?)";
+
/*
* Custom sql to add a column to SYSTEM.CATALOG table during upgrade. We can't use the regular
* ColumnMetaDataOps.UPSERT_COLUMN sql because the COLUMN_QUALIFIER column was added in 4.10. And
@@ -1062,7 +1068,7 @@ public MutationState createTable(CreateTableStatement statement, byte[][] splits
}
}
table = createTableInternal(statement, splits, parent, viewStatement, viewType, viewIndexIdType,
- rowKeyMatcher, viewColumnConstants, isViewColumnReferenced, false, null, null, null,
+ rowKeyMatcher, viewColumnConstants, isViewColumnReferenced, false, null, null, null, null,
tableProps, commonFamilyProps);
if (table == null || table.getType() == PTableType.VIEW || statement.isNoVerify() /*
@@ -1191,6 +1197,10 @@ private void populatePropertyMaps(ListMultimap> sta
continue;
}
+ if (tableType == PTableType.INDEX && "CONSISTENCY".equalsIgnoreCase(prop.getFirst())) {
+ continue;
+ }
+
// HTableDescriptor property or Phoenix Table Property
if (defaultDescriptor.getValue(Bytes.toBytes(prop.getFirst())) == null) {
// See PHOENIX-4891
@@ -1895,7 +1905,7 @@ public MutationState createIndex(CreateIndexStatement statement, byte[][] splits
statement.ifNotExists(), null, statement.getWhere(), statement.getBindCount(), null);
table = createTableInternal(tableStatement, splits, dataTable, null, null,
getViewIndexDataType(), null, null, null, allocateIndexId, statement.getIndexType(),
- asyncCreatedDate, null, tableProps, commonFamilyProps);
+ statement.getIndexConsistency(), asyncCreatedDate, null, tableProps, commonFamilyProps);
} finally {
deleteMutexCells(physicalSchemaName, physicalTableName, acquiredColumnMutexSet);
}
@@ -1905,6 +1915,13 @@ public MutationState createIndex(CreateIndexStatement statement, byte[][] splits
if (LOGGER.isInfoEnabled())
LOGGER.info("Created index " + table.getName().getString() + " at " + table.getTimeStamp());
+
+ if (
+ statement.getIndexConsistency() != null && statement.getIndexConsistency().isAsynchronous()
+ ) {
+ createCDCForEventuallyConsistentIndex(dataTable);
+ }
+
boolean asyncIndexBuildEnabled =
connection.getQueryServices().getProps().getBoolean(QueryServices.INDEX_ASYNC_BUILD_ENABLED,
QueryServicesOptions.DEFAULT_INDEX_ASYNC_BUILD_ENABLED);
@@ -2018,7 +2035,7 @@ public MutationState createCDC(CreateCDCStatement statement) throws SQLException
columnDefs, FACTORY.primaryKey(null, pkColumnDefs), Collections.emptyList(), PTableType.CDC,
statement.isIfNotExists(), null, null, statement.getBindCount(), null);
createTableInternal(tableStatement, null, dataTable, null, null, null, null, null, null, false,
- null, null, statement.getIncludeScopes(), tableProps, commonFamilyProps);
+ null, null, null, statement.getIncludeScopes(), tableProps, commonFamilyProps);
// for now, only track stream partition metadata for tables, TODO: updatable views
if (PTableType.TABLE == dataTable.getType()) {
updateStreamPartitionMetadata(dataTableFullName, cdcObjName);
@@ -2441,12 +2458,37 @@ private boolean checkIfParentIsTable(PTable view) {
return parentName.getString().equals(view.getPhysicalName().getString());
}
+ /**
+ * Creates a CDC stream on the data table for eventually consistent indexes if not already
+ * present.
+ * @param dataTable The data table PTable object for which to create the CDC stream.
+ * @throws SQLException If there is an error executing the CREATE CDC statement or accessing the
+ * database connection.
+ */
+ private void createCDCForEventuallyConsistentIndex(PTable dataTable) throws SQLException {
+ String dataTableName = dataTable.getTableName().getString();
+ String schemaName = dataTable.getSchemaName().getString();
+ String cdcName = "CDC_" + dataTableName;
+ String fullTableName = SchemaUtil.getCaseSensitiveColumnDisplayName(schemaName, dataTableName);
+ String cdcSql = "CREATE CDC IF NOT EXISTS \"" + cdcName + "\" ON " + fullTableName;
+ try {
+ connection.createStatement().execute(cdcSql);
+ } catch (SQLException e) {
+ if (SQLExceptionCode.CDC_ALREADY_ENABLED.getErrorCode() != e.getErrorCode()) {
+ LOGGER.error("Error while creating CDC for {}", fullTableName, e);
+ throw e;
+ }
+ LOGGER.debug("CDC already exists for {}", fullTableName, e);
+ }
+ }
+
private PTable createTableInternal(CreateTableStatement statement, byte[][] splits,
final PTable parent, String viewStatement, ViewType viewType, PDataType viewIndexIdType,
final byte[] rowKeyMatcher, final byte[][] viewColumnConstants,
final BitSet isViewColumnReferenced, boolean allocateIndexId, IndexType indexType,
- Date asyncCreatedDate, Set cdcIncludeScopes,
- Map tableProps, Map commonFamilyProps) throws SQLException {
+ IndexConsistency indexConsistency, Date asyncCreatedDate,
+ Set cdcIncludeScopes, Map tableProps,
+ Map commonFamilyProps) throws SQLException {
final PTableType tableType = statement.getTableType();
boolean wasAutoCommit = connection.getAutoCommit();
TableName tableNameNode = null;
@@ -3396,9 +3438,10 @@ else if (
.setTimeStamp(MetaDataProtocol.MIN_TABLE_TIMESTAMP).setIndexDisableTimestamp(0L)
.setSequenceNumber(PTable.INITIAL_SEQ_NUM).setImmutableRows(isImmutableRows)
.setDisableWAL(Boolean.TRUE.equals(disableWAL)).setMultiTenant(false).setStoreNulls(false)
- .setViewIndexIdType(viewIndexIdType).setIndexType(indexType).setUpdateCacheFrequency(0)
- .setNamespaceMapped(isNamespaceMapped).setAutoPartitionSeqName(autoPartitionSeq)
- .setAppendOnlySchema(isAppendOnlySchema).setImmutableStorageScheme(ONE_CELL_PER_COLUMN)
+ .setViewIndexIdType(viewIndexIdType).setIndexType(indexType).setIndexConsistency(null)
+ .setUpdateCacheFrequency(0).setNamespaceMapped(isNamespaceMapped)
+ .setAutoPartitionSeqName(autoPartitionSeq).setAppendOnlySchema(isAppendOnlySchema)
+ .setImmutableStorageScheme(ONE_CELL_PER_COLUMN)
.setQualifierEncodingScheme(NON_ENCODED_QUALIFIERS)
.setBaseColumnCount(QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT)
.setEncodedCQCounter(PTable.EncodedCQCounter.NULL_COUNTER)
@@ -3703,6 +3746,16 @@ public boolean isViewReferenced() {
tableUpsert.setBoolean(38, isStrictTTL);
+ if (tableType == PTableType.INDEX) {
+ if (indexConsistency == null) {
+ tableUpsert.setNull(39, Types.CHAR);
+ } else {
+ tableUpsert.setString(39, indexConsistency.getCodeAsString());
+ }
+ } else {
+ tableUpsert.setNull(39, Types.CHAR);
+ }
+
tableUpsert.execute();
if (asyncCreatedDate != null) {
@@ -3806,6 +3859,7 @@ public boolean isViewReferenced() {
.setDisableWAL(Boolean.TRUE.equals(disableWAL)).setMultiTenant(multiTenant)
.setStoreNulls(storeNulls).setViewType(viewType).setViewIndexIdType(viewIndexIdType)
.setViewIndexId(result.getViewIndexId()).setIndexType(indexType)
+ .setIndexConsistency(tableType == PTableType.INDEX ? indexConsistency : null)
.setTransactionProvider(transactionProvider).setUpdateCacheFrequency(updateCacheFrequency)
.setNamespaceMapped(isNamespaceMapped).setAutoPartitionSeqName(autoPartitionSeq)
.setAppendOnlySchema(isAppendOnlySchema).setImmutableStorageScheme(immutableStorageScheme)
@@ -5791,6 +5845,37 @@ public MutationState alterIndex(AlterIndexStatement statement) throws SQLExcepti
metaPropertiesEvaluated, table, schemaName, tableName, new MutableBoolean(false));
PIndexState newIndexState = statement.getIndexState();
+ IndexConsistency newIndexConsistency = statement.getIndexConsistency();
+
+ if (newIndexConsistency != null) {
+ try (PreparedStatement consistencyUpsert =
+ connection.prepareStatement(UPDATE_INDEX_CONSISTENCY)) {
+ consistencyUpsert.setString(1, tenantId);
+ consistencyUpsert.setString(2, schemaName);
+ consistencyUpsert.setString(3, indexName);
+ consistencyUpsert.setString(4, newIndexConsistency.getCodeAsString());
+ consistencyUpsert.execute();
+ connection.commit();
+
+ if (newIndexConsistency == IndexConsistency.EVENTUAL) {
+ PTable dataTable =
+ connection.getTable(tenantId, SchemaUtil.getTableName(schemaName, dataTableName));
+ createCDCForEventuallyConsistentIndex(dataTable);
+ }
+
+ connection.getQueryServices().clearTableFromCache(
+ connection.getTenantId() == null
+ ? ByteUtil.EMPTY_BYTE_ARRAY
+ : connection.getTenantId().getBytes(),
+ schemaName == null ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(schemaName),
+ Bytes.toBytes(indexName), HConstants.LATEST_TIMESTAMP);
+ connection.removeTable(connection.getTenantId(),
+ SchemaUtil.getTableName(schemaName, indexName),
+ SchemaUtil.getTableName(schemaName, dataTableName), HConstants.LATEST_TIMESTAMP);
+
+ return new MutationState(1, 1000, connection);
+ }
+ }
if (isAsync && newIndexState != PIndexState.REBUILD) {
throw new SQLExceptionInfo.Builder(SQLExceptionCode.ASYNC_NOT_ALLOWED)
@@ -5804,13 +5889,8 @@ public MutationState alterIndex(AlterIndexStatement statement) throws SQLExcepti
connection.setAutoCommit(false);
// Confirm index table is valid and up-to-date
TableRef indexRef = FromCompiler.getResolver(statement, connection).getTables().get(0);
- PreparedStatement tableUpsert = null;
- try {
- if (newIndexState == PIndexState.ACTIVE) {
- tableUpsert = connection.prepareStatement(UPDATE_INDEX_STATE_TO_ACTIVE);
- } else {
- tableUpsert = connection.prepareStatement(UPDATE_INDEX_STATE);
- }
+ try (PreparedStatement tableUpsert = connection.prepareStatement(
+ newIndexState == PIndexState.ACTIVE ? UPDATE_INDEX_STATE_TO_ACTIVE : UPDATE_INDEX_STATE)) {
tableUpsert.setString(1,
connection.getTenantId() == null ? null : connection.getTenantId().getString());
tableUpsert.setString(2, schemaName);
@@ -5821,10 +5901,6 @@ public MutationState alterIndex(AlterIndexStatement statement) throws SQLExcepti
tableUpsert.setLong(6, 0);
}
tableUpsert.execute();
- } finally {
- if (tableUpsert != null) {
- tableUpsert.close();
- }
}
Long timeStamp = indexRef.getTable().isTransactional() ? indexRef.getTimeStamp() : null;
List tableMetadata =
@@ -5913,8 +5989,8 @@ public MutationState alterIndex(AlterIndexStatement statement) throws SQLExcepti
}
}
} else {
- try {
- tableUpsert = connection.prepareStatement(UPDATE_INDEX_REBUILD_ASYNC_STATE);
+ try (PreparedStatement tableUpsert =
+ connection.prepareStatement(UPDATE_INDEX_REBUILD_ASYNC_STATE)) {
tableUpsert.setString(1,
connection.getTenantId() == null ? null : connection.getTenantId().getString());
tableUpsert.setString(2, schemaName);
@@ -5923,10 +5999,6 @@ public MutationState alterIndex(AlterIndexStatement statement) throws SQLExcepti
tableUpsert.setLong(4, beginTimestamp);
tableUpsert.execute();
connection.commit();
- } finally {
- if (tableUpsert != null) {
- tableUpsert.close();
- }
}
}
}
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/PTable.java b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/PTable.java
index 36a9ab739f8..effd7773ec2 100644
--- a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/PTable.java
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/PTable.java
@@ -39,6 +39,7 @@
import org.apache.phoenix.index.IndexMaintainer;
import org.apache.phoenix.jdbc.PhoenixConnection;
import org.apache.phoenix.schema.transform.TransformMaintainer;
+import org.apache.phoenix.schema.types.IndexConsistency;
import org.apache.phoenix.schema.types.PArrayDataType;
import org.apache.phoenix.schema.types.PArrayDataTypeDecoder;
import org.apache.phoenix.schema.types.PArrayDataTypeEncoder;
@@ -931,6 +932,8 @@ IndexMaintainer getIndexMaintainer(PTable dataTable, PTable cdcTable,
IndexType getIndexType();
+ IndexConsistency getIndexConsistency();
+
int getBaseColumnCount();
/**
@@ -1154,5 +1157,15 @@ enum CDCChangeScope {
* Include only the post image (state past the change) of the row.
*/
POST,
+
+ /**
+ * Include index mutations for eventually consistent indexes.
+ */
+ IDX_MUTATIONS,
+
+ /**
+ * Include raw before/after data row states as serialized Puts for index mutation generation.
+ */
+ DATA_ROW_STATE,
}
}
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index 027d0317054..dee96ceafc8 100644
--- a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -103,6 +103,7 @@
import org.apache.phoenix.query.QueryConstants;
import org.apache.phoenix.schema.RowKeySchema.RowKeySchemaBuilder;
import org.apache.phoenix.schema.transform.TransformMaintainer;
+import org.apache.phoenix.schema.types.IndexConsistency;
import org.apache.phoenix.schema.types.PBinary;
import org.apache.phoenix.schema.types.PChar;
import org.apache.phoenix.schema.types.PDataType;
@@ -193,6 +194,7 @@ public class PTableImpl implements PTable {
private final Long viewIndexId;
private final int estimatedSize;
private final IndexType indexType;
+ private final IndexConsistency indexConsistency;
private final int baseColumnCount;
private final boolean rowKeyOrderOptimizable; // TODO: remove when required that tables have been
// upgrade for PHOENIX-2067
@@ -267,6 +269,7 @@ public static class Builder {
private Long viewIndexId;
private int estimatedSize;
private IndexType indexType;
+ private IndexConsistency indexConsistency;
private int baseColumnCount;
private boolean rowKeyOrderOptimizable;
private boolean hasColumnsRequiringUpgrade;
@@ -534,6 +537,11 @@ public Builder setIndexType(IndexType indexType) {
return this;
}
+ public Builder setIndexConsistency(IndexConsistency indexConsistency) {
+ this.indexConsistency = indexConsistency;
+ return this;
+ }
+
public Builder setBaseColumnCount(int baseColumnCount) {
this.baseColumnCount = baseColumnCount;
return this;
@@ -994,6 +1002,7 @@ private PTableImpl(Builder builder) {
this.viewIndexId = builder.viewIndexId;
this.estimatedSize = builder.estimatedSize;
this.indexType = builder.indexType;
+ this.indexConsistency = builder.indexConsistency;
this.baseColumnCount = builder.baseColumnCount;
this.rowKeyOrderOptimizable = builder.rowKeyOrderOptimizable;
this.hasColumnsRequiringUpgrade = builder.hasColumnsRequiringUpgrade;
@@ -1052,6 +1061,7 @@ public static PTableImpl.Builder builderFromExisting(PTable table) {
.setMultiTenant(table.isMultiTenant()).setStoreNulls(table.getStoreNulls())
.setViewType(table.getViewType()).setViewIndexIdType(table.getviewIndexIdType())
.setViewIndexId(table.getViewIndexId()).setIndexType(table.getIndexType())
+ .setIndexConsistency(table.getIndexConsistency())
.setTransactionProvider(table.getTransactionProvider())
.setUpdateCacheFrequency(table.getUpdateCacheFrequency())
.setNamespaceMapped(table.isNamespaceMapped())
@@ -1890,6 +1900,11 @@ public IndexType getIndexType() {
return indexType;
}
+ @Override
+ public IndexConsistency getIndexConsistency() {
+ return type == PTableType.INDEX ? indexConsistency : null;
+ }
+
/**
* Construct a PTable instance from ProtoBuffered PTable instance
*/
@@ -2096,6 +2111,14 @@ public static PTable createFromProto(PTableProtos.PTable table) {
if (table.hasRowKeyMatcher()) {
rowKeyMatcher = table.getRowKeyMatcher().toByteArray();
}
+ IndexConsistency indexConsistency = null;
+ if (tableType == PTableType.INDEX) {
+ if (table.hasIndexConsistency()) {
+ indexConsistency = IndexConsistency.valueOf(table.getIndexConsistency());
+ } else {
+ indexConsistency = null;
+ }
+ }
try {
return new PTableImpl.Builder().setType(tableType).setState(indexState)
@@ -2103,7 +2126,7 @@ public static PTable createFromProto(PTableProtos.PTable table) {
.setSequenceNumber(sequenceNumber).setImmutableRows(isImmutableRows)
.setViewStatement(viewStatement).setDisableWAL(disableWAL).setMultiTenant(multiTenant)
.setStoreNulls(storeNulls).setViewType(viewType).setViewIndexIdType(viewIndexIdType)
- .setViewIndexId(viewIndexId).setIndexType(indexType)
+ .setViewIndexId(viewIndexId).setIndexType(indexType).setIndexConsistency(indexConsistency)
.setTransactionProvider(transactionProvider).setUpdateCacheFrequency(updateCacheFrequency)
.setNamespaceMapped(isNamespaceMapped).setAutoPartitionSeqName(autoPartitionSeqName)
.setAppendOnlySchema(isAppendOnlySchema)
@@ -2166,6 +2189,9 @@ public static PTableProtos.PTable toProto(PTable table, long clientVersion) {
builder.setIndexType(
ByteStringer.wrap(new byte[] { table.getIndexType().getSerializedValue() }));
}
+ if (table.getIndexConsistency() != null) {
+ builder.setIndexConsistency(table.getIndexConsistency().name());
+ }
}
builder.setSequenceNumber(table.getSequenceNumber());
builder.setTimeStamp(table.getTimeStamp());
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/tool/SchemaSQLUtil.java b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/tool/SchemaSQLUtil.java
index 6fc4c1c98a1..d6b513ad26e 100644
--- a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/tool/SchemaSQLUtil.java
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/tool/SchemaSQLUtil.java
@@ -26,6 +26,7 @@
import org.apache.phoenix.parse.CreateTableStatement;
import org.apache.phoenix.schema.PTable;
import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.types.IndexConsistency;
import org.apache.phoenix.util.SchemaUtil;
import org.apache.phoenix.thirdparty.com.google.common.collect.ListMultimap;
@@ -74,6 +75,12 @@ protected static String getCreateIndexSQL(CreateIndexStatement createStmt) {
if (createStmt.isAsync()) {
sb.append(" ASYNC");
}
+ if (
+ createStmt.getIndexConsistency() != null
+ && createStmt.getIndexConsistency() != IndexConsistency.getDefault()
+ ) {
+ sb.append(" CONSISTENCY=").append(createStmt.getIndexConsistency().name());
+ }
appendProperties(sb, createStmt.getProps());
return sb.toString();
}
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/transform/TransformClient.java b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/transform/TransformClient.java
index ebcd9f034ca..f37c414f5b7 100644
--- a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/transform/TransformClient.java
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/transform/TransformClient.java
@@ -258,9 +258,10 @@ protected static PTable addTransform(PTable table, MetaDataClient.MetaProperties
.setEstimatedSize(table.getEstimatedSize()).setFamilies(table.getColumnFamilies())
.setImmutableRows(table.isImmutableRows())
.setIsChangeDetectionEnabled(table.isChangeDetectionEnabled())
- .setIndexType(table.getIndexType()).setIndexes(Collections. emptyList())
- .setName(newTableName).setMultiTenant(table.isMultiTenant())
- .setParentName(table.getParentName()).setParentSchemaName(table.getParentSchemaName())
+ .setIndexType(table.getIndexType()).setIndexConsistency(table.getIndexConsistency())
+ .setIndexes(Collections.emptyList()).setName(newTableName)
+ .setMultiTenant(table.isMultiTenant()).setParentName(table.getParentName())
+ .setParentSchemaName(table.getParentSchemaName())
.setNamespaceMapped(table.isNamespaceMapped()).setSchemaName(table.getSchemaName())
.setPkColumns(table.getPKColumns()).setPkName(table.getPKName())
.setRowKeySchema(table.getRowKeySchema()).setStoreNulls(table.getStoreNulls())
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/types/IndexConsistency.java b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/types/IndexConsistency.java
new file mode 100644
index 00000000000..13b61fab4ef
--- /dev/null
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/types/IndexConsistency.java
@@ -0,0 +1,109 @@
+/*
+ * 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.schema.types;
+
+/**
+ * Enum representing the consistency level for Phoenix secondary indexes.
+ *
+ * STRONG: Synchronous index maintenance where index mutations are applied synchronously with
+ * data table mutations. This provides immediate consistency but may impact write performance and
+ * availability.
+ *
+ *
+ * EVENTUAL: Asynchronous index maintenance where index mutations are applied in the
+ * background. This provides better write performance and availability but with eventual consistency
+ * guarantees.
+ *
+ */
+public enum IndexConsistency {
+ /**
+ * Strong consistency - synchronous index maintenance (default behavior). Index updates are
+ * applied synchronously with data table updates.
+ */
+ STRONG('s'),
+
+ /**
+ * Eventual consistency - asynchronous index maintenance using CDC. Index updates are applied in
+ * the background, providing eventual consistency.
+ */
+ EVENTUAL('e');
+
+ private final char code;
+
+ IndexConsistency(char code) {
+ this.code = code;
+ }
+
+ /**
+ * Returns the single character code for the consistency.
+ * @return 's' for STRONG, 'e' for EVENTUAL
+ */
+ public char getCode() {
+ return code;
+ }
+
+ /**
+ * Returns the string representation of the code for storage.
+ * @return single character string for storage
+ */
+ public String getCodeAsString() {
+ return String.valueOf(code);
+ }
+
+ /**
+ * Parses an IndexConsistency from its stored code.
+ * @param code the stored code.
+ * @return the corresponding IndexConsistency, or null if code is null/empty/invalid.
+ */
+ public static IndexConsistency fromCode(String code) {
+ if (code == null || code.isEmpty()) {
+ return null;
+ }
+ char c = code.charAt(0);
+ for (IndexConsistency ic : values()) {
+ if (ic.code == c) {
+ return ic;
+ }
+ }
+ return null;
+ }
+
+ /**
+ * Returns the default consistency level for indexes.
+ * @return STRONG consistency as the default
+ */
+ public static IndexConsistency getDefault() {
+ return STRONG;
+ }
+
+ /**
+ * Checks if this consistency level requires synchronous index maintenance.
+ * @return true if synchronous maintenance is required, false otherwise
+ */
+ public boolean isSynchronous() {
+ return this == STRONG;
+ }
+
+ /**
+ * Checks if this consistency level uses asynchronous index maintenance.
+ * @return true if asynchronous maintenance is used, false otherwise
+ */
+ public boolean isAsynchronous() {
+ return this == EVENTUAL;
+ }
+}
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/util/CDCChangeBuilder.java b/phoenix-core-client/src/main/java/org/apache/phoenix/util/CDCChangeBuilder.java
index 45a7e957e2d..880a0a327b2 100644
--- a/phoenix-core-client/src/main/java/org/apache/phoenix/util/CDCChangeBuilder.java
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/util/CDCChangeBuilder.java
@@ -26,9 +26,12 @@
import static org.apache.phoenix.query.QueryConstants.CDC_UPSERT_EVENT_TYPE;
import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Set;
import org.apache.hadoop.hbase.Cell;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
import org.apache.phoenix.index.CDCTableInfo;
import org.apache.phoenix.schema.PTable;
@@ -36,6 +39,8 @@ public class CDCChangeBuilder {
private final boolean isChangeImageInScope;
private final boolean isPreImageInScope;
private final boolean isPostImageInScope;
+ private final boolean isIdxMutationsInScope;
+ private final boolean isDataRowStateInScope;
private final CDCTableInfo cdcDataTableInfo;
private String changeType;
private long lastDeletedTimestamp;
@@ -43,12 +48,20 @@ public class CDCChangeBuilder {
private Map preImage = null;
private Map changeImage = null;
+ private boolean isFullRowDelete;
+ private Map rawLatestBeforeChange;
+ private Map rawAtChange;
+ private Set rawDeletedColumnsAtChange;
+ private Map rawDeletedColumnsBeforeChange;
+
public CDCChangeBuilder(CDCTableInfo cdcDataTableInfo) {
this.cdcDataTableInfo = cdcDataTableInfo;
Set changeScopes = cdcDataTableInfo.getIncludeScopes();
isChangeImageInScope = changeScopes.contains(PTable.CDCChangeScope.CHANGE);
isPreImageInScope = changeScopes.contains(PTable.CDCChangeScope.PRE);
isPostImageInScope = changeScopes.contains(PTable.CDCChangeScope.POST);
+ isIdxMutationsInScope = changeScopes.contains(PTable.CDCChangeScope.IDX_MUTATIONS);
+ isDataRowStateInScope = changeScopes.contains(PTable.CDCChangeScope.DATA_ROW_STATE);
}
public void initChange(long ts) {
@@ -61,6 +74,13 @@ public void initChange(long ts) {
if (isChangeImageInScope || isPostImageInScope) {
changeImage = new HashMap<>();
}
+ if (isDataRowStateInScope) {
+ isFullRowDelete = false;
+ rawLatestBeforeChange = new LinkedHashMap<>();
+ rawAtChange = new LinkedHashMap<>();
+ rawDeletedColumnsAtChange = new HashSet<>();
+ rawDeletedColumnsBeforeChange = new HashMap<>();
+ }
}
public long getChangeTimestamp() {
@@ -77,6 +97,9 @@ public boolean isNonEmptyEvent() {
public void markAsDeletionEvent() {
changeType = CDC_DELETE_EVENT_TYPE;
+ if (isDataRowStateInScope) {
+ isFullRowDelete = true;
+ }
}
public long getLastDeletedTimestamp() {
@@ -141,12 +164,69 @@ public Map buildCDCEvent() {
}
public boolean isOlderThanChange(Cell cell) {
- return (cell.getTimestamp() < changeTimestamp && cell.getTimestamp() > lastDeletedTimestamp)
- ? true
- : false;
+ return cell.getTimestamp() < changeTimestamp && cell.getTimestamp() > lastDeletedTimestamp;
+ }
+
+ public void registerRawPut(Cell cell, ImmutableBytesPtr colKey) {
+ if (cell.getTimestamp() == changeTimestamp) {
+ rawAtChange.putIfAbsent(colKey, cell);
+ } else if (isOlderThanChange(cell)) {
+ Long colDeleteTs = rawDeletedColumnsBeforeChange.get(colKey);
+ if (
+ (colDeleteTs == null || cell.getTimestamp() > colDeleteTs)
+ && !rawLatestBeforeChange.containsKey(colKey)
+ ) {
+ rawLatestBeforeChange.put(colKey, cell);
+ }
+ }
+ }
+
+ public void registerRawDeleteColumn(Cell cell, ImmutableBytesPtr colKey) {
+ if (cell.getTimestamp() == changeTimestamp) {
+ rawDeletedColumnsAtChange.add(colKey);
+ } else if (isOlderThanChange(cell)) {
+ rawDeletedColumnsBeforeChange.putIfAbsent(colKey, cell.getTimestamp());
+ }
+ }
+
+ public boolean hasValidDataRowStateChanges() {
+ return isFullRowDelete || !rawAtChange.isEmpty() || !rawDeletedColumnsAtChange.isEmpty();
+ }
+
+ public boolean isFullRowDelete() {
+ return isFullRowDelete;
+ }
+
+ public Map getRawLatestBeforeChange() {
+ return rawLatestBeforeChange;
+ }
+
+ public Map getRawAtChange() {
+ return rawAtChange;
+ }
+
+ public Set getRawDeletedColumnsAtChange() {
+ return rawDeletedColumnsAtChange;
}
public boolean isPreImageInScope() {
return isPreImageInScope;
}
+
+ public boolean isPostImageInScope() {
+ return isPostImageInScope;
+ }
+
+ public boolean isChangeImageInScope() {
+ return isChangeImageInScope;
+ }
+
+ public boolean isIdxMutationsInScope() {
+ return isIdxMutationsInScope;
+ }
+
+ public boolean isDataRowStateInScope() {
+ return isDataRowStateInScope;
+ }
+
}
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/util/CDCUtil.java b/phoenix-core-client/src/main/java/org/apache/phoenix/util/CDCUtil.java
index 68c9b48aa77..d36c957f7a6 100644
--- a/phoenix-core-client/src/main/java/org/apache/phoenix/util/CDCUtil.java
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/util/CDCUtil.java
@@ -36,6 +36,7 @@
import org.apache.phoenix.exception.SQLExceptionCode;
import org.apache.phoenix.exception.SQLExceptionInfo;
import org.apache.phoenix.execute.DescVarLengthFastByteComparisons;
+import org.apache.phoenix.query.QueryConstants;
import org.apache.phoenix.schema.PIndexState;
import org.apache.phoenix.schema.PTable;
import org.apache.phoenix.schema.types.PDataType;
@@ -148,6 +149,34 @@ public static boolean hasCDCIndex(PTable table) {
return table.getIndexes().stream().anyMatch(CDCUtil::isCDCIndex);
}
+ /**
+ * Get the CDC object name for the given data table that can be used in SQL queries.
+ * @param dataTable The data table PTable object.
+ * @param withSchema If true, returns the CDC object name with schema prefix. If false, returns
+ * only the CDC object name.
+ * @return The CDC object name that can be used in SQL queries, optionally prefixed with schema
+ * name, or null if no active CDC index exists.
+ */
+ public static String getCDCObjectName(PTable dataTable, boolean withSchema) {
+ if (dataTable == null) {
+ return null;
+ }
+ PTable cdcIndex = getActiveCDCIndex(dataTable);
+ if (cdcIndex == null) {
+ return null;
+ }
+ String cdcIndexName = cdcIndex.getTableName().getString();
+ String schemaName = dataTable.getSchemaName().getString();
+ if (isCDCIndex(cdcIndexName)) {
+ String cdcObj = cdcIndexName.substring(CDC_INDEX_PREFIX.length());
+ if (!withSchema || schemaName == null || schemaName.isEmpty()) {
+ return cdcObj;
+ }
+ return schemaName + QueryConstants.NAME_SEPARATOR + cdcObj;
+ }
+ return null;
+ }
+
public static Scan setupScanForCDC(Scan scan) {
scan.setRaw(true);
scan.readAllVersions();
diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/util/MetaDataUtil.java b/phoenix-core-client/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
index 5396fc70f8c..9249a8ad68f 100644
--- a/phoenix-core-client/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
+++ b/phoenix-core-client/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
@@ -126,7 +126,8 @@ public class MetaDataUtil {
public static final List SYSTEM_TABLES_WITH_TTL_SUPPORTED =
Arrays.asList(SchemaUtil.getTableName(SYSTEM_CATALOG_SCHEMA, SYSTEM_CDC_STREAM_TABLE),
- SchemaUtil.getTableName(SYSTEM_CATALOG_SCHEMA, SYSTEM_CDC_STREAM_STATUS_TABLE));
+ SchemaUtil.getTableName(SYSTEM_CATALOG_SCHEMA, SYSTEM_CDC_STREAM_STATUS_TABLE),
+ SchemaUtil.getTableName(SYSTEM_CATALOG_SCHEMA, SYSTEM_IDX_CDC_TRACKER_TABLE));
public static Put getLastDDLTimestampUpdate(byte[] tableHeaderRowKey, long clientTimestamp,
long lastDDLTimestamp) {
diff --git a/phoenix-core-client/src/main/protobuf/IndexMutations.proto b/phoenix-core-client/src/main/protobuf/IndexMutations.proto
new file mode 100644
index 00000000000..cba4360dd76
--- /dev/null
+++ b/phoenix-core-client/src/main/protobuf/IndexMutations.proto
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+option java_package = "org.apache.phoenix.coprocessor.generated";
+option java_outer_classname = "IndexMutationsProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+// The batch of index mutations for eventually consistent indexes.
+// Each entry pairs a target index table name (tables[i]) with its
+// corresponding serialized HBase MutationProto (mutations[i]).
+message IndexMutations {
+ repeated bytes tables = 1;
+ repeated bytes mutations = 2;
+}
+
+// Raw data row states for generating index mutations.
+// Contains the data row key and serialized MutationProto for the before and after states.
+message DataRowStates {
+ optional bytes dataRowKey = 1;
+ optional bytes currentDataRowState = 2;
+ optional bytes nextDataRowState = 3;
+}
diff --git a/phoenix-core-client/src/main/protobuf/PTable.proto b/phoenix-core-client/src/main/protobuf/PTable.proto
index 43f53151147..a98239f9ed2 100644
--- a/phoenix-core-client/src/main/protobuf/PTable.proto
+++ b/phoenix-core-client/src/main/protobuf/PTable.proto
@@ -124,6 +124,7 @@ message PTable {
optional bytes ttl = 55;
optional bytes rowKeyMatcher = 56;
optional bool isStrictTTL = 57;
+ optional string indexConsistency = 58;
}
message EncodedCQCounter {
diff --git a/phoenix-core-client/src/main/protobuf/ServerCachingService.proto b/phoenix-core-client/src/main/protobuf/ServerCachingService.proto
index d8e19455273..2c4f358cccd 100644
--- a/phoenix-core-client/src/main/protobuf/ServerCachingService.proto
+++ b/phoenix-core-client/src/main/protobuf/ServerCachingService.proto
@@ -73,6 +73,7 @@ message IndexMaintainer {
repeated ColumnReference indexWhereColumns = 30;
optional bool isCDCIndex = 31;
optional int32 dataTableSaltBuckets = 32;
+ optional string indexConsistency = 33;
}
message TransformMaintainer {
diff --git a/phoenix-core-server/pom.xml b/phoenix-core-server/pom.xml
index 9f786ab111f..541557b9661 100644
--- a/phoenix-core-server/pom.xml
+++ b/phoenix-core-server/pom.xml
@@ -173,6 +173,10 @@
com.google.code.findbugsjsr305
+
+ org.xerial.snappy
+ snappy-java
+
diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/CDCGlobalIndexRegionScanner.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/CDCGlobalIndexRegionScanner.java
index 5bb0c402172..97b388dc43e 100644
--- a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/CDCGlobalIndexRegionScanner.java
+++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/CDCGlobalIndexRegionScanner.java
@@ -21,18 +21,22 @@
import static org.apache.phoenix.util.ByteUtil.EMPTY_BYTE_ARRAY;
import com.fasterxml.jackson.core.JsonProcessingException;
+import com.google.protobuf.ByteString;
import java.io.IOException;
import java.util.Arrays;
+import java.util.Base64;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import java.util.Set;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellBuilder;
import org.apache.hadoop.hbase.CellBuilderFactory;
import org.apache.hadoop.hbase.CellBuilderType;
import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
@@ -41,6 +45,7 @@
import org.apache.hadoop.hbase.regionserver.RegionScanner;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.phoenix.coprocessor.generated.CDCInfoProtos;
+import org.apache.phoenix.coprocessor.generated.IndexMutationsProtos;
import org.apache.phoenix.execute.TupleProjector;
import org.apache.phoenix.expression.Expression;
import org.apache.phoenix.expression.SingleCellColumnExpression;
@@ -50,6 +55,7 @@
import org.apache.phoenix.query.QueryConstants;
import org.apache.phoenix.schema.tuple.ResultTuple;
import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PVarchar;
import org.apache.phoenix.util.CDCChangeBuilder;
import org.apache.phoenix.util.CDCUtil;
import org.apache.phoenix.util.EncodedColumnsUtil;
@@ -57,6 +63,10 @@
import org.apache.phoenix.util.JacksonUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.xerial.snappy.Snappy;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
/**
* CDC (Change Data Capture) enabled region scanner for global indexes that processes uncovered CDC
@@ -86,6 +96,10 @@ public class CDCGlobalIndexRegionScanner extends UncoveredGlobalIndexRegionScann
private static final Logger LOGGER = LoggerFactory.getLogger(CDCGlobalIndexRegionScanner.class);
private CDCTableInfo cdcDataTableInfo;
private CDCChangeBuilder changeBuilder;
+ private static final byte[] SEPARATOR = { 0 };
+
+ private static final byte[] EMPTY_IDX_MUTATIONS = PVarchar.INSTANCE.toBytes(Base64.getEncoder()
+ .encodeToString(IndexMutationsProtos.IndexMutations.getDefaultInstance().toByteArray()));
public CDCGlobalIndexRegionScanner(final RegionScanner innerScanner, final Region region,
final Scan scan, final RegionCoprocessorEnvironment env, final Scan dataTableScan,
@@ -102,6 +116,13 @@ public CDCGlobalIndexRegionScanner(final RegionScanner innerScanner, final Regio
@Override
protected Scan prepareDataTableScan(Collection dataRowKeys) throws IOException {
+ if (
+ changeBuilder.isIdxMutationsInScope() && !changeBuilder.isDataRowStateInScope()
+ && !changeBuilder.isChangeImageInScope() && !changeBuilder.isPreImageInScope()
+ && !changeBuilder.isPostImageInScope()
+ ) {
+ return null;
+ }
// TODO: Get Timerange from the start row and end row of the index scan object
// and set it in the datatable scan object.
// if (scan.getStartRow().length == 8) {
@@ -112,7 +133,11 @@ protected Scan prepareDataTableScan(Collection dataRowKeys) throws IOExc
// stopTimeRange = PLong.INSTANCE.getCodec().decodeLong(
// scan.getStopRow(), 0, SortOrder.getDefault());
// }
- return CDCUtil.setupScanForCDC(prepareDataTableScan(dataRowKeys, true));
+ Scan dataScan = prepareDataTableScan(dataRowKeys, true);
+ if (dataScan == null) {
+ return null;
+ }
+ return CDCUtil.setupScanForCDC(dataScan);
}
protected boolean getNextCoveredIndexRow(List result) throws IOException {
@@ -120,9 +145,11 @@ protected boolean getNextCoveredIndexRow(List result) throws IOException {
List indexRow = indexRowIterator.next();
Cell indexCell = indexRow.get(0);
byte[] indexRowKey = ImmutableBytesPtr.cloneCellRowIfNecessary(indexCell);
+ if (handleIdxMutationsCDCEvent(indexRow, indexRowKey, indexCell, result)) {
+ return true;
+ }
if (indexRow.size() > 1) {
- boolean success = handlePreImageCDCEvent(indexRow, indexRowKey, indexCell, result);
- if (success) {
+ if (handlePreImageCDCEvent(indexRow, indexRowKey, indexCell, result)) {
return true;
}
}
@@ -158,59 +185,74 @@ protected boolean getNextCoveredIndexRow(List result) throws IOException {
// marker after that.
changeBuilder.setLastDeletedTimestamp(cell.getTimestamp());
}
- } else if (
- (cell.getType() == Cell.Type.DeleteColumn || cell.getType() == Cell.Type.Put)
- && !Arrays.equals(cellQual, emptyCQ)
- ) {
- if (!changeBuilder.isChangeRelevant(cell)) {
- // We don't need to build the change image, just skip it.
- continue;
- }
- // In this case, cell is the row, meaning we loop over rows..
- if (isSingleCell) {
- while (curColumnNum < cdcColumnInfoList.size()) {
- boolean hasValue = dataTableProjector.getSchema().extractValue(cell,
- (SingleCellColumnExpression) expressions[curColumnNum], ptr);
- if (hasValue) {
- Object cellValue = getColumnValue(ptr.get(), ptr.getOffset(), ptr.getLength(),
- cdcColumnInfoList.get(curColumnNum).getColumnType());
- changeBuilder.registerChange(cell, curColumnNum, cellValue);
+ } else
+ if (cell.getType() == Cell.Type.DeleteColumn || cell.getType() == Cell.Type.Put) {
+ boolean isEmptyCQ = Arrays.equals(cellQual, emptyCQ);
+ if (changeBuilder.isDataRowStateInScope()) {
+ ImmutableBytesPtr colKey =
+ new ImmutableBytesPtr(Bytes.add(cellFam, SEPARATOR, cellQual));
+ if (cell.getType() == Cell.Type.DeleteColumn) {
+ changeBuilder.registerRawDeleteColumn(cell, colKey);
+ } else {
+ changeBuilder.registerRawPut(cell, colKey);
}
- ++curColumnNum;
}
- break cellLoop;
- }
- while (true) {
- CDCTableInfo.CDCColumnInfo currentColumnInfo = cdcColumnInfoList.get(curColumnNum);
- int columnComparisonResult =
- CDCUtil.compareCellFamilyAndQualifier(cellFam, cellQual,
- currentColumnInfo.getColumnFamily(), currentColumnInfo.getColumnQualifier());
- if (columnComparisonResult > 0) {
- if (++curColumnNum >= cdcColumnInfoList.size()) {
- // Have no more column definitions, so the rest of the cells
- // must be for dropped columns and so can be ignored.
- break cellLoop;
- }
- // Continue looking for the right column definition
- // for this cell.
+ if (
+ isEmptyCQ || changeBuilder.isDataRowStateInScope()
+ && !changeBuilder.isChangeImageInScope() && !changeBuilder.isPreImageInScope()
+ && !changeBuilder.isPostImageInScope()
+ ) {
continue;
- } else if (columnComparisonResult < 0) {
- // We didn't find a column definition for this cell, ignore the
- // current cell but continue working on the rest of the cells.
- continue cellLoop;
}
+ // In this case, cell is the row, meaning we loop over rows..
+ if (isSingleCell) {
+ while (curColumnNum < cdcColumnInfoList.size()) {
+ boolean hasValue = dataTableProjector.getSchema().extractValue(cell,
+ (SingleCellColumnExpression) expressions[curColumnNum], ptr);
+ if (hasValue) {
+ Object cellValue = getColumnValue(ptr.get(), ptr.getOffset(), ptr.getLength(),
+ cdcColumnInfoList.get(curColumnNum).getColumnType());
+ changeBuilder.registerChange(cell, curColumnNum, cellValue);
+ }
+ ++curColumnNum;
+ }
+ break cellLoop;
+ }
+ while (true) {
+ CDCTableInfo.CDCColumnInfo currentColumnInfo =
+ cdcColumnInfoList.get(curColumnNum);
+ int columnComparisonResult =
+ CDCUtil.compareCellFamilyAndQualifier(cellFam, cellQual,
+ currentColumnInfo.getColumnFamily(), currentColumnInfo.getColumnQualifier());
+ if (columnComparisonResult > 0) {
+ if (++curColumnNum >= cdcColumnInfoList.size()) {
+ // Have no more column definitions, so the rest of the cells
+ // must be for dropped columns and so can be ignored.
+ break cellLoop;
+ }
+ // Continue looking for the right column definition
+ // for this cell.
+ continue;
+ } else if (columnComparisonResult < 0) {
+ // We didn't find a column definition for this cell, ignore the
+ // current cell but continue working on the rest of the cells.
+ continue cellLoop;
+ }
- // else, found the column definition.
- Object cellValue = cell.getType() == Cell.Type.DeleteColumn
- ? null
- : getColumnValue(cell, cdcColumnInfoList.get(curColumnNum).getColumnType());
- changeBuilder.registerChange(cell, curColumnNum, cellValue);
- // Done processing the current cell, check the next cell.
- break;
+ // else, found the column definition.
+ Object cellValue = cell.getType() == Cell.Type.DeleteColumn
+ ? null
+ : getColumnValue(cell, cdcColumnInfoList.get(curColumnNum).getColumnType());
+ changeBuilder.registerChange(cell, curColumnNum, cellValue);
+ // Done processing the current cell, check the next cell.
+ break;
+ }
}
- }
}
- if (changeBuilder.isNonEmptyEvent()) {
+ if (changeBuilder.isDataRowStateInScope()) {
+ buildDataRowStateResult(dataRowKey.copyBytesIfNecessary(), indexRowKey, indexCell,
+ result);
+ } else if (changeBuilder.isNonEmptyEvent()) {
Result cdcRow = getCDCImage(indexRowKey, indexCell);
if (cdcRow != null && tupleProjector != null) {
if (indexCell.getType() == Cell.Type.DeleteFamily) {
@@ -235,7 +277,12 @@ protected boolean getNextCoveredIndexRow(List result) throws IOException {
result.clear();
}
} else {
- result.clear();
+ if (changeBuilder.isDataRowStateInScope()) {
+ buildDataRowStateResult(dataRowKey.copyBytesIfNecessary(), indexRowKey, indexCell,
+ result);
+ } else {
+ result.clear();
+ }
}
return true;
@@ -317,18 +364,151 @@ private boolean handlePreImageCDCEvent(List indexRow, byte[] indexRowKey,
cdcJson.remove(QueryConstants.CDC_PRE_IMAGE);
cdcEventBytes = JacksonUtil.getObjectWriter(HashMap.class).writeValueAsBytes(cdcJson);
}
+ addResult(indexRowKey, indexCell, result, cdcDataCell, cdcEventBytes);
+ return true;
+ }
+
+ /**
+ * Builds a DataRowStates protobuf result from the raw cell maps collected by CDCChangeBuilder
+ * during the raw cell iteration. Constructs before/after HBase Put objects representing the row
+ * state before and after the change, serializes them, and populates the result. If no valid
+ * changes were found at the change timestamp (data not yet visible), only the dataRowKey is
+ * included, the consumer needs to retry.
+ * @param dataRowKey The data table row key bytes.
+ * @param indexRowKey The CDC index row key.
+ * @param indexCell The index cell.
+ * @param result The result list to populate.
+ * @throws IOException if serialization fails.
+ */
+ private void buildDataRowStateResult(byte[] dataRowKey, byte[] indexRowKey, Cell indexCell,
+ List result) throws IOException {
+ Put currentDataRowState = null;
+ Put nextDataRowState = null;
+ Map latestBeforeChange = changeBuilder.getRawLatestBeforeChange();
+ Map atChange = changeBuilder.getRawAtChange();
+ Set deletedColumnsAtChange = changeBuilder.getRawDeletedColumnsAtChange();
+ if (changeBuilder.hasValidDataRowStateChanges()) {
+ if (!latestBeforeChange.isEmpty()) {
+ currentDataRowState = new Put(dataRowKey);
+ for (Cell cell : latestBeforeChange.values()) {
+ currentDataRowState.add(cell);
+ }
+ }
+ if (!changeBuilder.isFullRowDelete()) {
+ Put nextState = new Put(dataRowKey);
+ for (Map.Entry entry : latestBeforeChange.entrySet()) {
+ if (
+ !atChange.containsKey(entry.getKey())
+ && !deletedColumnsAtChange.contains(entry.getKey())
+ ) {
+ nextState.add(entry.getValue());
+ }
+ }
+ for (Cell cell : atChange.values()) {
+ nextState.add(cell);
+ }
+ if (!nextState.isEmpty()) {
+ nextDataRowState = nextState;
+ }
+ }
+ }
+ IndexMutationsProtos.DataRowStates.Builder builder =
+ IndexMutationsProtos.DataRowStates.newBuilder();
+ builder.setDataRowKey(ByteString.copyFrom(dataRowKey));
+ if (currentDataRowState != null) {
+ builder.setCurrentDataRowState(ByteString.copyFrom(
+ ProtobufUtil.toMutation(ClientProtos.MutationProto.MutationType.PUT, currentDataRowState)
+ .toByteArray()));
+ }
+ if (nextDataRowState != null) {
+ builder.setNextDataRowState(ByteString.copyFrom(ProtobufUtil
+ .toMutation(ClientProtos.MutationProto.MutationType.PUT, nextDataRowState).toByteArray()));
+ }
+ String base64String = Base64.getEncoder().encodeToString(builder.build().toByteArray());
+ byte[] cdcEventBytes = PVarchar.INSTANCE.toBytes(base64String);
+ addResult(indexRowKey, indexCell, result, indexCell, cdcEventBytes);
+ }
+
+ /**
+ * Handles CDC event when IDX_MUTATIONS scope is enabled. Returns the index mutations as a
+ * serialized IndexMutations, or an empty proto if no mutations are present. Skips the data table
+ * scan.
+ * @param indexRow The CDC index row.
+ * @param indexRowKey The CDC index row key.
+ * @param indexCell The primary index cell.
+ * @param result The result list to populate.
+ * @return true if IDX_MUTATIONS scope is enabled, false otherwise.
+ * @throws IOException if decompression or proto parsing fails.
+ */
+ private boolean handleIdxMutationsCDCEvent(List indexRow, byte[] indexRowKey,
+ Cell indexCell, List result) throws IOException {
+ if (!changeBuilder.isIdxMutationsInScope()) {
+ return false;
+ }
+ byte[] idxMutationsBytes = EMPTY_IDX_MUTATIONS;
+ Cell idxMutationsCell = indexCell;
+ IndexMutationsProtos.IndexMutations preMutations = null;
+ IndexMutationsProtos.IndexMutations postMutations = null;
+ Cell preCell = null;
+ for (Cell cell : indexRow) {
+ if (
+ Bytes.equals(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
+ QueryConstants.CDC_INDEX_PRE_MUTATIONS_CQ_BYTES, 0,
+ QueryConstants.CDC_INDEX_PRE_MUTATIONS_CQ_BYTES.length)
+ ) {
+ byte[] rawBytes = CellUtil.cloneValue(cell);
+ preMutations = IndexMutationsProtos.IndexMutations.parseFrom(maybeDecompress(rawBytes));
+ preCell = cell;
+ } else if (
+ Bytes.equals(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
+ QueryConstants.CDC_INDEX_POST_MUTATIONS_CQ_BYTES, 0,
+ QueryConstants.CDC_INDEX_POST_MUTATIONS_CQ_BYTES.length)
+ ) {
+ byte[] rawBytes = CellUtil.cloneValue(cell);
+ postMutations = IndexMutationsProtos.IndexMutations.parseFrom(maybeDecompress(rawBytes));
+ }
+ }
+ if (preMutations != null) {
+ IndexMutationsProtos.IndexMutations merged = mergeIndexMutations(preMutations, postMutations);
+ String base64String = Base64.getEncoder().encodeToString(merged.toByteArray());
+ idxMutationsBytes = PVarchar.INSTANCE.toBytes(base64String);
+ idxMutationsCell = preCell;
+ }
+ addResult(indexRowKey, indexCell, result, idxMutationsCell, idxMutationsBytes);
+ return true;
+ }
+
+ private static byte[] maybeDecompress(byte[] input) {
+ try {
+ if (Snappy.isValidCompressedBuffer(input)) {
+ return Snappy.uncompress(input);
+ }
+ } catch (IOException e) {
+ LOGGER.error("Error uncompressing CDC Index mutations", e);
+ }
+ return input;
+ }
+
+ private IndexMutationsProtos.IndexMutations mergeIndexMutations(
+ IndexMutationsProtos.IndexMutations pre, IndexMutationsProtos.IndexMutations post) {
+ if (post == null) {
+ return pre;
+ }
+ return IndexMutationsProtos.IndexMutations.newBuilder().addAllTables(pre.getTablesList())
+ .addAllMutations(pre.getMutationsList()).addAllTables(post.getTablesList())
+ .addAllMutations(post.getMutationsList()).build();
+ }
+
+ private void addResult(byte[] indexRowKey, Cell indexCell, List result, Cell cdcDataCell,
+ byte[] cdcEventBytes) {
Result cdcRow =
createCDCResult(indexRowKey, indexCell, cdcDataCell.getTimestamp(), cdcEventBytes);
-
if (tupleProjector != null) {
result.add(indexCell);
IndexUtil.addTupleAsOneCell(result, new ResultTuple(cdcRow), tupleProjector, ptr);
} else {
result.clear();
}
- LOGGER.debug(
- "Processed CDC event with embedded data, skipped data table scan for" + " row key: {}",
- Bytes.toStringBinary(indexRowKey));
- return true;
}
+
}
diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index 9d3d0b3ca47..053976b586d 100644
--- a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@ -39,6 +39,7 @@
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ENCODING_SCHEME_BYTES;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.EXTERNAL_SCHEMA_ID_BYTES;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_CONSISTENCY_BYTES;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP_BYTES;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE_BYTES;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_TYPE_BYTES;
@@ -263,8 +264,10 @@
import org.apache.phoenix.schema.metrics.MetricsMetadataSourceFactory;
import org.apache.phoenix.schema.task.ServerTask;
import org.apache.phoenix.schema.task.SystemTaskParams;
+import org.apache.phoenix.schema.types.IndexConsistency;
import org.apache.phoenix.schema.types.PBinary;
import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PChar;
import org.apache.phoenix.schema.types.PDataType;
import org.apache.phoenix.schema.types.PInteger;
import org.apache.phoenix.schema.types.PLong;
@@ -418,6 +421,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements RegionCopr
createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, CDC_INCLUDE_BYTES);
private static final Cell INDEX_WHERE_KV =
createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, INDEX_WHERE_BYTES);
+ private static final Cell INDEX_CONSISTENCY_KV =
+ createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, INDEX_CONSISTENCY_BYTES);
private static final Cell TTL_KV =
createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, TTL_BYTES);
@@ -436,7 +441,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements RegionCopr
AUTO_PARTITION_SEQ_KV, APPEND_ONLY_SCHEMA_KV, STORAGE_SCHEME_KV, ENCODING_SCHEME_KV,
USE_STATS_FOR_PARALLELIZATION_KV, LAST_DDL_TIMESTAMP_KV, CHANGE_DETECTION_ENABLED_KV,
SCHEMA_VERSION_KV, EXTERNAL_SCHEMA_ID_KV, STREAMING_TOPIC_NAME_KV, INDEX_WHERE_KV,
- CDC_INCLUDE_KV, TTL_KV, ROW_KEY_MATCHER_KV, IS_STRICT_TTL_KV);
+ CDC_INCLUDE_KV, TTL_KV, ROW_KEY_MATCHER_KV, IS_STRICT_TTL_KV, INDEX_CONSISTENCY_KV);
static {
Collections.sort(TABLE_KV_COLUMNS, CellComparatorImpl.COMPARATOR);
@@ -498,6 +503,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements RegionCopr
private static final int TTL_INDEX = TABLE_KV_COLUMNS.indexOf(TTL_KV);
private static final int ROW_KEY_MATCHER_INDEX = TABLE_KV_COLUMNS.indexOf(ROW_KEY_MATCHER_KV);
private static final int IS_STRICT_TTL_INDEX = TABLE_KV_COLUMNS.indexOf(IS_STRICT_TTL_KV);
+ private static final int INDEX_CONSISTENCY_INDEX = TABLE_KV_COLUMNS.indexOf(INDEX_CONSISTENCY_KV);
// KeyValues for Column
private static final KeyValue DECIMAL_DIGITS_KV =
createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, DECIMAL_DIGITS_BYTES);
@@ -1586,6 +1592,17 @@ private PTable getTableFromCells(List tableCellList, List> allC
builder.setIsStrictTTL(
isStrictTTLKv != null ? isStrictTTL : oldTable == null || oldTable.isStrictTTL());
+ Cell indexConsistencyKv = tableKeyValues[INDEX_CONSISTENCY_INDEX];
+ IndexConsistency indexConsistency = null;
+ if (indexConsistencyKv != null) {
+ String consistencyStr = (String) PChar.INSTANCE.toObject(indexConsistencyKv.getValueArray(),
+ indexConsistencyKv.getValueOffset(), indexConsistencyKv.getValueLength());
+ indexConsistency = IndexConsistency.fromCode(consistencyStr);
+ }
+ builder.setIndexConsistency(indexConsistency != null ? indexConsistency
+ : oldTable != null ? oldTable.getIndexConsistency()
+ : null);
+
// Check the cell tag to see whether the view has modified this property
final byte[] tagUseStatsForParallelization = (useStatsForParallelizationKv == null)
? HConstants.EMPTY_BYTE_ARRAY
diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/UncoveredIndexRegionScanner.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/UncoveredIndexRegionScanner.java
index 640b85bc5ac..5d657541091 100644
--- a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/UncoveredIndexRegionScanner.java
+++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/UncoveredIndexRegionScanner.java
@@ -63,6 +63,7 @@
import org.apache.phoenix.util.ByteUtil;
import org.apache.phoenix.util.EnvironmentEdgeManager;
import org.apache.phoenix.util.IndexUtil;
+import org.apache.phoenix.util.PhoenixKeyValueUtil;
import org.apache.phoenix.util.ScanUtil;
import org.apache.phoenix.util.ServerUtil;
import org.slf4j.Logger;
@@ -280,7 +281,21 @@ protected boolean scanIndexTableRows(List result, final byte[] actualStart
indexToDataRowKeyMap.put(offset == 0 ? lastIndexRowKey : CellUtil.cloneRow(firstCell),
indexMaintainer.buildDataRowKey(new ImmutableBytesWritable(lastIndexRowKey),
viewConstants));
- indexRows.add(row);
+ // When a row has at most one cell, we can safely reuse it as we don't look at cell value,
+ // and we proceed with data table scan.
+ // For rows with multiple cells, we must copy each cell to avoid issues where the
+ // underlying byte buffers for cell values may be reused or invalidated by the scanner
+ // on subsequent nextRaw() calls and hence the cell value could be garbage under
+ // high GC pressure.
+ if (row.size() <= 1) {
+ indexRows.add(row);
+ } else {
+ List dupRow = new ArrayList<>(row.size());
+ for (Cell cell : row) {
+ dupRow.add(PhoenixKeyValueUtil.maybeCopyCell(cell));
+ }
+ indexRows.add(dupRow);
+ }
indexRowCount++;
if (
hasMore && (PhoenixScannerContext.isTimedOut(scannerContext, pageSizeMs)
diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/hbase/index/IndexCDCConsumer.java b/phoenix-core-server/src/main/java/org/apache/phoenix/hbase/index/IndexCDCConsumer.java
new file mode 100644
index 00000000000..6c4ceab789f
--- /dev/null
+++ b/phoenix-core-server/src/main/java/org/apache/phoenix/hbase/index/IndexCDCConsumer.java
@@ -0,0 +1,1347 @@
+/*
+ * 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.hbase.index;
+
+import com.google.protobuf.ByteString;
+import java.io.IOException;
+import java.sql.Date;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.ConnectionUtils;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.coprocessor.DelegateRegionCoprocessorEnvironment;
+import org.apache.phoenix.coprocessor.generated.IndexMutationsProtos;
+import org.apache.phoenix.coprocessorclient.MetaDataProtocol;
+import org.apache.phoenix.hbase.index.metrics.MetricsIndexCDCConsumerSource;
+import org.apache.phoenix.hbase.index.metrics.MetricsIndexerSourceFactory;
+import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.hbase.index.write.IndexWriter;
+import org.apache.phoenix.index.IndexMaintainer;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.RowKeySchema;
+import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.schema.types.IndexConsistency;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.CDCUtil;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
+import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.ServerUtil.ConnectionType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.phoenix.thirdparty.com.google.common.collect.ArrayListMultimap;
+import org.apache.phoenix.thirdparty.com.google.common.collect.ListMultimap;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+
+/**
+ * A single-threaded background consumer that processes CDC mutations for eventually consistent
+ * indexes. This consumer reads mutations from the CDC index table and applies them to the
+ * appropriate secondary indexes.
+ *
+ * The consumer tracks its progress in the SYSTEM.IDX_CDC_TRACKER table, allowing for proper
+ * handling of region splits and merges.
+ *
+ *
+ * Lifecycle:
+ *
+ *
On startup, the consumer first replays any remaining mutations from parent regions (if this
+ * region was created from a split or merge). A region will have multiple parent regions in case of
+ * region merge.
+ *
After replaying all parent mutations, the consumer marks each parent region as COMPLETE in
+ * SYSTEM.IDX_CDC_TRACKER.
+ *
The consumer then begins processing mutations for the current region. With every batch of
+ * rows consumed (e.g., 1000 rows), it updates LAST_TIMESTAMP in SYSTEM.IDX_CDC_TRACKER. The tracker
+ * entry with status IN_PROGRESS is created on the first batch update.
+ *
When the region is closed (due to split, merge, or shutdown), the consumer stops but does NOT
+ * mark itself as COMPLETE. The COMPLETE status is only set by child regions after they have
+ * replayed all parent mutations.
+ *
+ *
+ */
+public class IndexCDCConsumer implements Runnable {
+
+ private static final Logger LOG = LoggerFactory.getLogger(IndexCDCConsumer.class);
+
+ public static final String INDEX_CDC_CONSUMER_BATCH_SIZE =
+ "phoenix.index.cdc.consumer.batch.size";
+ private static final int DEFAULT_CDC_BATCH_SIZE = 500;
+
+ public static final String INDEX_CDC_CONSUMER_STARTUP_DELAY_MS =
+ "phoenix.index.cdc.consumer.startup.delay.ms";
+ private static final long DEFAULT_STARTUP_DELAY_MS = 10000;
+
+ /**
+ * The interval in milliseconds between processing batches when mutations are found.
+ */
+ public static final String INDEX_CDC_CONSUMER_POLL_INTERVAL_MS =
+ "phoenix.index.cdc.consumer.poll.interval.ms";
+ private static final long DEFAULT_POLL_INTERVAL_MS = 1000;
+
+ /**
+ * The time buffer in milliseconds subtracted from current time when querying CDC mutations to
+ * help avoid reading mutations that are too recent.
+ */
+ public static final String INDEX_CDC_CONSUMER_TIMESTAMP_BUFFER_MS =
+ "phoenix.index.cdc.consumer.timestamp.buffer.ms";
+ private static final long DEFAULT_TIMESTAMP_BUFFER_MS = 5000;
+
+ /**
+ * Maximum number of retries when CDC events exist but the corresponding data table mutations are
+ * not yet visible (or permanently failed). After exceeding this limit, the consumer advances past
+ * the unprocessable events to avoid blocking indefinitely. This is only used for index mutation
+ * generation approach (serializeCDCMutations = false).
+ */
+ public static final String INDEX_CDC_CONSUMER_MAX_DATA_VISIBILITY_RETRIES =
+ "phoenix.index.cdc.consumer.max.data.visibility.retries";
+ private static final int DEFAULT_MAX_DATA_VISIBILITY_RETRIES = 10;
+
+ public static final String INDEX_CDC_CONSUMER_RETRY_PAUSE_MS =
+ "phoenix.index.cdc.consumer.retry.pause.ms";
+ private static final long DEFAULT_RETRY_PAUSE_MS = 2000;
+
+ public static final String INDEX_CDC_CONSUMER_PARENT_PROGRESS_PAUSE_MS =
+ "phoenix.index.cdc.consumer.parent.progress.pause.ms";
+ private static final long DEFAULT_PARENT_PROGRESS_PAUSE_MS = 15000;
+
+ private final RegionCoprocessorEnvironment env;
+ private final String dataTableName;
+ private final String encodedRegionName;
+ private final IndexWriter indexWriter;
+ private final long pause;
+ private final long startupDelayMs;
+ private final int batchSize;
+ private final long pollIntervalMs;
+ private final long timestampBufferMs;
+ private final int maxDataVisibilityRetries;
+ private final long parentProgressPauseMs;
+ private final Configuration config;
+ private final boolean serializeCDCMutations;
+ private final MetricsIndexCDCConsumerSource metricSource;
+ private volatile boolean stopped = false;
+ private Thread consumerThread;
+ private boolean hasParentPartitions = false;
+ private PTable cachedDataTable;
+
+ private boolean tenantInit = false;
+ private boolean isMultiTenant = false;
+ private String tenantIdColName;
+ private PDataType> tenantIdDataType;
+ private TenantScanInfo ownRegionScanInfo;
+
+ private final Map ancestorScanInfoCache = new HashMap<>();
+
+ private static class TenantScanInfo {
+
+ private static final TenantScanInfo EMPTY = new TenantScanInfo("", "", null, null, null);
+
+ private final String filter;
+ private final String orderBy;
+ private final Object startValue;
+ private final Object endValue;
+ private final PDataType> dataType;
+
+ TenantScanInfo(String filter, String orderBy, Object startValue, Object endValue,
+ PDataType> dataType) {
+ this.filter = filter;
+ this.orderBy = orderBy;
+ this.startValue = startValue;
+ this.endValue = endValue;
+ this.dataType = dataType;
+ }
+
+ int bindParams(PreparedStatement ps, int startIndex) throws SQLException {
+ int idx = startIndex;
+ if (startValue != null) {
+ ps.setObject(idx++, startValue, dataType.getSqlType());
+ }
+ if (endValue != null) {
+ ps.setObject(idx++, endValue, dataType.getSqlType());
+ }
+ return idx;
+ }
+ }
+
+ /**
+ * Creates a new IndexCDCConsumer for the given region with configurable serialization mode.
+ * @param env region coprocessor environment.
+ * @param dataTableName name of the data table.
+ * @param serverName server name.
+ * @param serializeCDCMutations when true, consumes pre-serialized index mutations; when false,
+ * generates index mutations from data row states.
+ * @throws IOException if the IndexWriter cannot be created.
+ */
+ public IndexCDCConsumer(RegionCoprocessorEnvironment env, String dataTableName, String serverName,
+ boolean serializeCDCMutations) throws IOException {
+ this.env = env;
+ this.dataTableName = dataTableName;
+ this.encodedRegionName = env.getRegion().getRegionInfo().getEncodedName();
+ this.config = env.getConfiguration();
+ this.serializeCDCMutations = serializeCDCMutations;
+ this.pause = config.getLong(INDEX_CDC_CONSUMER_RETRY_PAUSE_MS, DEFAULT_RETRY_PAUSE_MS);
+ this.startupDelayMs =
+ config.getLong(INDEX_CDC_CONSUMER_STARTUP_DELAY_MS, DEFAULT_STARTUP_DELAY_MS);
+ int baseBatchSize = config.getInt(INDEX_CDC_CONSUMER_BATCH_SIZE, DEFAULT_CDC_BATCH_SIZE);
+ int jitter = ThreadLocalRandom.current().nextInt(baseBatchSize / 5 + 1);
+ this.batchSize = baseBatchSize + jitter;
+ this.pollIntervalMs =
+ config.getLong(INDEX_CDC_CONSUMER_POLL_INTERVAL_MS, DEFAULT_POLL_INTERVAL_MS);
+ this.timestampBufferMs =
+ config.getLong(INDEX_CDC_CONSUMER_TIMESTAMP_BUFFER_MS, DEFAULT_TIMESTAMP_BUFFER_MS);
+ this.maxDataVisibilityRetries = config.getInt(INDEX_CDC_CONSUMER_MAX_DATA_VISIBILITY_RETRIES,
+ DEFAULT_MAX_DATA_VISIBILITY_RETRIES);
+ this.parentProgressPauseMs =
+ config.getLong(INDEX_CDC_CONSUMER_PARENT_PROGRESS_PAUSE_MS, DEFAULT_PARENT_PROGRESS_PAUSE_MS);
+ this.metricSource = MetricsIndexerSourceFactory.getInstance().getIndexCDCConsumerSource();
+ DelegateRegionCoprocessorEnvironment indexWriterEnv =
+ new DelegateRegionCoprocessorEnvironment(env, ConnectionType.INDEX_WRITER_CONNECTION);
+ this.indexWriter =
+ new IndexWriter(indexWriterEnv, serverName + "-index-eventual-writer", false);
+ }
+
+ /**
+ * Starts the consumer thread in the background.
+ */
+ public void start() {
+ consumerThread =
+ new Thread(this, "IndexCDCConsumer-" + dataTableName + "-" + encodedRegionName);
+ consumerThread.setDaemon(true);
+ consumerThread.start();
+ }
+
+ /**
+ * Stops the consumer thread gracefully.
+ */
+ public void stop() {
+ stopped = true;
+ if (consumerThread != null) {
+ consumerThread.interrupt();
+ }
+ if (indexWriter != null) {
+ indexWriter.stop("IndexCDCConsumer stopped for " + dataTableName);
+ }
+ }
+
+ /**
+ * Sleeps for the specified duration if the consumer has not been stopped.
+ * @param millis the duration to sleep in milliseconds.
+ * @throws InterruptedException if the thread is interrupted while sleeping.
+ */
+ private void sleepIfNotStopped(long millis) throws InterruptedException {
+ if (!stopped) {
+ Thread.sleep(millis);
+ }
+ }
+
+ private PTable getDataTable(PhoenixConnection conn) throws SQLException {
+ PTable dataTable = cachedDataTable;
+ if (dataTable == null) {
+ dataTable = conn.getTable(dataTableName);
+ cachedDataTable = dataTable;
+ }
+ return dataTable;
+ }
+
+ private void refreshDataTableCache(PhoenixConnection conn) throws SQLException {
+ cachedDataTable = conn.getTable(dataTableName);
+ }
+
+ private void initTenantInfo(PhoenixConnection conn) throws SQLException {
+ if (tenantInit) {
+ return;
+ }
+ PTable dataTable = getDataTable(conn);
+ isMultiTenant = dataTable.isMultiTenant();
+ if (!isMultiTenant) {
+ ownRegionScanInfo = TenantScanInfo.EMPTY;
+ tenantInit = true;
+ return;
+ }
+ int tenantColIndex = dataTable.getBucketNum() != null ? 1 : 0;
+ PColumn tenantCol = dataTable.getPKColumns().get(tenantColIndex);
+ tenantIdColName = tenantCol.getName().getString();
+ tenantIdDataType = tenantCol.getDataType();
+
+ byte[] regionStartKey = env.getRegion().getRegionInfo().getStartKey();
+ byte[] regionEndKey = env.getRegion().getRegionInfo().getEndKey();
+ ownRegionScanInfo = buildTenantScanInfo(regionStartKey, regionEndKey, dataTable);
+ LOG.debug(
+ "Initialized multi-tenant scan for table {} region {}:"
+ + " tenantCol {}, startTenant {}, endTenant {}",
+ dataTableName, encodedRegionName, tenantIdColName, ownRegionScanInfo.startValue,
+ ownRegionScanInfo.endValue);
+ tenantInit = true;
+ }
+
+ private TenantScanInfo buildTenantScanInfo(byte[] startKey, byte[] endKey, PTable dataTable) {
+ Object startVal = extractTenantIdFromRegionKey(startKey, dataTable);
+ Object endVal = extractTenantIdFromRegionKey(endKey, dataTable);
+ StringBuilder sb = new StringBuilder();
+ if (startVal != null) {
+ sb.append("\"").append(tenantIdColName).append("\" >= ? AND ");
+ }
+ if (endVal != null) {
+ sb.append("\"").append(tenantIdColName).append("\" <= ? AND ");
+ }
+ String filter = sb.toString();
+ String orderBy = filter.isEmpty() ? "" : "\"" + tenantIdColName + "\" ASC,";
+ return new TenantScanInfo(filter, orderBy, startVal, endVal, tenantIdDataType);
+ }
+
+ private Object extractTenantIdFromRegionKey(byte[] regionKey, PTable dataTable) {
+ if (regionKey == null || regionKey.length == 0) {
+ return null;
+ }
+ final RowKeySchema schema = dataTable.getRowKeySchema();
+ int pkPos = dataTable.getBucketNum() != null ? 1 : 0;
+ ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+ int maxOffset = schema.iterator(regionKey, 0, regionKey.length, ptr);
+ for (int i = 0; i <= pkPos; i++) {
+ Boolean hasValue = schema.next(ptr, i, maxOffset);
+ if (!Boolean.TRUE.equals(hasValue)) {
+ return null;
+ }
+ }
+ byte[] tenantBytes = ByteUtil.copyKeyBytesIfNecessary(ptr);
+ PColumn tenantCol = dataTable.getPKColumns().get(pkPos);
+ return tenantCol.getDataType().toObject(tenantBytes, 0, tenantBytes.length,
+ tenantCol.getDataType(), tenantCol.getSortOrder(), tenantCol.getMaxLength(),
+ tenantCol.getScale());
+ }
+
+ private byte[][] lookupPartitionKeys(String partitionId) throws InterruptedException {
+ int retryCount = 0;
+ final String query = "SELECT PARTITION_START_KEY, PARTITION_END_KEY FROM "
+ + PhoenixDatabaseMetaData.SYSTEM_CDC_STREAM_NAME
+ + " WHERE TABLE_NAME = ? AND PARTITION_ID = ? LIMIT 1";
+ while (!stopped) {
+ try (
+ PhoenixConnection conn =
+ QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
+ PreparedStatement ps = conn.prepareStatement(query)) {
+ ps.setString(1, dataTableName);
+ ps.setString(2, partitionId);
+ try (ResultSet rs = ps.executeQuery()) {
+ if (rs.next()) {
+ byte[] startKey = rs.getBytes(1);
+ byte[] endKey = rs.getBytes(2);
+ return new byte[][] { startKey == null ? new byte[0] : startKey,
+ endKey == null ? new byte[0] : endKey };
+ }
+ }
+ LOG.error("No CDC_STREAM entry found for partition {} table {}. This should not happen.",
+ partitionId, dataTableName);
+ return new byte[][] { new byte[0], new byte[0] };
+ } catch (SQLException e) {
+ long sleepTime = ConnectionUtils.getPauseTime(pause, ++retryCount);
+ LOG.warn(
+ "Error while retrieving partition keys from CDC_STREAM for partition {} table {}. "
+ + "Retry #{}, sleeping {} ms before retrying...",
+ partitionId, dataTableName, retryCount, sleepTime, e);
+ sleepIfNotStopped(sleepTime);
+ }
+ }
+ return null;
+ }
+
+ private TenantScanInfo getPartitionTenantScanInfo(String partitionId)
+ throws InterruptedException {
+ if (!isMultiTenant) {
+ return TenantScanInfo.EMPTY;
+ }
+ if (partitionId.equals(encodedRegionName)) {
+ return ownRegionScanInfo;
+ }
+ TenantScanInfo cached = ancestorScanInfoCache.get(partitionId);
+ if (cached != null) {
+ return cached;
+ }
+ byte[][] keys = lookupPartitionKeys(partitionId);
+ if (keys == null) {
+ return TenantScanInfo.EMPTY;
+ }
+ TenantScanInfo info = buildTenantScanInfo(keys[0], keys[1], cachedDataTable);
+ ancestorScanInfoCache.put(partitionId, info);
+ return info;
+ }
+
+ @Override
+ public void run() {
+ try {
+ if (startupDelayMs > 0 && getCDCStreamNumPartitions() <= 1) {
+ sleepIfNotStopped(startupDelayMs);
+ }
+ if (stopped) {
+ return;
+ }
+ if (!hasEventuallyConsistentIndexes()) {
+ LOG.trace("No eventually consistent indexes found for table {}. Exiting consumer.",
+ dataTableName);
+ return;
+ }
+ LOG.info(
+ "IndexCDCConsumer started for table {} region {}"
+ + " [batchSize: {}, pollIntervalMs: {}, timestampBufferMs: {}, startupDelayMs: {},"
+ + " pause: {}, maxDataVisibilityRetries: {}, parentProgressPauseMs: {},"
+ + " serializeCDCMutations: {}]",
+ dataTableName, encodedRegionName, batchSize, pollIntervalMs, timestampBufferMs,
+ startupDelayMs, pause, maxDataVisibilityRetries, parentProgressPauseMs,
+ serializeCDCMutations);
+ if (!waitForCDCStreamEntry()) {
+ LOG.error(
+ "IndexCDCConsumer stopped while waiting for CDC_STREAM entry for table {} region {}",
+ dataTableName, encodedRegionName);
+ return;
+ }
+ long lastProcessedTimestamp = checkTrackerStatus(encodedRegionName, encodedRegionName);
+ if (lastProcessedTimestamp == -1) {
+ // should never happen as COMPLETE is only set by child regions for their parent regions
+ LOG.error(
+ "Unexpected COMPLETE status in IDX_CDC_TRACKER for current region. "
+ + "Table: {}, Partition: {}. Exiting consumer thread.",
+ dataTableName, encodedRegionName);
+ return;
+ } else if (lastProcessedTimestamp > 0) {
+ LOG.info(
+ "Found existing tracker entry for table {} region {} with lastTimestamp {}. "
+ + "Resuming from last position (region movement scenario).",
+ dataTableName, encodedRegionName, lastProcessedTimestamp);
+ } else {
+ if (hasParentPartitions) {
+ sleepIfNotStopped(timestampBufferMs + 1);
+ replayAndCompleteParentRegions(encodedRegionName);
+ } else {
+ LOG.info("No parent partitions for table {} region {}, skipping parent replay",
+ dataTableName, encodedRegionName);
+ }
+ }
+ int retryCount = 0;
+ while (!stopped) {
+ try {
+ long previousTimestamp = lastProcessedTimestamp;
+ if (serializeCDCMutations) {
+ lastProcessedTimestamp =
+ processCDCBatch(encodedRegionName, encodedRegionName, lastProcessedTimestamp, false);
+ } else {
+ lastProcessedTimestamp = processCDCBatchGenerated(encodedRegionName, encodedRegionName,
+ lastProcessedTimestamp, false);
+ }
+ if (lastProcessedTimestamp == previousTimestamp) {
+ sleepIfNotStopped(ConnectionUtils.getPauseTime(pause, ++retryCount));
+ } else {
+ retryCount = 0;
+ sleepIfNotStopped(pollIntervalMs);
+ }
+ } catch (Exception e) {
+ if (e instanceof InterruptedException) {
+ throw (InterruptedException) e;
+ }
+ metricSource.incrementCdcBatchFailureCount(dataTableName);
+ long sleepTime = ConnectionUtils.getPauseTime(pause, ++retryCount);
+ LOG.error(
+ "Error processing CDC mutations for table {} region {}. "
+ + "Retry #{}, sleeping {} ms before retrying...",
+ dataTableName, encodedRegionName, retryCount, sleepTime, e);
+ sleepIfNotStopped(sleepTime);
+ }
+ }
+ } catch (InterruptedException e) {
+ if (!stopped) {
+ LOG.warn("IndexCDCConsumer interrupted unexpectedly for table {} region {}", dataTableName,
+ encodedRegionName, e);
+ }
+ Thread.currentThread().interrupt();
+ }
+ LOG.info("IndexCDCConsumer exiting for table {} region {}", dataTableName, encodedRegionName);
+ }
+
+ private boolean hasEventuallyConsistentIndexes() throws InterruptedException {
+ int retryCount = 0;
+ while (!stopped) {
+ try (PhoenixConnection conn =
+ QueryUtil.getConnectionOnServer(config).unwrap(PhoenixConnection.class)) {
+ refreshDataTableCache(conn);
+ PTable dataTable = getDataTable(conn);
+ String cdcObjectName = CDCUtil.getCDCObjectName(dataTable, false);
+ if (cdcObjectName == null) {
+ LOG.debug("No CDC index found for table {}. Exiting consumer.", dataTableName);
+ return false;
+ }
+ for (PTable index : dataTable.getIndexes()) {
+ IndexConsistency consistency = index.getIndexConsistency();
+ if (consistency != null && consistency.isAsynchronous()) {
+ LOG.debug("Found eventually consistent index {} for table {}",
+ index.getName().getString(), dataTableName);
+ return true;
+ }
+ }
+ return false;
+ } catch (SQLException e) {
+ long sleepTime = ConnectionUtils.getPauseTime(pause, ++retryCount);
+ LOG.warn(
+ "Error checking for eventually consistent indexes for table {}. "
+ + "Retry #{}, sleeping {} ms before retrying...",
+ dataTableName, retryCount, sleepTime, e);
+ sleepIfNotStopped(sleepTime);
+ }
+ }
+ return false;
+ }
+
+ /**
+ * Retrieves the count of partitions for the given table.
+ * @return the count of CDC_STREAM rows for this table.
+ * @throws InterruptedException if the thread is interrupted while waiting.
+ */
+ private long getCDCStreamNumPartitions() throws InterruptedException {
+ int retryCount = 0;
+ String query = "SELECT COUNT(*) FROM " + PhoenixDatabaseMetaData.SYSTEM_CDC_STREAM_NAME
+ + " WHERE TABLE_NAME = ?";
+ while (!stopped) {
+ try (
+ PhoenixConnection conn =
+ QueryUtil.getConnectionOnServer(config).unwrap(PhoenixConnection.class);
+ PreparedStatement ps = conn.prepareStatement(query)) {
+ ps.setString(1, dataTableName);
+ try (ResultSet rs = ps.executeQuery()) {
+ if (rs.next()) {
+ return rs.getLong(1);
+ }
+ }
+ return 0;
+ } catch (SQLException e) {
+ if (e instanceof TableNotFoundException) {
+ TableNotFoundException tnfe = (TableNotFoundException) e;
+ // 5.3.0+ server with old metadata tables (i.e. EXECUTE_UPGRADE is not yet run)
+ if (PhoenixDatabaseMetaData.SYSTEM_CDC_STREAM_TABLE.equals(tnfe.getTableName())) {
+ stopped = true;
+ return -1;
+ }
+ }
+ long sleepTime = ConnectionUtils.getPauseTime(pause, ++retryCount);
+ LOG.warn(
+ "Error getting CDC_STREAM row count for table {}. "
+ + "Retry #{}, sleeping {} ms before retrying...",
+ dataTableName, retryCount, sleepTime, e);
+ sleepIfNotStopped(sleepTime);
+ }
+ }
+ return -1;
+ }
+
+ /**
+ * Waits for the CDC_STREAM entry for the given table and partition to be available.
+ * @return true if the entry was found, false if the consumer was stopped before finding the
+ * entry.
+ * @throws InterruptedException if the thread is interrupted while waiting.
+ */
+ private boolean waitForCDCStreamEntry() throws InterruptedException {
+ int retryCount = 0;
+ String query =
+ "SELECT PARENT_PARTITION_ID FROM " + PhoenixDatabaseMetaData.SYSTEM_CDC_STREAM_NAME
+ + " WHERE TABLE_NAME = ? AND PARTITION_ID = ? LIMIT 1";
+ while (!stopped) {
+ try (
+ PhoenixConnection conn =
+ QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
+ PreparedStatement ps = conn.prepareStatement(query)) {
+ ps.setString(1, dataTableName);
+ ps.setString(2, encodedRegionName);
+ try (ResultSet rs = ps.executeQuery()) {
+ if (rs.next()) {
+ String parentPartitionId = rs.getString(1);
+ hasParentPartitions = parentPartitionId != null;
+ LOG.debug("Found CDC_STREAM entry for table {} partition {}, hasParentPartitions={}",
+ dataTableName, encodedRegionName, hasParentPartitions);
+ return true;
+ }
+ }
+ long sleepTime = ConnectionUtils.getPauseTime(pause, ++retryCount);
+ LOG.info(
+ "CDC_STREAM entry not found for table {} partition {}. "
+ + "Attempt #{}, sleeping {} ms before retrying...",
+ dataTableName, encodedRegionName, retryCount, sleepTime);
+ sleepIfNotStopped(sleepTime);
+ } catch (SQLException e) {
+ long sleepTime = ConnectionUtils.getPauseTime(pause, ++retryCount);
+ LOG.warn(
+ "Error checking CDC_STREAM for table {} partition {}. "
+ + "Retry #{}, sleeping {} ms before retrying...",
+ dataTableName, encodedRegionName, retryCount, sleepTime, e);
+ sleepIfNotStopped(sleepTime);
+ }
+ }
+ return false;
+ }
+
+ /**
+ * Checks for an existing entry for the given partition and owner.
+ * @param partitionId the partition ID to check.
+ * @param ownerPartitionId the owner partition ID.
+ * @return the last processed timestamp if an entry exists with status IN_PROGRESS, -1 if the
+ * entry exists with status COMPLETE, or 0 if no entry exists.
+ * @throws InterruptedException if the thread is interrupted while waiting
+ */
+ private long checkTrackerStatus(String partitionId, String ownerPartitionId)
+ throws InterruptedException {
+ int retryCount = 0;
+ String query =
+ "SELECT LAST_TIMESTAMP, STATUS FROM " + PhoenixDatabaseMetaData.SYSTEM_IDX_CDC_TRACKER_NAME
+ + " WHERE TABLE_NAME = ? AND PARTITION_ID = ? AND OWNER_PARTITION_ID = ?";
+ while (!stopped) {
+ try (
+ PhoenixConnection conn =
+ QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
+ PreparedStatement ps = conn.prepareStatement(query)) {
+ ps.setString(1, dataTableName);
+ ps.setString(2, partitionId);
+ ps.setString(3, ownerPartitionId);
+ try (ResultSet rs = ps.executeQuery()) {
+ if (rs.next()) {
+ long lastTimestamp = rs.getLong(1);
+ String status = rs.getString(2);
+ LOG.debug(
+ "Found IDX_CDC_TRACKER entry for table {} partition {} owner {} "
+ + "with status={}, lastTimestamp={}",
+ dataTableName, partitionId, ownerPartitionId, status, lastTimestamp);
+ if (PhoenixDatabaseMetaData.TRACKER_STATUS_IN_PROGRESS.equals(status)) {
+ return lastTimestamp;
+ } else if (PhoenixDatabaseMetaData.TRACKER_STATUS_COMPLETE.equals(status)) {
+ return -1;
+ } else {
+ // Unknown status - should not happen
+ throw new IllegalStateException(
+ String.format("Unknown tracker status '%s' for table %s partition %s owner %s.",
+ status, dataTableName, partitionId, ownerPartitionId));
+ }
+ } else {
+ LOG.debug("No IDX_CDC_TRACKER entry found for table {} partition {} owner {}",
+ dataTableName, partitionId, ownerPartitionId);
+ return 0;
+ }
+ }
+ } catch (SQLException e) {
+ long sleepTime = ConnectionUtils.getPauseTime(pause, ++retryCount);
+ LOG.warn(
+ "Error checking IDX_CDC_TRACKER for table {} partition {} owner {}. "
+ + "Retry #{}, sleeping {} ms before retrying...",
+ dataTableName, partitionId, ownerPartitionId, retryCount, sleepTime, e);
+ sleepIfNotStopped(sleepTime);
+ }
+ }
+ return 0;
+ }
+
+ /**
+ * Checks if any consumer has already completed processing the given partition.
+ * @param partitionId the partition ID to check
+ * @return true if the partition is done processing.
+ * @throws InterruptedException if the thread is interrupted while waiting.
+ */
+ private boolean isPartitionCompleted(String partitionId) throws InterruptedException {
+ int retryCount = 0;
+ String query = "SELECT * FROM " + PhoenixDatabaseMetaData.SYSTEM_IDX_CDC_TRACKER_NAME
+ + " WHERE TABLE_NAME = ? AND PARTITION_ID = ? AND STATUS = ?";
+ while (!stopped) {
+ try (
+ PhoenixConnection conn =
+ QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
+ PreparedStatement ps = conn.prepareStatement(query)) {
+ ps.setString(1, dataTableName);
+ ps.setString(2, partitionId);
+ ps.setString(3, PhoenixDatabaseMetaData.TRACKER_STATUS_COMPLETE);
+ try (ResultSet rs = ps.executeQuery()) {
+ return rs.next();
+ }
+ } catch (SQLException e) {
+ long sleepTime = ConnectionUtils.getPauseTime(pause, ++retryCount);
+ LOG.warn(
+ "Error checking if partition {} is completed for table {}. "
+ + "Retry #{}, sleeping {} ms before retrying...",
+ partitionId, dataTableName, retryCount, sleepTime, e);
+ sleepIfNotStopped(sleepTime);
+ }
+ }
+ return false;
+ }
+
+ /**
+ * Gets the maximum last processed timestamp from any consumer's tracker record for the given
+ * partition. This is used when a child region needs to continue processing from where any
+ * previous consumer left off.
+ * @param partitionId the partition ID to get progress for.
+ * @return the max last processed timestamp across all owners, or 0 if not found.
+ * @throws InterruptedException if the thread is interrupted while waiting.
+ */
+ private long getParentProgress(String partitionId) throws InterruptedException {
+ int retryCount = 0;
+ String query =
+ "SELECT MAX(LAST_TIMESTAMP) FROM " + PhoenixDatabaseMetaData.SYSTEM_IDX_CDC_TRACKER_NAME
+ + " WHERE TABLE_NAME = ? AND PARTITION_ID = ? AND STATUS = ?";
+ while (!stopped) {
+ try (
+ PhoenixConnection conn =
+ QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
+ PreparedStatement ps = conn.prepareStatement(query)) {
+ ps.setString(1, dataTableName);
+ ps.setString(2, partitionId);
+ ps.setString(3, PhoenixDatabaseMetaData.TRACKER_STATUS_IN_PROGRESS);
+ try (ResultSet rs = ps.executeQuery()) {
+ if (rs.next()) {
+ long maxTimestamp = rs.getLong(1);
+ if (maxTimestamp > 0) {
+ LOG.debug("Found max progress {} for partition {} from previous consumers",
+ maxTimestamp, partitionId);
+ return maxTimestamp;
+ }
+ }
+ return 0;
+ }
+ } catch (SQLException e) {
+ long sleepTime = ConnectionUtils.getPauseTime(pause, ++retryCount);
+ LOG.warn(
+ "Error getting parent progress for partition {} table {}. "
+ + "Retry #{}, sleeping {} ms before retrying...",
+ partitionId, dataTableName, retryCount, sleepTime, e);
+ sleepIfNotStopped(sleepTime);
+ }
+ }
+ throw new InterruptedException("IndexCDCConsumer stopped while getting parent progress.");
+ }
+
+ /**
+ * Retrieves all parent partition IDs for the given partition.
+ * @param partitionId the partition ID to find parents for.
+ * @return list of parent partition IDs.
+ * @throws InterruptedException if the thread is interrupted.
+ */
+ private List getParentPartitionIds(String partitionId) throws InterruptedException {
+ int retryCount = 0;
+ String query =
+ "SELECT PARENT_PARTITION_ID FROM " + PhoenixDatabaseMetaData.SYSTEM_CDC_STREAM_NAME
+ + " WHERE TABLE_NAME = ? AND PARTITION_ID = ? AND PARENT_PARTITION_ID IS NOT NULL";
+ while (!stopped) {
+ try (
+ PhoenixConnection conn =
+ QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
+ PreparedStatement ps = conn.prepareStatement(query)) {
+ ps.setString(1, dataTableName);
+ ps.setString(2, partitionId);
+ List parentIds = new ArrayList<>();
+ try (ResultSet rs = ps.executeQuery()) {
+ while (rs.next()) {
+ parentIds.add(rs.getString(1));
+ }
+ }
+ LOG.debug("Found {} parent partition(s) for table {} partition {}: {}", parentIds.size(),
+ dataTableName, partitionId, parentIds);
+ return parentIds;
+ } catch (SQLException e) {
+ long sleepTime = ConnectionUtils.getPauseTime(pause, ++retryCount);
+ LOG.warn(
+ "Error querying parent partitions from CDC_STREAM for table {} partition {}. "
+ + "Retry #{}, sleeping {} ms before retrying...",
+ dataTableName, partitionId, retryCount, sleepTime, e);
+ sleepIfNotStopped(sleepTime);
+ }
+ }
+ return Collections.emptyList();
+ }
+
+ /**
+ * Processes all remaining CDC mutations for the given partition until completion, then marks the
+ * partition as COMPLETE. Used to replay ancestor partitions (regions) after they are split or
+ * merged into new regions.
+ * @param partitionId the partition ID to process.
+ * @param ownerPartitionId the owner partition ID.
+ * @param lastProcessedTimestamp the timestamp to start processing from.
+ * @throws InterruptedException if the thread is interrupted.
+ */
+ private void processPartitionToCompletion(String partitionId, String ownerPartitionId,
+ long lastProcessedTimestamp) throws InterruptedException {
+ LOG.debug(
+ "Processing partition {} owner {} to completion for table {}, starting from timestamp {}",
+ partitionId, ownerPartitionId, dataTableName, lastProcessedTimestamp);
+ long currentLastProcessedTimestamp = lastProcessedTimestamp;
+ int retryCount = 0;
+ int batchCount = 0;
+ while (!stopped) {
+ try {
+ if (batchCount > 0) {
+ if (isPartitionCompleted(partitionId)) {
+ return;
+ }
+ long otherProgress = getParentProgress(partitionId);
+ if (otherProgress > currentLastProcessedTimestamp) {
+ long previousOtherProgress;
+ do {
+ previousOtherProgress = otherProgress;
+ sleepIfNotStopped(parentProgressPauseMs);
+ if (isPartitionCompleted(partitionId)) {
+ return;
+ }
+ otherProgress = getParentProgress(partitionId);
+ } while (!stopped && otherProgress > previousOtherProgress);
+ currentLastProcessedTimestamp = otherProgress;
+ }
+ }
+ long newTimestamp;
+ if (serializeCDCMutations) {
+ newTimestamp =
+ processCDCBatch(partitionId, ownerPartitionId, currentLastProcessedTimestamp, true);
+ } else {
+ newTimestamp = processCDCBatchGenerated(partitionId, ownerPartitionId,
+ currentLastProcessedTimestamp, true);
+ }
+ batchCount++;
+ retryCount = 0;
+ if (newTimestamp == currentLastProcessedTimestamp) {
+ if (isPartitionCompleted(partitionId)) {
+ LOG.info(
+ "Partition {} for table {} was completed by another consumer before {} could mark it",
+ partitionId, dataTableName, ownerPartitionId);
+ return;
+ }
+ LOG.info("Partition {} owner {} for table {} fully processed, marking as COMPLETE",
+ partitionId, ownerPartitionId, dataTableName);
+ try (PhoenixConnection conn = QueryUtil.getConnectionOnServer(env.getConfiguration())
+ .unwrap(PhoenixConnection.class)) {
+ updateTrackerProgress(conn, partitionId, ownerPartitionId,
+ currentLastProcessedTimestamp, PhoenixDatabaseMetaData.TRACKER_STATUS_COMPLETE);
+ }
+ return;
+ }
+ currentLastProcessedTimestamp = newTimestamp;
+ } catch (SQLException | IOException e) {
+ metricSource.incrementCdcBatchFailureCount(dataTableName);
+ long sleepTime = ConnectionUtils.getPauseTime(pause, ++retryCount);
+ LOG.warn(
+ "Error processing CDC batch for partition {} owner {} table {} "
+ + "lastProcessedTimestamp {}. Retry #{}, sleeping {} ms",
+ partitionId, ownerPartitionId, dataTableName, currentLastProcessedTimestamp, retryCount,
+ sleepTime, e);
+ sleepIfNotStopped(sleepTime);
+ }
+ }
+ LOG.info("Processing partition {} (owner {}) stopped before completion for table {}",
+ partitionId, ownerPartitionId, dataTableName);
+ }
+
+ /**
+ * Recursive helper method to replay and complete parent regions. Replays all remaining mutations
+ * from parent regions and marks them as COMPLETE. This is called during initialization when a
+ * region has parent regions (split/merge scenarios).
+ * @param partitionId the partition (region) ID to find and process parents for.
+ */
+ private void replayAndCompleteParentRegions(String partitionId) throws InterruptedException {
+ List parentIds = getParentPartitionIds(partitionId);
+ if (parentIds.isEmpty()) {
+ LOG.debug("No parent partitions found for table {} partition {}", dataTableName, partitionId);
+ return;
+ }
+ for (String parentId : parentIds) {
+ if (stopped) {
+ return;
+ }
+ if (isPartitionCompleted(parentId)) {
+ LOG.debug("Parent partition {} for table {} already has a COMPLETE record, skipping",
+ parentId, dataTableName);
+ continue;
+ }
+ replayAndCompleteParentRegions(parentId);
+ if (isPartitionCompleted(parentId)) {
+ LOG.debug(
+ "Parent partition {} for table {} was completed by sibling during ancestor processing, "
+ + "skipping",
+ parentId, dataTableName);
+ continue;
+ }
+ long parentProgress = getParentProgress(parentId);
+ LOG.debug("Processing/Resuming parent partition {} for table {} from timestamp {} owner: {}",
+ parentId, dataTableName, parentProgress, encodedRegionName);
+ processPartitionToCompletion(parentId, encodedRegionName, parentProgress);
+ }
+ }
+
+ /**
+ * Processes a batch of CDC mutations for the given partition starting from the specified
+ * timestamp. This method reads mutations from the CDC index, applies them to the respective
+ * eventually consistent index tables, and updates the progress in SYSTEM.IDX_CDC_TRACKER.
+ * @param partitionId the partition (region) ID to process CDC mutations for.
+ * @param ownerPartitionId the owner partition ID.
+ * @param lastProcessedTimestamp the timestamp to start reading CDC mutations from.
+ * @param isParentReplay true if replaying a closed parent partition.
+ * @return the new last processed timestamp after this batch, or the same timestamp if no new
+ * records were found.
+ * @throws SQLException if SQL error occurs.
+ * @throws IOException if an I/O error occurs.
+ * @throws InterruptedException if the thread is interrupted while waiting.
+ */
+ private long processCDCBatch(String partitionId, String ownerPartitionId,
+ long lastProcessedTimestamp, boolean isParentReplay)
+ throws SQLException, IOException, InterruptedException {
+ long batchStartTime = EnvironmentEdgeManager.currentTimeMillis();
+ LOG.debug("Processing CDC batch for table {} partition {} owner {} from timestamp {}",
+ dataTableName, partitionId, ownerPartitionId, lastProcessedTimestamp);
+ try (PhoenixConnection conn =
+ QueryUtil.getConnectionOnServer(config).unwrap(PhoenixConnection.class)) {
+ initTenantInfo(conn);
+ String cdcObjectName = getCdcObjectName(conn);
+ TenantScanInfo scanInfo = getPartitionTenantScanInfo(partitionId);
+ String cdcQuery;
+ if (isParentReplay) {
+ cdcQuery = String.format(
+ "SELECT /*+ CDC_INCLUDE(IDX_MUTATIONS) */ PHOENIX_ROW_TIMESTAMP(), \"CDC JSON\" "
+ + "FROM %s WHERE %s PARTITION_ID() = ? AND PHOENIX_ROW_TIMESTAMP() > ? "
+ + "ORDER BY %s PARTITION_ID() ASC, PHOENIX_ROW_TIMESTAMP() ASC LIMIT ?",
+ cdcObjectName, scanInfo.filter, scanInfo.orderBy);
+ } else {
+ cdcQuery = String.format(
+ "SELECT /*+ CDC_INCLUDE(IDX_MUTATIONS) */ PHOENIX_ROW_TIMESTAMP(), \"CDC JSON\" "
+ + "FROM %s WHERE %s PARTITION_ID() = ? AND PHOENIX_ROW_TIMESTAMP() > ? "
+ + "AND PHOENIX_ROW_TIMESTAMP() < ? "
+ + "ORDER BY %s PARTITION_ID() ASC, PHOENIX_ROW_TIMESTAMP() ASC LIMIT ?",
+ cdcObjectName, scanInfo.filter, scanInfo.orderBy);
+ }
+ List> batchMutations = new ArrayList<>();
+ long newLastTimestamp = lastProcessedTimestamp;
+ boolean hasMoreRows = true;
+ int retryCount = 0;
+ while (hasMoreRows && batchMutations.isEmpty()) {
+ try (PreparedStatement ps = conn.prepareStatement(cdcQuery)) {
+ setStatementParams(scanInfo, partitionId, isParentReplay, newLastTimestamp, ps);
+ Pair result =
+ getMutationsAndTimestamp(ps, newLastTimestamp, batchMutations);
+ hasMoreRows = result.getSecond();
+ if (hasMoreRows) {
+ newLastTimestamp = result.getFirst();
+ if (batchMutations.isEmpty()) {
+ sleepIfNotStopped(ConnectionUtils.getPauseTime(pause, ++retryCount));
+ }
+ }
+ }
+ }
+ // With predefined LIMIT, there might be more rows with the same timestamp that were not
+ // included in this batch.
+ if (newLastTimestamp > lastProcessedTimestamp) {
+ String sameTimestampQuery = String.format(
+ "SELECT /*+ CDC_INCLUDE(IDX_MUTATIONS) */ PHOENIX_ROW_TIMESTAMP(), \"CDC JSON\" "
+ + "FROM %s WHERE %s PARTITION_ID() = ? AND PHOENIX_ROW_TIMESTAMP() = ? "
+ + "ORDER BY %s PARTITION_ID() ASC, PHOENIX_ROW_TIMESTAMP() ASC",
+ cdcObjectName, scanInfo.filter, scanInfo.orderBy);
+ final long timestampToRefetch = newLastTimestamp;
+ batchMutations.removeIf(pair -> pair.getFirst() == timestampToRefetch);
+ try (PreparedStatement ps = conn.prepareStatement(sameTimestampQuery)) {
+ int idx = scanInfo.bindParams(ps, 1);
+ ps.setString(idx++, partitionId);
+ ps.setDate(idx, new Date(newLastTimestamp));
+ Pair result =
+ getMutationsAndTimestamp(ps, newLastTimestamp, batchMutations);
+ newLastTimestamp = result.getFirst();
+ if (newLastTimestamp != timestampToRefetch) {
+ throw new IOException("Unexpected timestamp mismatch: expected " + timestampToRefetch
+ + " but got " + newLastTimestamp);
+ }
+ }
+ }
+ executeIndexMutations(partitionId, batchMutations, ownerPartitionId, newLastTimestamp);
+ if (!batchMutations.isEmpty()) {
+ metricSource.updateCdcBatchProcessTime(dataTableName,
+ EnvironmentEdgeManager.currentTimeMillis() - batchStartTime);
+ metricSource.incrementCdcBatchCount(dataTableName);
+ metricSource.updateCdcLag(dataTableName,
+ EnvironmentEdgeManager.currentTimeMillis() - newLastTimestamp);
+ updateTrackerProgress(conn, partitionId, ownerPartitionId, newLastTimestamp,
+ PhoenixDatabaseMetaData.TRACKER_STATUS_IN_PROGRESS);
+ }
+ return newLastTimestamp;
+ }
+ }
+
+ private String getCdcObjectName(PhoenixConnection conn) throws SQLException {
+ PTable dataTable = getDataTable(conn);
+ String cdcObjectName = CDCUtil.getCDCObjectName(dataTable, false);
+ if (cdcObjectName == null) {
+ throw new SQLException("No CDC object found for table " + dataTableName);
+ }
+ return SchemaUtil.getEscapedTableName(dataTable.getSchemaName().getString(), cdcObjectName);
+ }
+
+ /**
+ * Processes a batch of CDC events for the given partition starting from the specified timestamp
+ * by generating index mutations from data row states. This method queries the CDC index with the
+ * DATA_ROW_STATE scope, which triggers a server-side data table scan to reconstruct the
+ * before-image ({@code currentDataRowState}) and after-image ({@code nextDataRowState}) for each
+ * change.
+ * @param partitionId the partition (region) ID to process CDC events for.
+ * @param ownerPartitionId the owner partition ID.
+ * @param lastProcessedTimestamp the timestamp to start processing CDC events from.
+ * @param isParentReplay true if replaying a closed parent partition.
+ * @return the new last processed timestamp after this batch, or the same timestamp if no new
+ * records were found.
+ * @throws SQLException if a SQL error occurs.
+ * @throws IOException if an I/O error occurs.
+ * @throws InterruptedException if the thread is interrupted while waiting.
+ */
+ private long processCDCBatchGenerated(String partitionId, String ownerPartitionId,
+ long lastProcessedTimestamp, boolean isParentReplay)
+ throws SQLException, IOException, InterruptedException {
+ long batchStartTime = EnvironmentEdgeManager.currentTimeMillis();
+ LOG.debug(
+ "Processing CDC batch (generated mode) for table {} partition {} owner {} from timestamp {}",
+ dataTableName, partitionId, ownerPartitionId, lastProcessedTimestamp);
+ try (PhoenixConnection conn =
+ QueryUtil.getConnectionOnServer(config).unwrap(PhoenixConnection.class)) {
+ initTenantInfo(conn);
+ String cdcObjectName = getCdcObjectName(conn);
+ TenantScanInfo scanInfo = getPartitionTenantScanInfo(partitionId);
+ String cdcQuery;
+ if (isParentReplay) {
+ cdcQuery = String.format(
+ "SELECT /*+ CDC_INCLUDE(DATA_ROW_STATE) */ PHOENIX_ROW_TIMESTAMP(), \"CDC JSON\" "
+ + "FROM %s WHERE %s PARTITION_ID() = ? AND PHOENIX_ROW_TIMESTAMP() > ? "
+ + "ORDER BY %s PARTITION_ID() ASC, PHOENIX_ROW_TIMESTAMP() ASC LIMIT ?",
+ cdcObjectName, scanInfo.filter, scanInfo.orderBy);
+ } else {
+ cdcQuery = String.format(
+ "SELECT /*+ CDC_INCLUDE(DATA_ROW_STATE) */ PHOENIX_ROW_TIMESTAMP(), \"CDC JSON\" "
+ + "FROM %s WHERE %s PARTITION_ID() = ? AND PHOENIX_ROW_TIMESTAMP() > ? "
+ + "AND PHOENIX_ROW_TIMESTAMP() < ? "
+ + "ORDER BY %s PARTITION_ID() ASC, PHOENIX_ROW_TIMESTAMP() ASC LIMIT ?",
+ cdcObjectName, scanInfo.filter, scanInfo.orderBy);
+ }
+
+ List> batchStates = new ArrayList<>();
+ long newLastTimestamp = lastProcessedTimestamp;
+ long[] lastScannedTimestamp = { lastProcessedTimestamp };
+ boolean hasMoreRows = true;
+ int retryCount = 0;
+ while (hasMoreRows && batchStates.isEmpty()) {
+ try (PreparedStatement ps = conn.prepareStatement(cdcQuery)) {
+ setStatementParams(scanInfo, partitionId, isParentReplay, newLastTimestamp, ps);
+ Pair result =
+ getDataRowStatesAndTimestamp(ps, newLastTimestamp, batchStates, lastScannedTimestamp);
+ hasMoreRows = result.getSecond();
+ if (hasMoreRows) {
+ if (!batchStates.isEmpty()) {
+ newLastTimestamp = result.getFirst();
+ } else if (retryCount >= maxDataVisibilityRetries) {
+ LOG.warn(
+ "Skipping CDC events for table {} partition {} from timestamp {}"
+ + " to {} after {} retries — data table mutations may have failed",
+ dataTableName, partitionId, newLastTimestamp, lastScannedTimestamp[0], retryCount);
+ newLastTimestamp = lastScannedTimestamp[0];
+ break;
+ } else {
+ // CDC index entries are written but the data is not yet visible.
+ // Don't advance newLastTimestamp so the same events are re-fetched
+ // once the data becomes visible.
+ sleepIfNotStopped(ConnectionUtils.getPauseTime(pause, ++retryCount));
+ }
+ }
+ }
+ }
+ if (newLastTimestamp > lastProcessedTimestamp) {
+ String sameTimestampQuery = String.format(
+ "SELECT /*+ CDC_INCLUDE(DATA_ROW_STATE) */ PHOENIX_ROW_TIMESTAMP(), \"CDC JSON\" "
+ + "FROM %s WHERE %s PARTITION_ID() = ? AND PHOENIX_ROW_TIMESTAMP() = ? "
+ + "ORDER BY %s PARTITION_ID() ASC, PHOENIX_ROW_TIMESTAMP() ASC",
+ cdcObjectName, scanInfo.filter, scanInfo.orderBy);
+ final long timestampToRefetch = newLastTimestamp;
+ batchStates.removeIf(pair -> pair.getFirst() == timestampToRefetch);
+ try (PreparedStatement ps = conn.prepareStatement(sameTimestampQuery)) {
+ int idx = scanInfo.bindParams(ps, 1);
+ ps.setString(idx++, partitionId);
+ ps.setDate(idx, new Date(newLastTimestamp));
+ Pair result =
+ getDataRowStatesAndTimestamp(ps, newLastTimestamp, batchStates, lastScannedTimestamp);
+ newLastTimestamp = result.getFirst();
+ if (batchStates.isEmpty()) {
+ newLastTimestamp = timestampToRefetch;
+ } else if (newLastTimestamp != timestampToRefetch) {
+ throw new IOException("Unexpected timestamp mismatch: expected " + timestampToRefetch
+ + " but got " + newLastTimestamp);
+ }
+ }
+ }
+ generateAndApplyIndexMutations(conn, batchStates, partitionId, ownerPartitionId,
+ newLastTimestamp);
+ if (!batchStates.isEmpty()) {
+ metricSource.updateCdcBatchProcessTime(dataTableName,
+ EnvironmentEdgeManager.currentTimeMillis() - batchStartTime);
+ metricSource.incrementCdcBatchCount(dataTableName);
+ metricSource.updateCdcLag(dataTableName,
+ EnvironmentEdgeManager.currentTimeMillis() - newLastTimestamp);
+ }
+ if (newLastTimestamp > lastProcessedTimestamp) {
+ updateTrackerProgress(conn, partitionId, ownerPartitionId, newLastTimestamp,
+ PhoenixDatabaseMetaData.TRACKER_STATUS_IN_PROGRESS);
+ }
+ return newLastTimestamp;
+ }
+ }
+
+ private void setStatementParams(TenantScanInfo scanInfo, String partitionId,
+ boolean isParentReplay, long newLastTimestamp, PreparedStatement ps) throws SQLException {
+ int idx = scanInfo.bindParams(ps, 1);
+ ps.setString(idx++, partitionId);
+ ps.setDate(idx++, new Date(newLastTimestamp));
+ if (isParentReplay) {
+ ps.setInt(idx, batchSize);
+ } else {
+ long currentTime = EnvironmentEdgeManager.currentTimeMillis() - timestampBufferMs;
+ ps.setDate(idx++, new Date(currentTime));
+ ps.setInt(idx, batchSize);
+ }
+ }
+
+ private static Pair getDataRowStatesAndTimestamp(PreparedStatement ps,
+ long initialLastTimestamp, List> batchStates,
+ long[] lastScannedTimestamp) throws SQLException, IOException {
+ boolean hasRows = false;
+ long lastTimestamp = initialLastTimestamp;
+ lastScannedTimestamp[0] = initialLastTimestamp;
+ try (ResultSet rs = ps.executeQuery()) {
+ while (rs.next()) {
+ hasRows = true;
+ long rowTimestamp = rs.getDate(1).getTime();
+ lastScannedTimestamp[0] = rowTimestamp;
+ String cdcValue = rs.getString(2);
+ if (cdcValue != null && !cdcValue.isEmpty()) {
+ byte[] protoBytes = Base64.getDecoder().decode(cdcValue);
+ IndexMutationsProtos.DataRowStates dataRowStates =
+ IndexMutationsProtos.DataRowStates.parseFrom(protoBytes);
+ if (
+ dataRowStates.hasDataRowKey()
+ && (dataRowStates.hasCurrentDataRowState() || dataRowStates.hasNextDataRowState())
+ ) {
+ batchStates.add(Pair.newPair(rowTimestamp, dataRowStates));
+ lastTimestamp = rowTimestamp;
+ }
+ }
+ }
+ }
+ return Pair.newPair(lastTimestamp, hasRows);
+ }
+
+ private void generateAndApplyIndexMutations(PhoenixConnection conn,
+ List> batchStates, String partitionId,
+ String ownerPartitionId, long lastProcessedTimestamp) throws SQLException, IOException {
+ if (batchStates.isEmpty()) {
+ return;
+ }
+ refreshDataTableCache(conn);
+ PTable dataTable = getDataTable(conn);
+ byte[] encodedRegionNameBytes = env.getRegion().getRegionInfo().getEncodedNameAsBytes();
+ List> indexTables = new ArrayList<>();
+ for (PTable index : dataTable.getIndexes()) {
+ IndexConsistency consistency = index.getIndexConsistency();
+ if (consistency != null && consistency.isAsynchronous()) {
+ IndexMaintainer maintainer = index.getIndexMaintainer(dataTable, conn);
+ HTableInterfaceReference tableRef =
+ new HTableInterfaceReference(new ImmutableBytesPtr(maintainer.getIndexTableName()));
+ indexTables.add(new Pair<>(maintainer, tableRef));
+ }
+ }
+ if (indexTables.isEmpty()) {
+ return;
+ }
+ ListMultimap indexUpdates = ArrayListMultimap.create();
+ int totalMutations = 0;
+ long generateStartTime = EnvironmentEdgeManager.currentTimeMillis();
+ for (Pair entry : batchStates) {
+ long ts = entry.getFirst();
+ IndexMutationsProtos.DataRowStates dataRowStates = entry.getSecond();
+ byte[] dataRowKey = dataRowStates.getDataRowKey().toByteArray();
+ ImmutableBytesPtr rowKeyPtr = new ImmutableBytesPtr(dataRowKey);
+
+ Put currentDataRowState = null;
+ if (dataRowStates.hasCurrentDataRowState()) {
+ ClientProtos.MutationProto currentProto = ClientProtos.MutationProto
+ .parseFrom(dataRowStates.getCurrentDataRowState().toByteArray());
+ Mutation currentMutation = ProtobufUtil.toMutation(currentProto);
+ if (currentMutation instanceof Put) {
+ currentDataRowState = (Put) currentMutation;
+ }
+ }
+ Put nextDataRowState = null;
+ if (dataRowStates.hasNextDataRowState()) {
+ ClientProtos.MutationProto nextProto =
+ ClientProtos.MutationProto.parseFrom(dataRowStates.getNextDataRowState().toByteArray());
+ Mutation nextMutation = ProtobufUtil.toMutation(nextProto);
+ if (nextMutation instanceof Put) {
+ nextDataRowState = (Put) nextMutation;
+ }
+ }
+ if (currentDataRowState == null && nextDataRowState == null) {
+ continue;
+ }
+ IndexRegionObserver.generateIndexMutationsForRow(rowKeyPtr, currentDataRowState,
+ nextDataRowState, ts, encodedRegionNameBytes, QueryConstants.VERIFIED_BYTES, indexTables,
+ indexUpdates);
+ if (indexUpdates.size() >= batchSize) {
+ metricSource.updateCdcMutationGenerateTime(dataTableName,
+ EnvironmentEdgeManager.currentTimeMillis() - generateStartTime);
+ long applyStartTime = EnvironmentEdgeManager.currentTimeMillis();
+ indexWriter.write(indexUpdates, false, MetaDataProtocol.PHOENIX_VERSION);
+ metricSource.updateCdcMutationApplyTime(dataTableName,
+ EnvironmentEdgeManager.currentTimeMillis() - applyStartTime);
+ totalMutations += indexUpdates.size();
+ indexUpdates.clear();
+ generateStartTime = EnvironmentEdgeManager.currentTimeMillis();
+ }
+ }
+ if (!indexUpdates.isEmpty()) {
+ metricSource.updateCdcMutationGenerateTime(dataTableName,
+ EnvironmentEdgeManager.currentTimeMillis() - generateStartTime);
+ long applyStartTime = EnvironmentEdgeManager.currentTimeMillis();
+ indexWriter.write(indexUpdates, false, MetaDataProtocol.PHOENIX_VERSION);
+ metricSource.updateCdcMutationApplyTime(dataTableName,
+ EnvironmentEdgeManager.currentTimeMillis() - applyStartTime);
+ totalMutations += indexUpdates.size();
+ }
+ if (totalMutations > 0) {
+ metricSource.incrementCdcMutationCount(dataTableName, totalMutations);
+ LOG.debug(
+ "Applied total {} index mutations for table {} partition {} owner {} "
+ + ", last processed timestamp {}",
+ totalMutations, dataTableName, partitionId, ownerPartitionId, lastProcessedTimestamp);
+ }
+ }
+
+ private void executeIndexMutations(String partitionId,
+ List> batchMutations, String ownerPartitionId,
+ long lastProcessedTimestamp) throws SQLException, IOException {
+ if (!batchMutations.isEmpty()) {
+ ListMultimap indexUpdates = ArrayListMultimap.create();
+ Map tableRefCache = new HashMap<>();
+ int totalMutations = 0;
+ for (Pair batchMutation : batchMutations) {
+ IndexMutationsProtos.IndexMutations mutationsProto = batchMutation.getSecond();
+ List tables = mutationsProto.getTablesList();
+ List mutations = mutationsProto.getMutationsList();
+ if (tables.size() != mutations.size()) {
+ throw new SQLException("Tables and mutations sizes do not match. Tables size: "
+ + tables.size() + ", mutations size: " + mutations.size());
+ }
+ for (int i = 0; i < tables.size(); i++) {
+ byte[] indexTableName = tables.get(i).toByteArray();
+ byte[] mutationBytes = mutations.get(i).toByteArray();
+ ImmutableBytesPtr tableNamePtr = new ImmutableBytesPtr(indexTableName);
+ HTableInterfaceReference tableRef =
+ tableRefCache.computeIfAbsent(tableNamePtr, HTableInterfaceReference::new);
+ ClientProtos.MutationProto mProto = ClientProtos.MutationProto.parseFrom(mutationBytes);
+ Mutation mutation = ProtobufUtil.toMutation(mProto);
+ indexUpdates.put(tableRef, mutation);
+ }
+ if (indexUpdates.size() >= batchSize) {
+ long applyStartTime = EnvironmentEdgeManager.currentTimeMillis();
+ indexWriter.write(indexUpdates, false, MetaDataProtocol.PHOENIX_VERSION);
+ metricSource.updateCdcMutationApplyTime(dataTableName,
+ EnvironmentEdgeManager.currentTimeMillis() - applyStartTime);
+ totalMutations += indexUpdates.size();
+ indexUpdates.clear();
+ }
+ }
+ if (!indexUpdates.isEmpty()) {
+ long applyStartTime = EnvironmentEdgeManager.currentTimeMillis();
+ indexWriter.write(indexUpdates, false, MetaDataProtocol.PHOENIX_VERSION);
+ metricSource.updateCdcMutationApplyTime(dataTableName,
+ EnvironmentEdgeManager.currentTimeMillis() - applyStartTime);
+ totalMutations += indexUpdates.size();
+ }
+ if (totalMutations > 0) {
+ metricSource.incrementCdcMutationCount(dataTableName, totalMutations);
+ LOG.debug(
+ "Applied total {} index mutations for table {} partition {} owner {} "
+ + ", last processed timestamp {}",
+ totalMutations, dataTableName, partitionId, ownerPartitionId, lastProcessedTimestamp);
+ }
+ }
+ }
+
+ private void updateTrackerProgress(PhoenixConnection conn, String partitionId,
+ String ownerPartitionId, long lastTimestamp, String status) throws SQLException {
+ String upsertSql = "UPSERT INTO " + PhoenixDatabaseMetaData.SYSTEM_IDX_CDC_TRACKER_NAME
+ + " (TABLE_NAME, PARTITION_ID, OWNER_PARTITION_ID, LAST_TIMESTAMP, STATUS) "
+ + "VALUES (?, ?, ?, ?, ?)";
+ try (PreparedStatement ps = conn.prepareStatement(upsertSql)) {
+ ps.setString(1, dataTableName);
+ ps.setString(2, partitionId);
+ ps.setString(3, ownerPartitionId);
+ ps.setLong(4, lastTimestamp);
+ ps.setString(5, status);
+ ps.executeUpdate();
+ conn.commit();
+ LOG.debug("Updated tracker for table {} partition {} owner {} to timestamp {} with status {}",
+ dataTableName, partitionId, ownerPartitionId, lastTimestamp, status);
+ }
+ }
+
+ /**
+ * Executes the prepared statement and extracts mutations and timestamps from the result set.
+ * @param ps the prepared statement to execute.
+ * @param initialLastTimestamp the initial last timestamp to use if no rows are found.
+ * @param batchMutations list to add mutations to.
+ * @return Pair of last processed timestamp and boolean to indicate if any rows were returned.
+ */
+ private static Pair getMutationsAndTimestamp(PreparedStatement ps,
+ long initialLastTimestamp, List> batchMutations)
+ throws SQLException, IOException {
+ boolean hasRows = false;
+ long lastTimestamp = initialLastTimestamp;
+ try (ResultSet rs = ps.executeQuery()) {
+ while (rs.next()) {
+ hasRows = true;
+ lastTimestamp = rs.getDate(1).getTime();
+ String cdcValue = rs.getString(2);
+ if (cdcValue != null && !cdcValue.isEmpty()) {
+ byte[] protoBytes = Base64.getDecoder().decode(cdcValue);
+ IndexMutationsProtos.IndexMutations mutationsProto =
+ IndexMutationsProtos.IndexMutations.parseFrom(protoBytes);
+ if (mutationsProto.getTablesCount() > 0 && mutationsProto.getMutationsCount() > 0) {
+ batchMutations.add(Pair.newPair(lastTimestamp, mutationsProto));
+ }
+ }
+ }
+ }
+ return Pair.newPair(lastTimestamp, hasRows);
+ }
+
+}
diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java b/phoenix-core-server/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java
index fc55b2435dc..9763388effb 100644
--- a/phoenix-core-server/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java
+++ b/phoenix-core-server/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java
@@ -27,6 +27,7 @@
import static org.apache.phoenix.index.PhoenixIndexBuilderHelper.RETURN_RESULT;
import static org.apache.phoenix.util.ByteUtil.EMPTY_BYTE_ARRAY;
+import com.google.protobuf.ByteString;
import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
import java.io.EOFException;
@@ -84,6 +85,7 @@
import org.apache.htrace.TraceScope;
import org.apache.phoenix.compile.ScanRanges;
import org.apache.phoenix.coprocessor.DelegateRegionCoprocessorEnvironment;
+import org.apache.phoenix.coprocessor.generated.IndexMutationsProtos;
import org.apache.phoenix.coprocessor.generated.PTableProtos;
import org.apache.phoenix.coprocessorclient.BaseScannerRegionObserverConstants;
import org.apache.phoenix.exception.DataExceedsCapacityException;
@@ -143,6 +145,7 @@
import org.apache.phoenix.util.ServerUtil.ConnectionType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.xerial.snappy.Snappy;
import org.apache.phoenix.thirdparty.com.google.common.base.Preconditions;
import org.apache.phoenix.thirdparty.com.google.common.collect.ArrayListMultimap;
@@ -151,6 +154,9 @@
import org.apache.phoenix.thirdparty.com.google.common.collect.Maps;
import org.apache.phoenix.thirdparty.com.google.common.collect.Sets;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+
/**
* Do all the work of managing index updates from a single coprocessor. All Puts/Delets are passed
* to an {@link IndexBuilder} to determine the actual updates to make. We don't need to implement
@@ -166,6 +172,64 @@ public class IndexRegionObserver implements RegionCoprocessor, RegionObserver {
private static final OperationStatus NOWRITE = new OperationStatus(SUCCESS);
public static final String PHOENIX_APPEND_METADATA_TO_WAL = "phoenix.append.metadata.to.wal";
public static final boolean DEFAULT_PHOENIX_APPEND_METADATA_TO_WAL = false;
+ public static final String PHOENIX_INDEX_CDC_CONSUMER_ENABLED =
+ "phoenix.index.cdc.consumer.enabled";
+ public static final boolean DEFAULT_PHOENIX_INDEX_CDC_CONSUMER_ENABLED = true;
+ public static final String PHOENIX_INDEX_CDC_MUTATIONS_COMPRESS_ENABLED =
+ "phoenix.index.cdc.mutations.compress.enabled";
+ public static final boolean DEFAULT_PHOENIX_INDEX_CDC_MUTATIONS_COMPRESS_ENABLED = false;
+ /**
+ * Controls which approach is used for implementing eventually consistent global secondary indexes
+ * via the {@link IndexCDCConsumer}.
+ *
+ * During {@code preBatchMutate}, {@link IndexRegionObserver} generates index mutations for each
+ * data table mutation and serializes them into a Protobuf {@code IndexMutations} message. This
+ * serialized payload is written as a column value in the CDC index table row alongside the CDC
+ * event. The {@link IndexCDCConsumer} later reads these pre-computed mutations from the CDC
+ * index, deserializes them, and applies them directly to the index table(s). In this approach,
+ * the consumer does not need to understand index structure or re-derive mutations — it simply
+ * replays what was already computed on the write path. The trade-off is increased CDC index row
+ * size due to the serialized mutation payload, and additional write IO on the CDC index table.
+ *
+ *
+ * Approach 2: Generated mutations from data row states (default, value = false)
+ *
+ *
+ * During {@code preBatchMutate}, {@link IndexRegionObserver} writes only a lightweight CDC index
+ * entry without serialized index mutations. Instead, the CDC event is created with the
+ * {@code DATA_ROW_STATE} scope. When the {@link IndexCDCConsumer} processes these events, it
+ * reads the CDC index rows which trigger a server-side scan of the data table (via
+ * {@code CDCGlobalIndexRegionScanner}) to reconstruct the before-image
+ * ({@code currentDataRowState}) and after-image ({@code nextDataRowState}) of the data row at the
+ * change timestamp. These raw row states are returned as a Protobuf {@code DataRowStates}
+ * message. The consumer then feeds these states into {@code generateIndexMutationsForRow()} — the
+ * same core utility used by {@link IndexRegionObserver#prepareIndexMutations} on the write path —
+ * to derive index mutations at consume time. This approach keeps CDC index rows small, avoids
+ * additional write IO, and generates mutations based on the current index definition, but
+ * requires an additional data table read per CDC event and is sensitive to data visibility
+ * timing. Make sure max lookback age is long enough to retain before and after images of the row.
+ *
+ *
+ * When to use which approach:
+ *
+ *
+ *
Use Approach 2 (serialize = false, default) to minimize write IO: no serialized
+ * mutations are written to the CDC index, keeping CDC index rows small and write latency uniform.
+ * The trade-off is higher read IO at consume time — the consumer performs an additional data
+ * table point-lookup with a raw scan per CDC event to reconstruct row states.
+ *
Use Approach 1 (serialize = true) to minimize read IO: the consumer reads
+ * pre-computed mutations from the CDC index and applies them directly, with no data table scan
+ * required at consume time. The trade-off is higher write IO — serialized index mutations are
+ * written alongside each CDC index entry, increasing CDC index row size and write-path latency.
+ * Although CDC index is expected to have TTL same as the data table max lookback age.
+ *
+ */
+ public static final String PHOENIX_INDEX_CDC_MUTATION_SERIALIZE =
+ "phoenix.index.cdc.mutation.serialize";
+ public static final boolean DEFAULT_PHOENIX_INDEX_CDC_MUTATION_SERIALIZE = false;
/**
* Class to represent pending data table rows
@@ -183,15 +247,16 @@ private class PendingRow {
this.rowKey = rowKey;
}
- public boolean add(BatchMutateContext context) {
+ public BatchMutateContext addAndGetPrevCtx(BatchMutateContext context) {
synchronized (this) {
if (usable) {
+ BatchMutateContext previousContext = lastContext;
count++;
lastContext = context;
- return true;
+ return previousContext;
}
}
- return false;
+ return null;
}
public void remove() {
@@ -208,9 +273,6 @@ public int getCount() {
return count;
}
- public BatchMutateContext getLastContext() {
- return lastContext;
- }
}
private static boolean ignoreIndexRebuildForTesting = false;
@@ -274,6 +336,15 @@ public static class BatchMutateContext {
// The collection of candidate index mutations that will be applied after the data table
// mutations.
private ListMultimap> indexUpdates;
+ // Map of data table row key to IndexMutations bytes
+ // containing pre-index mutations for eventually consistent indexes
+ // (index mutations with UNVERIFIED Puts only, no Deletes)
+ private Map cdcPreMutationsBytes;
+ // Map of data table row key to IndexMutations bytes
+ // containing post-index mutations for eventually consistent indexes
+ // (index mutations with VERIFIED Puts for covered,
+ // no Put mutations for uncovered, and Deletes if needed)
+ private Map cdcPostMutationsBytes;
private List rowLocks =
Lists.newArrayListWithExpectedSize(QueryServicesOptions.DEFAULT_MUTATE_BATCH_SIZE);
// TreeSet to improve locking efficiency and avoid deadlock (PHOENIX-6871 and HBASE-17924)
@@ -410,10 +481,14 @@ public int getMaxPendingRowCount() {
private int concurrentMutationWaitDuration;
private String dataTableName;
private boolean shouldWALAppend = DEFAULT_PHOENIX_APPEND_METADATA_TO_WAL;
+ private boolean indexCDCConsumerEnabled = DEFAULT_PHOENIX_INDEX_CDC_CONSUMER_ENABLED;
+ private boolean compressCDCMutations = DEFAULT_PHOENIX_INDEX_CDC_MUTATIONS_COMPRESS_ENABLED;
+ private boolean serializeCDCMutations = DEFAULT_PHOENIX_INDEX_CDC_MUTATION_SERIALIZE;
private boolean isNamespaceEnabled = false;
private boolean useBloomFilter = false;
private long lastTimestamp = 0;
private List> batchesWithLastTimestamp = new ArrayList<>();
+ private IndexCDCConsumer indexCDCConsumer;
private static final int DEFAULT_ROWLOCK_WAIT_DURATION = 30000;
private static final int DEFAULT_CONCURRENT_MUTATION_WAIT_DURATION_IN_MS = 100;
private byte[] encodedRegionName;
@@ -465,12 +540,27 @@ public void start(CoprocessorEnvironment e) throws IOException {
this.dataTableName = env.getRegionInfo().getTable().getNameAsString();
this.shouldWALAppend = env.getConfiguration().getBoolean(PHOENIX_APPEND_METADATA_TO_WAL,
DEFAULT_PHOENIX_APPEND_METADATA_TO_WAL);
+ this.indexCDCConsumerEnabled = env.getConfiguration()
+ .getBoolean(PHOENIX_INDEX_CDC_CONSUMER_ENABLED, DEFAULT_PHOENIX_INDEX_CDC_CONSUMER_ENABLED);
+ this.compressCDCMutations =
+ env.getConfiguration().getBoolean(PHOENIX_INDEX_CDC_MUTATIONS_COMPRESS_ENABLED,
+ DEFAULT_PHOENIX_INDEX_CDC_MUTATIONS_COMPRESS_ENABLED);
+ this.serializeCDCMutations = env.getConfiguration().getBoolean(
+ PHOENIX_INDEX_CDC_MUTATION_SERIALIZE, DEFAULT_PHOENIX_INDEX_CDC_MUTATION_SERIALIZE);
this.isNamespaceEnabled =
SchemaUtil.isNamespaceMappingEnabled(PTableType.INDEX, env.getConfiguration());
TableDescriptor tableDescriptor = env.getRegion().getTableDescriptor();
BloomType bloomFilterType = tableDescriptor.getColumnFamilies()[0].getBloomFilterType();
// when the table descriptor changes, the coproc is reloaded
this.useBloomFilter = bloomFilterType == BloomType.ROW;
+ if (
+ this.indexCDCConsumerEnabled && !this.dataTableName.startsWith("SYSTEM.")
+ && !this.dataTableName.startsWith("SYSTEM:")
+ ) {
+ this.indexCDCConsumer =
+ new IndexCDCConsumer(env, this.dataTableName, serverName, this.serializeCDCMutations);
+ this.indexCDCConsumer.start();
+ }
} catch (NoSuchMethodError ex) {
disabled = true;
LOG.error("Must be too early a version of HBase. Disabled coprocessor ", ex);
@@ -507,6 +597,9 @@ public void stop(CoprocessorEnvironment e) throws IOException {
this.builder.stop(msg);
this.preWriter.stop(msg);
this.postWriter.stop(msg);
+ if (this.indexCDCConsumer != null) {
+ this.indexCDCConsumer.stop();
+ }
}
/**
@@ -1082,8 +1175,8 @@ private void getCurrentRowStates(ObserverContext c
keys.add(PVarbinary.INSTANCE.getKeyRange(rowKeyPtr.get(), SortOrder.ASC));
} else {
// There is a pending row for this row key. We need to retrieve the row from memory
- BatchMutateContext lastContext = existingPendingRow.getLastContext();
- if (existingPendingRow.add(context)) {
+ BatchMutateContext lastContext = existingPendingRow.addAndGetPrevCtx(context);
+ if (lastContext != null) {
BatchMutatePhase phase = lastContext.getCurrentPhase();
Preconditions.checkArgument(phase != BatchMutatePhase.POST,
"the phase of the last batch cannot be POST");
@@ -1169,6 +1262,76 @@ public static Mutation getDeleteIndexMutation(Put dataRowState, IndexMaintainer
IndexMaintainer.DeleteType.ALL_VERSIONS, ts);
}
+ public static void generateIndexMutationsForRow(ImmutableBytesPtr rowKeyPtr,
+ Put currentDataRowState, Put nextDataRowState, long ts, byte[] encodedRegionName,
+ byte[] emptyColumnValue, List> indexTables,
+ ListMultimap indexUpdates) throws IOException {
+ for (Pair pair : indexTables) {
+ IndexMaintainer indexMaintainer = pair.getFirst();
+ HTableInterfaceReference hTableInterfaceReference = pair.getSecond();
+ if (
+ nextDataRowState != null && indexMaintainer.shouldPrepareIndexMutations(nextDataRowState)
+ ) {
+ ValueGetter nextDataRowVG = new IndexUtil.SimpleValueGetter(nextDataRowState);
+ Put indexPut = indexMaintainer.buildUpdateMutation(GenericKeyValueBuilder.INSTANCE,
+ nextDataRowVG, rowKeyPtr, ts, null, null, false, encodedRegionName);
+ if (indexPut == null) {
+ // No covered column. Just prepare an index row with the empty column
+ byte[] indexRowKey = indexMaintainer.buildRowKey(nextDataRowVG, rowKeyPtr, null, null, ts,
+ encodedRegionName);
+ indexPut = new Put(indexRowKey);
+ } else {
+ IndexUtil.removeEmptyColumn(indexPut,
+ indexMaintainer.getEmptyKeyValueFamily().copyBytesIfNecessary(),
+ indexMaintainer.getEmptyKeyValueQualifier());
+ }
+ byte[] finalEmptyColumnValue =
+ indexMaintainer.isUncovered() ? QueryConstants.UNVERIFIED_BYTES : emptyColumnValue;
+ indexPut.addColumn(indexMaintainer.getEmptyKeyValueFamily().copyBytesIfNecessary(),
+ indexMaintainer.getEmptyKeyValueQualifier(), ts, finalEmptyColumnValue);
+ indexUpdates.put(hTableInterfaceReference, indexPut);
+ if (!ignoreWritingDeleteColumnsToIndex) {
+ Delete deleteColumn = indexMaintainer.buildDeleteColumnMutation(indexPut, ts);
+ if (deleteColumn != null) {
+ indexUpdates.put(hTableInterfaceReference, deleteColumn);
+ }
+ }
+ // Delete the current index row if the new index key is different from the
+ // current one and the index is not a CDC index
+ if (currentDataRowState != null) {
+ ValueGetter currentDataRowVG = new IndexUtil.SimpleValueGetter(currentDataRowState);
+ byte[] indexRowKeyForCurrentDataRow = indexMaintainer.buildRowKey(currentDataRowVG,
+ rowKeyPtr, null, null, ts, encodedRegionName);
+ if (
+ !indexMaintainer.isCDCIndex()
+ && Bytes.compareTo(indexPut.getRow(), indexRowKeyForCurrentDataRow) != 0
+ ) {
+ Mutation del = indexMaintainer.buildRowDeleteMutation(indexRowKeyForCurrentDataRow,
+ IndexMaintainer.DeleteType.ALL_VERSIONS, ts);
+ indexUpdates.put(hTableInterfaceReference, del);
+ }
+ }
+ } else if (
+ currentDataRowState != null
+ && indexMaintainer.shouldPrepareIndexMutations(currentDataRowState)
+ ) {
+ if (indexMaintainer.isCDCIndex()) {
+ // CDC Index needs two a delete marker for referencing the data table
+ // delete mutation with the right index row key, that is, the index row key
+ // starting with ts
+ Put cdcDataRowState = new Put(currentDataRowState.getRow());
+ cdcDataRowState.addColumn(indexMaintainer.getDataEmptyKeyValueCF(),
+ indexMaintainer.getEmptyKeyValueQualifierForDataTable(), ts, ByteUtil.EMPTY_BYTE_ARRAY);
+ indexUpdates.put(hTableInterfaceReference, getDeleteIndexMutation(cdcDataRowState,
+ indexMaintainer, ts, rowKeyPtr, encodedRegionName));
+ } else {
+ indexUpdates.put(hTableInterfaceReference, getDeleteIndexMutation(currentDataRowState,
+ indexMaintainer, ts, rowKeyPtr, encodedRegionName));
+ }
+ }
+ }
+ }
+
/**
* Generate the index update for a data row from the mutation that are obtained by merging the
* previous data row state with the pending row mutation.
@@ -1181,6 +1344,12 @@ private void prepareIndexMutations(BatchMutateContext context, List(indexMaintainer, hTableInterfaceReference));
@@ -1193,73 +1362,12 @@ private void prepareIndexMutations(BatchMutateContext context, List pair : indexTables) {
- IndexMaintainer indexMaintainer = pair.getFirst();
- HTableInterfaceReference hTableInterfaceReference = pair.getSecond();
- if (
- nextDataRowState != null && indexMaintainer.shouldPrepareIndexMutations(nextDataRowState)
- ) {
- ValueGetter nextDataRowVG = new IndexUtil.SimpleValueGetter(nextDataRowState);
- Put indexPut = indexMaintainer.buildUpdateMutation(GenericKeyValueBuilder.INSTANCE,
- nextDataRowVG, rowKeyPtr, ts, null, null, false, encodedRegionName);
- if (indexPut == null) {
- // No covered column. Just prepare an index row with the empty column
- byte[] indexRowKey = indexMaintainer.buildRowKey(nextDataRowVG, rowKeyPtr, null, null,
- ts, encodedRegionName);
- indexPut = new Put(indexRowKey);
- } else {
- IndexUtil.removeEmptyColumn(indexPut,
- indexMaintainer.getEmptyKeyValueFamily().copyBytesIfNecessary(),
- indexMaintainer.getEmptyKeyValueQualifier());
- }
- indexPut.addColumn(indexMaintainer.getEmptyKeyValueFamily().copyBytesIfNecessary(),
- indexMaintainer.getEmptyKeyValueQualifier(), ts, QueryConstants.UNVERIFIED_BYTES);
- context.indexUpdates.put(hTableInterfaceReference,
- new Pair(indexPut, rowKeyPtr.get()));
- if (!ignoreWritingDeleteColumnsToIndex) {
- Delete deleteColumn = indexMaintainer.buildDeleteColumnMutation(indexPut, ts);
- if (deleteColumn != null) {
- context.indexUpdates.put(hTableInterfaceReference,
- new Pair(deleteColumn, rowKeyPtr.get()));
- }
- }
- // Delete the current index row if the new index key is different from the
- // current one and the index is not a CDC index
- if (currentDataRowState != null) {
- ValueGetter currentDataRowVG = new IndexUtil.SimpleValueGetter(currentDataRowState);
- byte[] indexRowKeyForCurrentDataRow = indexMaintainer.buildRowKey(currentDataRowVG,
- rowKeyPtr, null, null, ts, encodedRegionName);
- if (
- !indexMaintainer.isCDCIndex()
- && Bytes.compareTo(indexPut.getRow(), indexRowKeyForCurrentDataRow) != 0
- ) {
- Mutation del = indexMaintainer.buildRowDeleteMutation(indexRowKeyForCurrentDataRow,
- IndexMaintainer.DeleteType.ALL_VERSIONS, ts);
- context.indexUpdates.put(hTableInterfaceReference,
- new Pair(del, rowKeyPtr.get()));
- }
- }
- } else if (
- currentDataRowState != null
- && indexMaintainer.shouldPrepareIndexMutations(currentDataRowState)
- ) {
- if (indexMaintainer.isCDCIndex()) {
- // CDC Index needs two a delete marker for referencing the data table
- // delete mutation with the right index row key, that is, the index row key
- // starting with ts
- Put cdcDataRowState = new Put(currentDataRowState.getRow());
- cdcDataRowState.addColumn(indexMaintainer.getDataEmptyKeyValueCF(),
- indexMaintainer.getEmptyKeyValueQualifierForDataTable(), ts,
- ByteUtil.EMPTY_BYTE_ARRAY);
- context.indexUpdates.put(hTableInterfaceReference,
- new Pair(getDeleteIndexMutation(cdcDataRowState, indexMaintainer,
- ts, rowKeyPtr, encodedRegionName), rowKeyPtr.get()));
- } else {
- context.indexUpdates.put(hTableInterfaceReference,
- new Pair(getDeleteIndexMutation(currentDataRowState,
- indexMaintainer, ts, rowKeyPtr, encodedRegionName), rowKeyPtr.get()));
- }
- }
+ ListMultimap idxUpdates = ArrayListMultimap.create();
+ generateIndexMutationsForRow(rowKeyPtr, currentDataRowState, nextDataRowState, ts,
+ encodedRegionName, QueryConstants.UNVERIFIED_BYTES, indexTables, idxUpdates);
+ for (Map.Entry idxUpdate : idxUpdates.entries()) {
+ context.indexUpdates.put(idxUpdate.getKey(),
+ new Pair<>(idxUpdate.getValue(), rowKeyPtr.get()));
}
}
}
@@ -1287,9 +1395,20 @@ private void preparePreIndexMutations(BatchMutateContext context, long batchTime
ArrayListMultimap.> create();
prepareIndexMutations(context, maintainers, batchTimestamp);
+ if (serializeCDCMutations) {
+ prepareEventuallyConsistentIndexMutations(context, batchTimestamp, maintainers,
+ compressCDCMutations);
+ }
+
context.preIndexUpdates = ArrayListMultimap. create();
int updateCount = 0;
for (IndexMaintainer indexMaintainer : maintainers) {
+ if (
+ indexMaintainer.getIndexConsistency() != null
+ && indexMaintainer.getIndexConsistency().isAsynchronous()
+ ) {
+ continue;
+ }
updateCount++;
byte[] emptyCF = indexMaintainer.getEmptyKeyValueFamily().copyBytesIfNecessary();
byte[] emptyCQ = indexMaintainer.getEmptyKeyValueQualifier();
@@ -1299,6 +1418,15 @@ private void preparePreIndexMutations(BatchMutateContext context, long batchTime
for (Pair update : updates) {
Mutation m = update.getFirst();
if (m instanceof Put) {
+ if (indexMaintainer.isCDCIndex() && context.cdcPreMutationsBytes != null) {
+ ImmutableBytesPtr dataRowKeyPtr = new ImmutableBytesPtr(update.getSecond());
+ byte[] cdcMutationsBytes = context.cdcPreMutationsBytes.get(dataRowKeyPtr);
+ if (cdcMutationsBytes != null) {
+ ((Put) m).addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES,
+ QueryConstants.CDC_INDEX_PRE_MUTATIONS_CQ_BYTES, batchTimestamp,
+ cdcMutationsBytes);
+ }
+ }
// This will be done before the data table row is updated (i.e., in the first write
// phase)
context.preIndexUpdates.put(hTableInterfaceReference, m);
@@ -1309,6 +1437,15 @@ private void preparePreIndexMutations(BatchMutateContext context, long batchTime
Put unverifiedPut = new Put(m.getRow());
unverifiedPut.addColumn(emptyCF, emptyCQ, batchTimestamp,
QueryConstants.UNVERIFIED_BYTES);
+ if (indexMaintainer.isCDCIndex() && context.cdcPreMutationsBytes != null) {
+ ImmutableBytesPtr dataRowKeyPtr = new ImmutableBytesPtr(update.getSecond());
+ byte[] cdcMutationsBytes = context.cdcPreMutationsBytes.get(dataRowKeyPtr);
+ if (cdcMutationsBytes != null) {
+ unverifiedPut.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES,
+ QueryConstants.CDC_INDEX_PRE_MUTATIONS_CQ_BYTES, batchTimestamp,
+ cdcMutationsBytes);
+ }
+ }
// This will be done before the data table row is updated (i.e., in the first write
// phase)
context.preIndexUpdates.put(hTableInterfaceReference, unverifiedPut);
@@ -1319,6 +1456,113 @@ private void preparePreIndexMutations(BatchMutateContext context, long batchTime
}
}
+ /**
+ * Prepares pre-phase and post-phase cdc mutations for eventually consistent indexes.
+ * @param context batch mutate context.
+ * @param batchTimestamp the timestamp to use for mutations.
+ * @param maintainers the list of index maintainers.
+ * @param compressMutations whether to Snappy-compress the serialized proto bytes.
+ * @throws IOException if there is an error.
+ */
+ private static void prepareEventuallyConsistentIndexMutations(BatchMutateContext context,
+ long batchTimestamp, List maintainers, boolean compressMutations)
+ throws IOException {
+ // Store pre-index and post-index mutations for each data table rowkey
+ Map preBuilderMap =
+ new HashMap<>();
+ Map postBuilderMap =
+ new HashMap<>();
+
+ for (IndexMaintainer indexMaintainer : maintainers) {
+ if (
+ indexMaintainer.getIndexConsistency() == null
+ || !indexMaintainer.getIndexConsistency().isAsynchronous()
+ ) {
+ continue;
+ }
+ byte[] emptyCF = indexMaintainer.getEmptyKeyValueFamily().copyBytesIfNecessary();
+ byte[] emptyCQ = indexMaintainer.getEmptyKeyValueQualifier();
+ HTableInterfaceReference hTableInterfaceReference =
+ new HTableInterfaceReference(new ImmutableBytesPtr(indexMaintainer.getIndexTableName()));
+ List> updates = context.indexUpdates.get(hTableInterfaceReference);
+ for (Pair update : updates) {
+ Mutation m = update.getFirst();
+ byte[] dataRowKey = update.getSecond();
+ ImmutableBytesPtr rowKeyPtr = new ImmutableBytesPtr(dataRowKey);
+ IndexMutationsProtos.IndexMutations.Builder preBuilder = preBuilderMap
+ .computeIfAbsent(rowKeyPtr, k -> IndexMutationsProtos.IndexMutations.newBuilder());
+ IndexMutationsProtos.IndexMutations.Builder postBuilder = postBuilderMap
+ .computeIfAbsent(rowKeyPtr, k -> IndexMutationsProtos.IndexMutations.newBuilder());
+ if (m instanceof Put) {
+ preBuilder.addTables(ByteString.copyFrom(indexMaintainer.getIndexTableName()));
+ byte[] preMutation =
+ ProtobufUtil.toMutation(ClientProtos.MutationProto.MutationType.PUT, m).toByteArray();
+ preBuilder.addMutations(ByteString.copyFrom(preMutation));
+ if (!indexMaintainer.isUncovered()) {
+ Put verifiedPut = new Put(m.getRow());
+ verifiedPut.addColumn(emptyCF, emptyCQ, batchTimestamp, QueryConstants.VERIFIED_BYTES);
+ postBuilder.addTables(ByteString.copyFrom(indexMaintainer.getIndexTableName()));
+ byte[] postMutation = ProtobufUtil
+ .toMutation(ClientProtos.MutationProto.MutationType.PUT, verifiedPut).toByteArray();
+ postBuilder.addMutations(ByteString.copyFrom(postMutation));
+ }
+ } else {
+ if (IndexUtil.isDeleteFamily(m)) {
+ Put unverifiedPut = new Put(m.getRow());
+ unverifiedPut.addColumn(emptyCF, emptyCQ, batchTimestamp,
+ QueryConstants.UNVERIFIED_BYTES);
+ preBuilder.addTables(ByteString.copyFrom(indexMaintainer.getIndexTableName()));
+ byte[] preMutation = ProtobufUtil
+ .toMutation(ClientProtos.MutationProto.MutationType.PUT, unverifiedPut).toByteArray();
+ preBuilder.addMutations(ByteString.copyFrom(preMutation));
+ }
+ postBuilder.addTables(ByteString.copyFrom(indexMaintainer.getIndexTableName()));
+ byte[] deleteMutation = ProtobufUtil
+ .toMutation(ClientProtos.MutationProto.MutationType.DELETE, m).toByteArray();
+ postBuilder.addMutations(ByteString.copyFrom(deleteMutation));
+ }
+ }
+ }
+
+ if (!preBuilderMap.isEmpty()) {
+ context.cdcPreMutationsBytes = new HashMap<>();
+ for (Map.Entry entry : preBuilderMap.entrySet()) {
+ ImmutableBytesPtr rowKey = entry.getKey();
+ IndexMutationsProtos.IndexMutations.Builder builder = entry.getValue();
+ if (builder.getTablesCount() != builder.getMutationsCount()) {
+ throw new DoNotRetryIOException(
+ "Pre-phase tables and mutations sizes do not match for row key. Tables size: "
+ + builder.getTablesCount() + " , mutations size: " + builder.getMutationsCount());
+ }
+ if (builder.getTablesCount() > 0) {
+ byte[] protoBytes = builder.build().toByteArray();
+ context.cdcPreMutationsBytes.put(rowKey,
+ compressMutations ? Snappy.compress(protoBytes) : protoBytes);
+ }
+ }
+ }
+
+ if (!postBuilderMap.isEmpty()) {
+ context.cdcPostMutationsBytes = new HashMap<>();
+ for (Map.Entry entry : postBuilderMap.entrySet()) {
+ ImmutableBytesPtr rowKey = entry.getKey();
+ IndexMutationsProtos.IndexMutations.Builder builder = entry.getValue();
+ if (builder.getTablesCount() != builder.getMutationsCount()) {
+ throw new DoNotRetryIOException(
+ "Post-phase tables and mutations sizes do not match for row key. Tables size: "
+ + builder.getTablesCount() + " , mutations size: " + builder.getMutationsCount());
+ }
+ if (builder.getTablesCount() > 0) {
+ byte[] protoBytes = builder.build().toByteArray();
+ context.cdcPostMutationsBytes.put(rowKey,
+ compressMutations ? Snappy.compress(protoBytes) : protoBytes);
+ }
+ }
+ }
+ }
+
protected PhoenixIndexMetaData getPhoenixIndexMetaData(
ObserverContext observerContext,
MiniBatchOperationInProgress miniBatchOp) throws IOException {
@@ -1337,6 +1581,12 @@ private void preparePostIndexMutations(BatchMutateContext context, long batchTim
context.postIndexUpdates = ArrayListMultimap. create();
List maintainers = indexMetaData.getIndexMaintainers();
for (IndexMaintainer indexMaintainer : maintainers) {
+ if (
+ indexMaintainer.getIndexConsistency() != null
+ && indexMaintainer.getIndexConsistency().isAsynchronous()
+ ) {
+ continue;
+ }
byte[] emptyCF = indexMaintainer.getEmptyKeyValueFamily().copyBytesIfNecessary();
byte[] emptyCQ = indexMaintainer.getEmptyKeyValueQualifier();
HTableInterfaceReference hTableInterfaceReference =
@@ -1356,6 +1606,31 @@ private void preparePostIndexMutations(BatchMutateContext context, long batchTim
}
}
}
+
+ if (context.cdcPostMutationsBytes != null && !context.cdcPostMutationsBytes.isEmpty()) {
+ for (IndexMaintainer indexMaintainer : maintainers) {
+ if (!indexMaintainer.isCDCIndex()) {
+ continue;
+ }
+ HTableInterfaceReference hTableInterfaceReference =
+ new HTableInterfaceReference(new ImmutableBytesPtr(indexMaintainer.getIndexTableName()));
+ List> updates = context.indexUpdates.get(hTableInterfaceReference);
+ for (Pair update : updates) {
+ Mutation m = update.getFirst();
+ if (m instanceof Put) {
+ ImmutableBytesPtr dataRowKeyPtr = new ImmutableBytesPtr(update.getSecond());
+ byte[] cdcMutationsBytes = context.cdcPostMutationsBytes.get(dataRowKeyPtr);
+ if (cdcMutationsBytes != null) {
+ Put postPut = new Put(m.getRow());
+ postPut.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES,
+ QueryConstants.CDC_INDEX_POST_MUTATIONS_CQ_BYTES, batchTimestamp,
+ cdcMutationsBytes);
+ context.postIndexUpdates.put(hTableInterfaceReference, postPut);
+ }
+ }
+ }
+ }
+ }
// all cleanup will be done in postBatchMutateIndispensably()
}
@@ -1445,7 +1720,7 @@ private void waitForPreviousConcurrentBatch(TableName table, BatchMutateContext
CountDownLatch countDownLatch = lastContext.getCountDownLatch();
if (countDownLatch == null) {
// phase changed from PRE to either FAILED or POST
- if (phase == BatchMutatePhase.FAILED) {
+ if (lastContext.getCurrentPhase() == BatchMutatePhase.FAILED) {
context.currentPhase = BatchMutatePhase.FAILED;
break;
}
diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/hbase/index/write/TrackingParallelWriterIndexCommitter.java b/phoenix-core-server/src/main/java/org/apache/phoenix/hbase/index/write/TrackingParallelWriterIndexCommitter.java
index 69d782e078d..fcf839b7d33 100644
--- a/phoenix-core-server/src/main/java/org/apache/phoenix/hbase/index/write/TrackingParallelWriterIndexCommitter.java
+++ b/phoenix-core-server/src/main/java/org/apache/phoenix/hbase/index/write/TrackingParallelWriterIndexCommitter.java
@@ -151,11 +151,13 @@ public void write(Multimap toWrite,
Set>> entries = toWrite.asMap().entrySet();
TaskBatch tasks = new TaskBatch(entries.size());
List tables = new ArrayList(entries.size());
+ int totalMutations = 0;
for (Entry> entry : entries) {
// get the mutations for each table. We leak the implementation here a little bit to save
// doing a complete copy over of all the index update for each table.
final List mutations =
kvBuilder.cloneIfNecessary((List) entry.getValue());
+ totalMutations += mutations.size();
// track each reference so we can get at it easily later, when determing failures
final HTableInterfaceReference tableReference = entry.getKey();
final RegionCoprocessorEnvironment env = this.env;
@@ -249,7 +251,10 @@ private void throwFailureIfDone() throws SingleIndexWriteFailureException {
Pair, List>> resultsAndFutures = null;
try {
- LOGGER.debug("Waiting on index update tasks to complete...");
+ LOGGER.debug(
+ "Waiting on index update tasks to complete... "
+ + "tasks size {}, total mutations {}, num tables {}",
+ tasks.size(), totalMutations, tables.size());
resultsAndFutures = this.pool.submitUninterruptible(tasks);
} catch (ExecutionException e) {
throw new RuntimeException(
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java
index e0d3bc1b35c..ec5992b50aa 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java
@@ -105,7 +105,7 @@ public abstract class BasePermissionsIT extends BaseTest {
private static final Set PHOENIX_SYSTEM_TABLES =
new HashSet<>(Arrays.asList("SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS",
"SYSTEM.FUNCTION", "SYSTEM.MUTEX", "SYSTEM.CHILD_LINK", "SYSTEM.TRANSFORM",
- "SYSTEM.CDC_STREAM_STATUS", "SYSTEM.CDC_STREAM"));
+ "SYSTEM.CDC_STREAM_STATUS", "SYSTEM.CDC_STREAM", "SYSTEM.IDX_CDC_TRACKER"));
private static final Set PHOENIX_SYSTEM_TABLES_IDENTIFIERS =
new HashSet<>(Arrays.asList("SYSTEM.\"CATALOG\"", "SYSTEM.\"SEQUENCE\"", "SYSTEM.\"STATS\"",
@@ -123,7 +123,7 @@ public abstract class BasePermissionsIT extends BaseTest {
static final Set PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES =
new HashSet<>(Arrays.asList("SYSTEM:CATALOG", "SYSTEM:SEQUENCE", "SYSTEM:STATS",
"SYSTEM:FUNCTION", "SYSTEM:MUTEX", "SYSTEM:CHILD_LINK", "SYSTEM:TRANSFORM",
- "SYSTEM:CDC_STREAM_STATUS", "SYSTEM:CDC_STREAM"));
+ "SYSTEM:CDC_STREAM_STATUS", "SYSTEM:CDC_STREAM", "SYSTEM:IDX_CDC_TRACKER"));
// Create Multiple users so that we can use Hadoop UGI to run tasks as various users
// Permissions can be granted or revoke by superusers and admins only
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/Bson4IT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/Bson4IT.java
index 103f0e6df5e..f35cc50f5c2 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/Bson4IT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/Bson4IT.java
@@ -37,6 +37,7 @@
import java.sql.Statement;
import java.util.Arrays;
import java.util.Collection;
+import java.util.List;
import java.util.Properties;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.hbase.util.Bytes;
@@ -47,9 +48,13 @@
import org.apache.phoenix.hbase.index.metrics.GlobalIndexCheckerSource;
import org.apache.phoenix.hbase.index.metrics.GlobalIndexCheckerSourceImpl;
import org.apache.phoenix.hbase.index.metrics.MetricsIndexerSourceFactory;
+import org.apache.phoenix.jdbc.PhoenixConnection;
import org.apache.phoenix.jdbc.PhoenixPreparedStatement;
import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableKey;
import org.apache.phoenix.schema.types.PDouble;
+import org.apache.phoenix.util.CDCUtil;
import org.apache.phoenix.util.PropertiesUtil;
import org.bson.BsonArray;
import org.bson.BsonBinary;
@@ -145,6 +150,12 @@ public void testBsonValueFunction() throws Exception {
conn.commit();
+ PhoenixConnection pcon = conn.unwrap(PhoenixConnection.class);
+ PTable dataTable = pcon.getTable(new PTableKey(pcon.getTenantId(), tableName));
+ List indexes = dataTable.getIndexes();
+ assertEquals("Data table should have exactly 2 indexes", 2, indexes.size());
+ assertFalse("Data table should not have CDC index", CDCUtil.hasCDCIndex(dataTable));
+
ResultSet rs = conn.createStatement().executeQuery("SELECT count(*) FROM " + tableName);
assertTrue(rs.next());
assertEquals(3, rs.getInt(1));
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/Bson5IT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/Bson5IT.java
index 99f4642baf6..3e6a4f19891 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/Bson5IT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/Bson5IT.java
@@ -17,6 +17,9 @@
*/
package org.apache.phoenix.end2end;
+import static org.apache.phoenix.hbase.index.IndexCDCConsumer.INDEX_CDC_CONSUMER_RETRY_PAUSE_MS;
+import static org.apache.phoenix.hbase.index.IndexCDCConsumer.INDEX_CDC_CONSUMER_TIMESTAMP_BUFFER_MS;
+import static org.apache.phoenix.hbase.index.IndexRegionObserver.PHOENIX_INDEX_CDC_MUTATION_SERIALIZE;
import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
@@ -40,10 +43,15 @@
import java.util.Properties;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.compile.ExplainPlan;
+import org.apache.phoenix.compile.ExplainPlanAttributes;
+import org.apache.phoenix.coprocessorclient.BaseScannerRegionObserverConstants;
+import org.apache.phoenix.jdbc.PhoenixPreparedStatement;
import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
import org.apache.phoenix.schema.types.PDouble;
-import org.apache.phoenix.util.CDCUtil;
import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
import org.bson.BsonArray;
import org.bson.BsonBinary;
import org.bson.BsonDocument;
@@ -52,19 +60,33 @@
import org.bson.BsonString;
import org.bson.RawBsonDocument;
import org.junit.Assert;
+import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.apache.phoenix.thirdparty.com.google.common.base.Preconditions;
+import org.apache.phoenix.thirdparty.com.google.common.collect.Maps;
/**
* Tests for BSON with expression field key alias.
*/
-@Category(ParallelStatsDisabledTest.class)
+@Category(NeedsOwnMiniClusterTest.class)
public class Bson5IT extends ParallelStatsDisabledIT {
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+ @BeforeClass
+ public static synchronized void doSetup() throws Exception {
+ Map props = Maps.newHashMapWithExpectedSize(1);
+ props.put(BaseScannerRegionObserverConstants.PHOENIX_MAX_LOOKBACK_AGE_CONF_KEY,
+ Integer.toString(60 * 60));
+ props.put(QueryServices.USE_STATS_FOR_PARALLELIZATION, Boolean.toString(false));
+ props.put(INDEX_CDC_CONSUMER_TIMESTAMP_BUFFER_MS, Integer.toString(2000));
+ props.put(INDEX_CDC_CONSUMER_RETRY_PAUSE_MS, Integer.toString(5));
+ props.put(PHOENIX_INDEX_CDC_MUTATION_SERIALIZE, Boolean.TRUE.toString());
+ setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+ }
+
private static String getJsonString(String jsonFilePath) throws IOException {
URL fileUrl = Bson5IT.class.getClassLoader().getResource(jsonFilePath);
Preconditions.checkArgument(fileUrl != null, "File path " + jsonFilePath + " seems invalid");
@@ -79,14 +101,24 @@ public void testBsonOpsWithSqlConditionsUpdateSuccess() throws Exception {
Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
String tableName = generateUniqueName();
String cdcName = generateUniqueName();
+ String indexName1 = "IDX1_" + tableName;
+ String indexName2 = "IDX2_" + tableName;
try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
String ddl = "CREATE TABLE " + tableName + " (PK1 VARCHAR NOT NULL, C1 VARCHAR, COL BSON"
+ " CONSTRAINT pk PRIMARY KEY(PK1))";
String cdcDdl = "CREATE CDC " + cdcName + " ON " + tableName;
+ String indexDdl1 = "CREATE UNCOVERED INDEX " + indexName1 + " ON " + tableName
+ + "(BSON_VALUE(COL, 'rather[3].outline.clock', 'VARCHAR')) "
+ + "WHERE BSON_VALUE(COL, 'rather[3].outline.clock', 'VARCHAR') IS NOT NULL "
+ + "CONSISTENCY = EVENTUAL";
+ String indexDdl2 = "CREATE UNCOVERED INDEX " + indexName2 + " ON " + tableName
+ + "(BSON_VALUE(COL, 'result[1].location.coordinates.longitude', 'DOUBLE')) "
+ + "WHERE BSON_VALUE(COL, 'result[1].location.coordinates.longitude', 'DOUBLE') IS NOT NULL "
+ + "CONSISTENCY = EVENTUAL";
conn.createStatement().execute(ddl);
conn.createStatement().execute(cdcDdl);
- IndexToolIT.runIndexTool(false, "", tableName,
- "\"" + CDCUtil.getCDCIndexName(cdcName) + "\"");
+ conn.createStatement().execute(indexDdl1);
+ conn.createStatement().execute(indexDdl2);
Timestamp ts1 = new Timestamp(System.currentTimeMillis());
Thread.sleep(100);
@@ -118,6 +150,30 @@ public void testBsonOpsWithSqlConditionsUpdateSuccess() throws Exception {
Thread.sleep(100);
Timestamp ts2 = new Timestamp(System.currentTimeMillis());
+ PreparedStatement indexPs = conn.prepareStatement("SELECT PK1, COL FROM " + tableName
+ + " WHERE BSON_VALUE(COL, 'rather[3].outline.clock', 'VARCHAR') = ?");
+ indexPs.setString(1, "personal");
+ ResultSet indexRs = indexPs.executeQuery();
+ assertFalse(indexRs.next());
+
+ Thread.sleep(11000);
+ indexRs = indexPs.executeQuery();
+ assertTrue(indexRs.next());
+ assertEquals("pk1010", indexRs.getString(1));
+ BsonDocument actualDoc = (BsonDocument) indexRs.getObject(2);
+ assertEquals(bsonDocument2, actualDoc);
+ assertFalse(indexRs.next());
+ validateExplainPlan(indexPs, indexName1, "RANGE SCAN ");
+
+ indexPs = conn.prepareStatement("SELECT PK1 FROM " + tableName
+ + " WHERE BSON_VALUE(COL, 'result[1].location.coordinates.longitude', 'DOUBLE') = ?");
+ indexPs.setDouble(1, 52.3736);
+ indexRs = indexPs.executeQuery();
+ assertTrue(indexRs.next());
+ assertEquals("pk1011", indexRs.getString(1));
+ assertFalse(indexRs.next());
+ validateExplainPlan(indexPs, indexName2, "RANGE SCAN ");
+
testCDCAfterFirstUpsert(conn, cdcName, ts1, ts2, bsonDocument1, bsonDocument2, bsonDocument3);
ts1 = new Timestamp(System.currentTimeMillis());
@@ -312,14 +368,24 @@ public void testBsonOpsWithSqlConditionsUpdateFailure() throws Exception {
Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
String tableName = generateUniqueName();
String cdcName = generateUniqueName();
+ String indexName1 = "IDX1_" + tableName;
+ String indexName2 = "IDX2_" + tableName;
try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
String ddl = "CREATE TABLE " + tableName + " (PK1 VARCHAR NOT NULL, C1 VARCHAR, COL BSON"
+ " CONSTRAINT pk PRIMARY KEY(PK1))";
String cdcDdl = "CREATE CDC " + cdcName + " ON " + tableName;
+ String indexDdl1 = "CREATE INDEX " + indexName1 + " ON " + tableName
+ + "(BSON_VALUE(COL, 'rather[3].outline.clock', 'VARCHAR')) INCLUDE(COL) "
+ + "WHERE BSON_VALUE(COL, 'rather[3].outline.clock', 'VARCHAR') IS NOT NULL "
+ + "CONSISTENCY = EVENTUAL";
+ String indexDdl2 = "CREATE UNCOVERED INDEX " + indexName2 + " ON " + tableName
+ + "(BSON_VALUE(COL, 'result[1].location.coordinates.longitude', 'DOUBLE')) "
+ + "WHERE BSON_VALUE(COL, 'result[1].location.coordinates.longitude', 'DOUBLE') IS NOT NULL "
+ + "CONSISTENCY = EVENTUAL";
conn.createStatement().execute(ddl);
conn.createStatement().execute(cdcDdl);
- IndexToolIT.runIndexTool(false, "", tableName,
- "\"" + CDCUtil.getCDCIndexName(cdcName) + "\"");
+ conn.createStatement().execute(indexDdl1);
+ conn.createStatement().execute(indexDdl2);
Timestamp ts1 = new Timestamp(System.currentTimeMillis());
Thread.sleep(100);
@@ -352,6 +418,27 @@ public void testBsonOpsWithSqlConditionsUpdateFailure() throws Exception {
Thread.sleep(100);
Timestamp ts2 = new Timestamp(System.currentTimeMillis());
+ Thread.sleep(11000);
+ PreparedStatement indexPs = conn.prepareStatement("SELECT PK1, COL FROM " + tableName
+ + " WHERE BSON_VALUE(COL, 'rather[3].outline.clock', 'VARCHAR') = ?");
+ indexPs.setString(1, "personal");
+ ResultSet indexRs = indexPs.executeQuery();
+ assertTrue(indexRs.next());
+ assertEquals("pk1010", indexRs.getString(1));
+ BsonDocument actualDoc = (BsonDocument) indexRs.getObject(2);
+ assertEquals(bsonDocument2, actualDoc);
+ assertFalse(indexRs.next());
+ validateExplainPlan(indexPs, indexName1, "RANGE SCAN ");
+
+ indexPs = conn.prepareStatement("SELECT PK1 FROM " + tableName
+ + " WHERE BSON_VALUE(COL, 'result[1].location.coordinates.longitude', 'DOUBLE') = ?");
+ indexPs.setDouble(1, 52.3736);
+ indexRs = indexPs.executeQuery();
+ assertTrue(indexRs.next());
+ assertEquals("pk1011", indexRs.getString(1));
+ assertFalse(indexRs.next());
+ validateExplainPlan(indexPs, indexName2, "RANGE SCAN ");
+
testCDCAfterFirstUpsert(conn, cdcName, ts1, ts2, bsonDocument1, bsonDocument2, bsonDocument3);
Thread.sleep(100);
@@ -654,4 +741,13 @@ private static void testCDCUpdateOneRowChange(Connection conn, String cdcName, T
}
}
+ private static void validateExplainPlan(PreparedStatement ps, String tableName, String scanType)
+ throws SQLException {
+ ExplainPlan plan = ps.unwrap(PhoenixPreparedStatement.class).optimizeQuery().getExplainPlan();
+ ExplainPlanAttributes explainPlanAttributes = plan.getPlanStepsAsAttributes();
+ assertEquals(tableName, explainPlanAttributes.getTableName());
+ assertEquals("PARALLEL 1-WAY", explainPlanAttributes.getIteratorTypeAndScanSize());
+ assertEquals(scanType, explainPlanAttributes.getExplainScanType());
+ }
+
}
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConcurrentMutationsCoveredEventualGenerateIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConcurrentMutationsCoveredEventualGenerateIT.java
new file mode 100644
index 00000000000..0088a38a645
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConcurrentMutationsCoveredEventualGenerateIT.java
@@ -0,0 +1,73 @@
+/*
+ * 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.apache.phoenix.hbase.index.IndexCDCConsumer.INDEX_CDC_CONSUMER_BATCH_SIZE;
+import static org.apache.phoenix.hbase.index.IndexCDCConsumer.INDEX_CDC_CONSUMER_PARENT_PROGRESS_PAUSE_MS;
+import static org.apache.phoenix.hbase.index.IndexCDCConsumer.INDEX_CDC_CONSUMER_RETRY_PAUSE_MS;
+import static org.apache.phoenix.hbase.index.IndexCDCConsumer.INDEX_CDC_CONSUMER_TIMESTAMP_BUFFER_MS;
+import static org.apache.phoenix.hbase.index.IndexRegionObserver.PHOENIX_INDEX_CDC_MUTATION_SERIALIZE;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Map;
+import org.apache.phoenix.coprocessor.PhoenixMasterObserver;
+import org.apache.phoenix.coprocessorclient.BaseScannerRegionObserverConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+import org.junit.runners.Parameterized;
+
+import org.apache.phoenix.thirdparty.com.google.common.collect.Maps;
+
+@Category(NeedsOwnMiniClusterTest.class)
+public class ConcurrentMutationsCoveredEventualGenerateIT
+ extends ConcurrentMutationsExtendedIndexIT {
+
+ private static final int MAX_LOOKBACK_AGE = 1000000;
+
+ public ConcurrentMutationsCoveredEventualGenerateIT(boolean uncovered, boolean eventual) {
+ super(uncovered, eventual);
+ }
+
+ @BeforeClass
+ public static synchronized void doSetup() throws Exception {
+ Map props = Maps.newHashMapWithExpectedSize(10);
+ props.put(QueryServices.GLOBAL_INDEX_ROW_AGE_THRESHOLD_TO_DELETE_MS_ATTRIB, Long.toString(0));
+ props.put(BaseScannerRegionObserverConstants.PHOENIX_MAX_LOOKBACK_AGE_CONF_KEY,
+ Integer.toString(MAX_LOOKBACK_AGE));
+ props.put("hbase.rowlock.wait.duration", "100");
+ props.put("phoenix.index.concurrent.wait.duration.ms", "10");
+ props.put(QueryServices.TASK_HANDLING_INTERVAL_MS_ATTRIB, Long.toString(2));
+ props.put(QueryServices.TASK_HANDLING_INITIAL_DELAY_MS_ATTRIB, Long.toString(1));
+ props.put(INDEX_CDC_CONSUMER_BATCH_SIZE, Integer.toString(4500));
+ props.put(INDEX_CDC_CONSUMER_TIMESTAMP_BUFFER_MS, Integer.toString(5000));
+ props.put(INDEX_CDC_CONSUMER_RETRY_PAUSE_MS, Integer.toString(200));
+ props.put(QueryServices.PHOENIX_SERVER_PAGE_SIZE_MS, Integer.toString(-1));
+ props.put("hbase.coprocessor.master.classes", PhoenixMasterObserver.class.getName());
+ props.put(PHOENIX_INDEX_CDC_MUTATION_SERIALIZE, Boolean.FALSE.toString());
+ props.put(INDEX_CDC_CONSUMER_PARENT_PROGRESS_PAUSE_MS, Integer.toString(1000));
+ setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+ }
+
+ @Parameterized.Parameters(name = "uncovered={0}, eventual={1}")
+ public static synchronized Collection