[
https://issues.apache.org/jira/browse/PHOENIX-6888?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17696113#comment-17696113
]
ASF GitHub Bot commented on PHOENIX-6888:
-----------------------------------------
stoty commented on code in PR #1569:
URL: https://github.com/apache/phoenix/pull/1569#discussion_r1124272815
##########
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java:
##########
@@ -426,58 +421,39 @@ RegionScanner getWrappedScanner(final
ObserverContext<RegionCoprocessorEnvironme
public void
preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Store
store,
ScanType scanType, ScanOptions
options, CompactionLifeCycleTracker tracker,
CompactionRequest request) throws
IOException {
- Configuration conf = c.getEnvironment().getConfiguration();
- if (isMaxLookbackTimeEnabled(conf)) {
- setScanOptionsForFlushesAndCompactions(conf, options, store,
scanType);
- }
+ setScanOptionsForFlushesAndCompactions(options);
}
@Override
public void
preFlushScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Store
store,
ScanOptions options,
FlushLifeCycleTracker tracker) throws IOException {
- Configuration conf = c.getEnvironment().getConfiguration();
- if (isMaxLookbackTimeEnabled(conf)) {
- setScanOptionsForFlushesAndCompactions(conf, options, store,
ScanType.COMPACT_RETAIN_DELETES);
- }
+ setScanOptionsForFlushesAndCompactions(options);
}
@Override
public void preMemStoreCompactionCompactScannerOpen(
- ObserverContext<RegionCoprocessorEnvironment> c, Store store,
ScanOptions options)
- throws IOException {
- Configuration conf = c.getEnvironment().getConfiguration();
- if (isMaxLookbackTimeEnabled(conf)) {
- MemoryCompactionPolicy inMemPolicy =
- store.getColumnFamilyDescriptor().getInMemoryCompaction();
- ScanType scanType;
- //the eager and adaptive in-memory compaction policies can purge
versions; the others
- // can't. (Eager always does; adaptive sometimes does)
- if (inMemPolicy.equals(MemoryCompactionPolicy.EAGER) ||
- inMemPolicy.equals(MemoryCompactionPolicy.ADAPTIVE)) {
- scanType = ScanType.COMPACT_DROP_DELETES;
- } else {
- scanType = ScanType.COMPACT_RETAIN_DELETES;
- }
- setScanOptionsForFlushesAndCompactions(conf, options, store,
scanType);
- }
+ ObserverContext<RegionCoprocessorEnvironment> c, Store store,
ScanOptions options) {
+ setScanOptionsForFlushesAndCompactions(options);
+
}
@Override
public void
preStoreScannerOpen(ObserverContext<RegionCoprocessorEnvironment> ctx, Store
store,
ScanOptions options) throws
IOException {
- if (!storeFileScanDoesntNeedAlteration(options)) {
- //PHOENIX-4277
> Fixing TTL and Max Lookback Issues for Phoenix Tables
> -----------------------------------------------------
>
> Key: PHOENIX-6888
> URL: https://issues.apache.org/jira/browse/PHOENIX-6888
> Project: Phoenix
> Issue Type: Bug
> Affects Versions: 5.1.3
> Reporter: Kadir Ozdemir
> Assignee: Kadir Ozdemir
> Priority: Major
>
> In HBase, the unit of data is a cell and data retention rules are executed at
> the cell level. These rules are defined at the column family level. Phoenix
> leverages the data retention features of HBase and exposes them to its users
> to provide its TTL feature at the table level. However, these rules (since
> they are defined at the cell level instead of the row level) results in
> partial row retention that in turn creates data integrity issues at the
> Phoenix level.
> Similarly, Phoenix’s max lookback feature leverages HBase deleted data
> retention capabilities to preserve deleted cells within a configurable max
> lookback. This requires two data retention windows, max lookback and TTL. One
> end of these windows is the current time and the end is a moment in the past
> (i.e., current time minus the window size). Typically, the max lookback
> window is shorter than the TTL window. In the max lookback window, we would
> like to preserve the complete history of mutations regardless of how many
> cell versions these mutations generated. In the remaining TTL window outside
> the max lookback, we would like to apply the data retention rules defined
> above. However, HBase provides only one data retention window. Thus, the max
> lookback window had to be extended to become TTL window and the max lookback
> feature results in unwantedly retaining deleted data for the maximum of max
> lookback and TTL periods.
> This Jira is to fix both of these issues.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)