[
https://issues.apache.org/jira/browse/PHOENIX-7006?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17827165#comment-17827165
]
ASF GitHub Bot commented on PHOENIX-7006:
-----------------------------------------
haridsv commented on code in PR #1751:
URL: https://github.com/apache/phoenix/pull/1751#discussion_r1510336887
##########
phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java:
##########
@@ -1587,6 +1623,56 @@ private PTable getTableFromCells(List<Cell>
tableCellList, List<List<Cell>> allC
return builder.build();
}
+ private Long scanMaxLookbackAgeFromParent(byte[] key, long
clientTimeStamp) throws IOException {
+ Scan scan = MetaDataUtil.newTableRowsScan(key, MIN_TABLE_TIMESTAMP,
clientTimeStamp);
+ Table sysCat = ServerUtil.getHTableForCoprocessorScan(this.env,
+ SchemaUtil.getPhysicalTableName(SYSTEM_CATALOG_NAME_BYTES,
env.getConfiguration()));
+ ResultScanner scanner = sysCat.getScanner(scan);
+ Result result = scanner.next();
+ boolean startCheckingForLink = false;
+ byte[] parentTableKey = null;
+ do {
+ if (result == null) {
+ return null;
+ }
+ else if (startCheckingForLink) {
+ byte[] linkTypeBytes = result.getValue(TABLE_FAMILY_BYTES,
LINK_TYPE_BYTES);
+ if (linkTypeBytes != null) {
+ LinkType linkType =
LinkType.fromSerializedValue(linkTypeBytes[0]);
+ int rowKeyColMetadataLength = 5;
+ byte[][] rowKeyMetaData = new
byte[rowKeyColMetadataLength][];
+ getVarChars(result.getRow(), rowKeyColMetadataLength,
rowKeyMetaData);
Review Comment:
It is not in general a good practice to do static imports of functions.
##########
phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java:
##########
@@ -1587,6 +1623,56 @@ private PTable getTableFromCells(List<Cell>
tableCellList, List<List<Cell>> allC
return builder.build();
}
+ private Long scanMaxLookbackAgeFromParent(byte[] key, long
clientTimeStamp) throws IOException {
+ Scan scan = MetaDataUtil.newTableRowsScan(key, MIN_TABLE_TIMESTAMP,
clientTimeStamp);
+ Table sysCat = ServerUtil.getHTableForCoprocessorScan(this.env,
+ SchemaUtil.getPhysicalTableName(SYSTEM_CATALOG_NAME_BYTES,
env.getConfiguration()));
+ ResultScanner scanner = sysCat.getScanner(scan);
+ Result result = scanner.next();
+ boolean startCheckingForLink = false;
+ byte[] parentTableKey = null;
+ do {
+ if (result == null) {
+ return null;
+ }
+ else if (startCheckingForLink) {
+ byte[] linkTypeBytes = result.getValue(TABLE_FAMILY_BYTES,
LINK_TYPE_BYTES);
+ if (linkTypeBytes != null) {
+ LinkType linkType =
LinkType.fromSerializedValue(linkTypeBytes[0]);
+ int rowKeyColMetadataLength = 5;
+ byte[][] rowKeyMetaData = new
byte[rowKeyColMetadataLength][];
+ getVarChars(result.getRow(), rowKeyColMetadataLength,
rowKeyMetaData);
+ if (linkType == VIEW_INDEX_PARENT_TABLE) {
+ parentTableKey =
getParentTableKeyFromChildRowKeyMetaData(rowKeyMetaData);
+ return scanMaxLookbackAgeFromParent(parentTableKey,
clientTimeStamp);
Review Comment:
Why not `break` here and let the recursive call at line 1663 do the job?
> Configure maxLookbackAge at table level
> ---------------------------------------
>
> Key: PHOENIX-7006
> URL: https://issues.apache.org/jira/browse/PHOENIX-7006
> Project: Phoenix
> Issue Type: Improvement
> Reporter: Viraj Jasani
> Assignee: Sanjeet Malhotra
> Priority: Major
> Fix For: 5.3.0
>
>
> Phoenix max lookback age feature preserves live or deleted row versions that
> are only visible through the max lookback window, it does not preserve any
> unwanted row versions that should not be visible through the max lookback
> window. More details on the max lookback redesign: PHOENIX-6888
> As of today, maxlookback age is only configurable at the cluster level
> (config key: {_}phoenix.max.lookback.age.seconds{_}), meaning the same value
> is used by all tables. This does not allow individual table level compaction
> scanner to be able to retain data based on the table level maxlookback age.
> Setting max lookback age at the table level can serve multiple purposes e.g.
> change-data-capture (PHOENIX-7001) for individual table should have it's own
> latest data retention period.
> The purpose of this Jira is to allow maxlookback age as a table level
> property:
> * New column in SYSTEM.CATALOG to preserve table level maxlookback age
> * PTable object to read the value of maxlookback from SYSTEM.CATALOG
> * Allow CREATE/ALTER TABLE DDLs to provide maxlookback attribute
> * CompactionScanner should use table level maxlookbackAge, if available,
> else use cluster level config
--
This message was sent by Atlassian Jira
(v8.20.10#820010)