Skip to content

Commit a51a57a

Browse files
palashcPalash Chauhan
and
Palash Chauhan
authored
PHOENIX-7460 : Update CDC Stream metadata when a data table region splits (#2051)
Co-authored-by: Palash Chauhan <[email protected]>
1 parent 91cae25 commit a51a57a

File tree

3 files changed

+467
-9
lines changed

3 files changed

+467
-9
lines changed
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,209 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
package org.apache.phoenix.coprocessor;
19+
20+
import java.sql.Connection;
21+
import java.sql.PreparedStatement;
22+
import java.sql.ResultSet;
23+
import java.sql.SQLException;
24+
import java.sql.Types;
25+
import java.util.Optional;
26+
27+
import org.apache.hadoop.conf.Configuration;
28+
import org.apache.hadoop.hbase.TableName;
29+
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
30+
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
31+
import org.apache.hadoop.hbase.coprocessor.MasterObserver;
32+
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
33+
import org.apache.hadoop.hbase.client.RegionInfo;
34+
import org.apache.hadoop.hbase.util.Bytes;
35+
import org.apache.phoenix.schema.PTable;
36+
import org.apache.phoenix.schema.TableNotFoundException;
37+
import org.apache.phoenix.util.CDCUtil;
38+
import org.apache.phoenix.util.PhoenixRuntime;
39+
import org.apache.phoenix.util.QueryUtil;
40+
import org.slf4j.Logger;
41+
import org.slf4j.LoggerFactory;
42+
43+
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CDC_STREAM_NAME;
44+
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CDC_STREAM_STATUS_NAME;
45+
46+
/**
47+
* Master Coprocessor for Phoenix.
48+
*/
49+
public class PhoenixMasterObserver implements MasterObserver, MasterCoprocessor {
50+
private static final Logger LOGGER = LoggerFactory.getLogger(PhoenixMasterObserver.class);
51+
52+
private static final String STREAM_STATUS_QUERY
53+
= "SELECT STREAM_NAME FROM " + SYSTEM_CDC_STREAM_STATUS_NAME
54+
+ " WHERE TABLE_NAME = ? AND STREAM_STATUS='"
55+
+ CDCUtil.CdcStreamStatus.ENABLED.getSerializedValue() + "'";
56+
57+
// tableName, streamName, partitionId, parentId, startTime, endTime, startKey, endKey
58+
private static final String PARTITION_UPSERT_SQL
59+
= "UPSERT INTO " + SYSTEM_CDC_STREAM_NAME + " VALUES (?,?,?,?,?,?,?,?)";
60+
61+
private static final String PARENT_PARTITION_QUERY
62+
= "SELECT PARTITION_ID FROM " + SYSTEM_CDC_STREAM_NAME
63+
+ " WHERE TABLE_NAME = ? AND STREAM_NAME = ? ";
64+
65+
private static final String PARENT_PARTITION_UPDATE_END_TIME_SQL
66+
= "UPSERT INTO " + SYSTEM_CDC_STREAM_NAME + " (TABLE_NAME, STREAM_NAME, PARTITION_ID, "
67+
+ "PARTITION_END_TIME) VALUES (?,?,?,?)";
68+
69+
@Override
70+
public Optional<MasterObserver> getMasterObserver() {
71+
return Optional.of(this);
72+
}
73+
74+
/**
75+
* Update parent -> daughter relationship for CDC Streams.
76+
* - find parent partition id using start/end keys of daughters
77+
* - upsert partition metadata for the 2 daughters
78+
* - update the end time on the parent's partition metadata
79+
* @param c the environment to interact with the framework and master
80+
* @param regionInfoA the left daughter region
81+
* @param regionInfoB the right daughter region
82+
*/
83+
@Override
84+
public void postCompletedSplitRegionAction(final ObserverContext<MasterCoprocessorEnvironment> c,
85+
final RegionInfo regionInfoA,
86+
final RegionInfo regionInfoB) {
87+
Configuration conf = c.getEnvironment().getConfiguration();
88+
try {
89+
Connection conn = QueryUtil.getConnectionOnServer(conf);
90+
// CDC will be enabled on Phoenix tables only
91+
PTable phoenixTable = getPhoenixTable(conn, regionInfoA.getTable());
92+
if (phoenixTable == null) {
93+
LOGGER.info("{} is not a Phoenix Table, skipping partition metadata update.",
94+
regionInfoA.getTable());
95+
return;
96+
}
97+
// find streamName with ENABLED status
98+
String tableName = phoenixTable.getName().getString();
99+
PreparedStatement pstmt = conn.prepareStatement(STREAM_STATUS_QUERY);
100+
pstmt.setString(1, tableName);
101+
ResultSet rs = pstmt.executeQuery();
102+
if (rs.next()) {
103+
String streamName = rs.getString(1);
104+
LOGGER.info("Updating partition metadata for table={}, stream={} daughters {} {}",
105+
tableName, streamName, regionInfoA.getEncodedName(), regionInfoB.getEncodedName());
106+
String parentPartitionID = getParentPartitionId(conn, tableName, streamName, regionInfoA, regionInfoB);
107+
upsertDaughterPartition(conn, tableName, streamName, parentPartitionID, regionInfoA);
108+
upsertDaughterPartition(conn, tableName, streamName, parentPartitionID, regionInfoB);
109+
updateParentPartitionEndTime(conn, tableName, streamName, parentPartitionID, regionInfoA.getRegionId());
110+
}
111+
} catch (SQLException e) {
112+
LOGGER.error("Unable to update CDC Stream Partition metadata during split with daughter regions: {} {}",
113+
regionInfoA.getEncodedName(), regionInfoB.getEncodedName(), e);
114+
}
115+
}
116+
117+
private PTable getPhoenixTable(Connection conn, TableName tableName) throws SQLException {
118+
PTable pTable;
119+
try {
120+
pTable = PhoenixRuntime.getTable(conn, tableName.toString());
121+
} catch (TableNotFoundException e) {
122+
return null;
123+
}
124+
return pTable;
125+
}
126+
127+
/**
128+
* Lookup parent's partition id (region's encoded name) in SYSTEM.CDC_STREAM.
129+
* RegionInfoA is left daughter and RegionInfoB is right daughter so parent's key range would
130+
* be [RegionInfoA stratKey, RegionInfoB endKey]
131+
*/
132+
private String getParentPartitionId(Connection conn, String tableName, String streamName,
133+
RegionInfo regionInfoA, RegionInfo regionInfoB)
134+
throws SQLException {
135+
byte[] parentStartKey = regionInfoA.getStartKey();
136+
byte[] parentEndKey = regionInfoB.getEndKey();
137+
138+
StringBuilder qb = new StringBuilder(PARENT_PARTITION_QUERY);
139+
if (parentStartKey.length == 0) {
140+
qb.append(" AND PARTITION_START_KEY IS NULL ");
141+
} else {
142+
qb.append(" AND PARTITION_START_KEY = ? ");
143+
}
144+
if (parentEndKey.length == 0) {
145+
qb.append(" AND PARTITION_END_KEY IS NULL ");
146+
} else {
147+
qb.append(" AND PARTITION_END_KEY = ? ");
148+
}
149+
150+
PreparedStatement pstmt = conn.prepareStatement(qb.toString());
151+
int index = 1;
152+
pstmt.setString(index++, tableName);
153+
pstmt.setString(index++, streamName);
154+
if (parentStartKey.length > 0) pstmt.setBytes(index++, parentStartKey);
155+
if (parentEndKey.length > 0) pstmt.setBytes(index++, parentEndKey);
156+
LOGGER.info("Query to get parent partition id: " + pstmt);
157+
ResultSet rs = pstmt.executeQuery();
158+
if (rs.next()) {
159+
return rs.getString(1);
160+
} else {
161+
throw new SQLException(String.format("Could not find parent of the provided daughters: "
162+
+ "startKeyA=%s endKeyA=%s startKeyB=%s endKeyB=%s",
163+
Bytes.toStringBinary(regionInfoA.getStartKey()),
164+
Bytes.toStringBinary(regionInfoA.getEndKey()),
165+
Bytes.toStringBinary(regionInfoB.getStartKey()),
166+
Bytes.toStringBinary(regionInfoB.getEndKey())));
167+
}
168+
}
169+
170+
/**
171+
* Insert partition metadata for a daughter region from the split.
172+
*/
173+
private void upsertDaughterPartition(Connection conn, String tableName,
174+
String streamName, String parentPartitionID,
175+
RegionInfo regionInfo)
176+
throws SQLException {
177+
String partitionId = regionInfo.getEncodedName();
178+
long startTime = regionInfo.getRegionId();
179+
byte[] startKey = regionInfo.getStartKey();
180+
byte[] endKey = regionInfo.getEndKey();
181+
PreparedStatement pstmt = conn.prepareStatement(PARTITION_UPSERT_SQL);
182+
pstmt.setString(1, tableName);
183+
pstmt.setString(2, streamName);
184+
pstmt.setString(3, partitionId);
185+
pstmt.setString(4, parentPartitionID);
186+
pstmt.setLong(5, startTime);
187+
// endTime in not set when inserting a new partition
188+
pstmt.setNull(6, Types.BIGINT);
189+
pstmt.setBytes(7, startKey.length == 0 ? null : startKey);
190+
pstmt.setBytes(8, endKey.length == 0 ? null : endKey);
191+
pstmt.executeUpdate();
192+
conn.commit();
193+
}
194+
195+
/**
196+
* Update parent partition's endTime by setting it to daughter's startTime.
197+
*/
198+
private void updateParentPartitionEndTime(Connection conn, String tableName,
199+
String streamName, String parentPartitionID,
200+
long daughterStartTime) throws SQLException {
201+
PreparedStatement pstmt = conn.prepareStatement(PARENT_PARTITION_UPDATE_END_TIME_SQL);
202+
pstmt.setString(1, tableName);
203+
pstmt.setString(2, streamName);
204+
pstmt.setString(3, parentPartitionID);
205+
pstmt.setLong(4, daughterStartTime);
206+
pstmt.executeUpdate();
207+
conn.commit();
208+
}
209+
}

phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/tasks/CdcStreamPartitionMetadataTask.java

+9-6
Original file line numberDiff line numberDiff line change
@@ -37,6 +37,7 @@
3737
import java.sql.PreparedStatement;
3838
import java.sql.SQLException;
3939
import java.sql.Timestamp;
40+
import java.sql.Types;
4041
import java.util.List;
4142

4243
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CDC_STREAM_NAME;
@@ -55,9 +56,8 @@ public class CdcStreamPartitionMetadataTask extends BaseTask {
5556
private static final String CDC_STREAM_STATUS_UPSERT_SQL
5657
= "UPSERT INTO " + SYSTEM_CDC_STREAM_STATUS_NAME + " VALUES (?, ?, ?)";
5758

58-
// parent_partition_id will be null, set partition_end_time to -1
5959
private static final String CDC_STREAM_PARTITION_UPSERT_SQL
60-
= "UPSERT INTO " + SYSTEM_CDC_STREAM_NAME + " VALUES (?,?,?,null,?,-1,?,?)";
60+
= "UPSERT INTO " + SYSTEM_CDC_STREAM_NAME + " VALUES (?,?,?,?,?,?,?,?)";
6161

6262
@Override
6363
public TaskRegionObserver.TaskResult run(Task.TaskRecord taskRecord) {
@@ -136,13 +136,16 @@ private void upsertPartitionMetadata(PhoenixConnection pconn, String tableName,
136136
throws SQLException {
137137
try (PreparedStatement ps = pconn.prepareStatement(CDC_STREAM_PARTITION_UPSERT_SQL)) {
138138
for (HRegionLocation tableRegion : tableRegions) {
139-
RegionInfo ri = tableRegion.getRegionInfo();
139+
// set parent_partition_id, partition_end_time to null
140+
RegionInfo ri = tableRegion.getRegion();
140141
ps.setString(1, tableName);
141142
ps.setString(2, streamName);
142143
ps.setString(3, ri.getEncodedName());
143-
ps.setLong(4, ri.getRegionId());
144-
ps.setBytes(5, ri.getStartKey());
145-
ps.setBytes(6, ri.getEndKey());
144+
ps.setNull(4, Types.VARCHAR);
145+
ps.setLong(5, ri.getRegionId());
146+
ps.setNull(6, Types.BIGINT);
147+
ps.setBytes(7, ri.getStartKey());
148+
ps.setBytes(8, ri.getEndKey());
146149
ps.executeUpdate();
147150
}
148151
pconn.commit();

0 commit comments

Comments
 (0)