This is an automated email from the ASF dual-hosted git repository.

wchevreuil pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 41244b1be2e HBASE-29569: Implement a built-in TieringValueProvider for 
parsing the date value from the rowkey (#7593) (#7642)
41244b1be2e is described below

commit 41244b1be2ec1c5d0da65aff1faf44c7b5473947
Author: Wellington Ramos Chevreuil <[email protected]>
AuthorDate: Thu Jan 22 11:28:38 2026 +0000

    HBASE-29569: Implement a built-in TieringValueProvider for parsing the date 
value from the rowkey (#7593) (#7642)
    
    (cherry-picked from commit ff8420095e24bc800e9d18a0e98168001151bb7f)
    
    Signed-off-by: Wellington Chevreuil <[email protected]>
---
 .../RowKeyDateTieringValueProvider.java            | 148 +++++++++++++++++
 .../compactions/TestCustomCellTieredCompactor.java | 176 ++++++++++++++++++++
 .../TestRowKeyDateTieringValueProvider.java        | 177 +++++++++++++++++++++
 3 files changed, 501 insertions(+)

diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RowKeyDateTieringValueProvider.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RowKeyDateTieringValueProvider.java
new file mode 100644
index 00000000000..17f2180a663
--- /dev/null
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RowKeyDateTieringValueProvider.java
@@ -0,0 +1,148 @@
+/*
+ * 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.hadoop.hbase.regionserver.compactions;
+
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ExtendedCell;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Provides a tiering value for compactions by extracting and parsing a date 
from the row key. This
+ * implementation uses a configurable regex and date format to locate and 
parse a date substring
+ * from the row key and returns the parsed epoch time in milliseconds. 
Configuration properties can
+ * be set at globally or at table level.
+ */
[email protected]
+public class RowKeyDateTieringValueProvider implements 
CustomTieredCompactor.TieringValueProvider {
+  private static final Logger LOG = 
LoggerFactory.getLogger(RowKeyDateTieringValueProvider.class);
+  public static final String TIERING_KEY_DATE_PATTERN = 
"TIERING_KEY_DATE_PATTERN";
+  public static final String TIERING_KEY_DATE_FORMAT = 
"TIERING_KEY_DATE_FORMAT";
+  public static final String TIERING_KEY_DATE_GROUP = "TIERING_KEY_DATE_GROUP";
+  private Pattern rowKeyPattern;
+  private SimpleDateFormat dateFormat;
+  private Integer rowKeyRegexExtractGroup;
+
+  @Override
+  public void init(Configuration conf) throws Exception {
+    // Initialize regex pattern
+    String regexPatternStr = conf.get(TIERING_KEY_DATE_PATTERN);
+    if (regexPatternStr == null || regexPatternStr.isEmpty()) {
+      throw new IllegalArgumentException(
+        "Configuration property '" + TIERING_KEY_DATE_PATTERN + "' is 
required");
+    }
+    rowKeyPattern = Pattern.compile(regexPatternStr);
+
+    // Initialize date format
+    String dateFormatStr = conf.get(TIERING_KEY_DATE_FORMAT);
+    if (dateFormatStr == null || dateFormatStr.isEmpty()) {
+      throw new IllegalArgumentException(
+        "Configuration property '" + TIERING_KEY_DATE_FORMAT + "' is 
required");
+    }
+    try {
+      dateFormat = new SimpleDateFormat(dateFormatStr);
+      dateFormat.setLenient(false);
+    } catch (Exception e) {
+      throw new IllegalArgumentException("Invalid date format for 
Configuration property '"
+        + TIERING_KEY_DATE_FORMAT + "': " + dateFormatStr, e);
+    }
+
+    // Initialize regex extract group
+    String extractGroupStr = conf.get(TIERING_KEY_DATE_GROUP, "0");
+    try {
+      rowKeyRegexExtractGroup = Integer.parseInt(extractGroupStr);
+    } catch (NumberFormatException e) {
+      throw new IllegalArgumentException(
+        "Configuration property '" + TIERING_KEY_DATE_GROUP + "' must be a 
valid integer", e);
+    }
+    if (rowKeyRegexExtractGroup < 0) {
+      throw new IllegalArgumentException(
+        "Configuration property '" + TIERING_KEY_DATE_GROUP + "' must be 
non-negative");
+    }
+    // Validate extract group exists in pattern
+    int groupCount = rowKeyPattern.matcher("").groupCount();
+    if (rowKeyRegexExtractGroup > groupCount) {
+      throw new IllegalArgumentException(
+        "Extract group " + rowKeyRegexExtractGroup + " exceeds pattern group 
count " + groupCount);
+    }
+
+    LOG.info("Initialized RowKeyDateTieringValueProvider with regex='{}', 
dateFormat='{}' ",
+      regexPatternStr, dateFormat);
+  }
+
+  @Override
+  public long getTieringValue(ExtendedCell cell) {
+    if (rowKeyPattern == null || dateFormat == null || rowKeyRegexExtractGroup 
== null) {
+      throw new IllegalStateException("RowKeyDateTieringValueProvider not 
initialized properly");
+    }
+    byte[] rowArray = new byte[cell.getRowLength()];
+    System.arraycopy(cell.getRowArray(), cell.getRowOffset(), rowArray, 0, 
cell.getRowLength());
+    String rowKeyStr;
+    try {
+      rowKeyStr = Bytes.toString(rowArray);
+      // Validate UTF-8 encoding
+      if (rowKeyStr.contains("\ufffd")) {
+        LOG.debug("Failed to extract the date from row key due to invalid 
UTF-8 encoding");
+        return Long.MAX_VALUE;
+      }
+    } catch (Exception e) {
+      LOG.debug("Failed to convert row key to string", e);
+      return Long.MAX_VALUE;
+    }
+
+    Matcher matcher = rowKeyPattern.matcher(rowKeyStr);
+    if (!matcher.find()) {
+      LOG.debug("Row key '{}' does not match the regex pattern", rowKeyStr);
+      return Long.MAX_VALUE;
+    }
+
+    String extractedValue = null;
+    try {
+      extractedValue = matcher.group(rowKeyRegexExtractGroup);
+      if (extractedValue == null || extractedValue.isEmpty()) {
+        LOG.debug("No value extracted from row key '{}' using regex pattern", 
rowKeyStr);
+        return Long.MAX_VALUE;
+      }
+      return dateFormat.parse(extractedValue).getTime();
+    } catch (ParseException e) {
+      LOG.debug("Error parsing date value '{}' extracted from row key '{}'", 
extractedValue,
+        rowKeyStr, e);
+    } catch (Exception e) {
+      LOG.debug("Unexpected error while parsing date from row key '{}'", 
rowKeyStr, e);
+    }
+    return Long.MAX_VALUE;
+  }
+
+  public Pattern getRowKeyPattern() {
+    return rowKeyPattern;
+  }
+
+  public SimpleDateFormat getDateFormat() {
+    return dateFormat;
+  }
+
+  public Integer getRowKeyRegexExtractGroup() {
+    return rowKeyRegexExtractGroup;
+  }
+}
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCustomCellTieredCompactor.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCustomCellTieredCompactor.java
index 331dd41e4f1..c9a5368de4d 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCustomCellTieredCompactor.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCustomCellTieredCompactor.java
@@ -19,12 +19,17 @@ package org.apache.hadoop.hbase.regionserver.compactions;
 
 import static 
org.apache.hadoop.hbase.regionserver.CustomTieringMultiFileWriter.CUSTOM_TIERING_TIME_RANGE;
 import static 
org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieringValueProvider.TIERING_CELL_QUALIFIER;
+import static 
org.apache.hadoop.hbase.regionserver.compactions.CustomTieredCompactor.TIERING_VALUE_PROVIDER;
+import static 
org.apache.hadoop.hbase.regionserver.compactions.RowKeyDateTieringValueProvider.TIERING_KEY_DATE_FORMAT;
+import static 
org.apache.hadoop.hbase.regionserver.compactions.RowKeyDateTieringValueProvider.TIERING_KEY_DATE_PATTERN;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.text.SimpleDateFormat;
 import java.util.ArrayList;
+import java.util.Date;
 import java.util.List;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -145,4 +150,175 @@ public class TestCustomCellTieredCompactor {
         }
       });
   }
+
+  @Test
+  public void testCustomCellTieredCompactorWithRowKeyDateTieringValue() throws 
Exception {
+    // Restart mini cluster with RowKeyDateTieringValueProvider
+    utility.shutdownMiniCluster();
+    utility.getConfiguration().set(TIERING_VALUE_PROVIDER,
+      RowKeyDateTieringValueProvider.class.getName());
+    utility.startMiniCluster();
+
+    ColumnFamilyDescriptorBuilder clmBuilder = 
ColumnFamilyDescriptorBuilder.newBuilder(FAMILY);
+    clmBuilder.setValue("hbase.hstore.engine.class", 
CustomTieredStoreEngine.class.getName());
+
+    // Table 1: Date at end with format yyyyMMddHHmmssSSS
+    TableName table1Name = TableName.valueOf("testTable1");
+    TableDescriptorBuilder tbl1Builder = 
TableDescriptorBuilder.newBuilder(table1Name);
+    tbl1Builder.setColumnFamily(clmBuilder.build());
+    tbl1Builder.setValue(TIERING_KEY_DATE_PATTERN, "(\\d{17})$");
+    tbl1Builder.setValue(TIERING_KEY_DATE_FORMAT, "yyyyMMddHHmmssSSS");
+    utility.getAdmin().createTable(tbl1Builder.build());
+    utility.waitTableAvailable(table1Name);
+
+    // Table 2: Date at beginning with format yyyy-MM-dd HH:mm:ss
+    TableName table2Name = TableName.valueOf("testTable2");
+    TableDescriptorBuilder tbl2Builder = 
TableDescriptorBuilder.newBuilder(table2Name);
+    tbl2Builder.setColumnFamily(clmBuilder.build());
+    tbl2Builder.setValue(TIERING_KEY_DATE_PATTERN, "^(\\d{4}-\\d{2}-\\d{2} 
\\d{2}:\\d{2}:\\d{2})");
+    tbl2Builder.setValue(TIERING_KEY_DATE_FORMAT, "yyyy-MM-dd HH:mm:ss");
+    utility.getAdmin().createTable(tbl2Builder.build());
+    utility.waitTableAvailable(table2Name);
+
+    Connection connection = utility.getConnection();
+    long recordTime = System.currentTimeMillis();
+    long oldTime = recordTime - (11L * 366L * 24L * 60L * 60L * 1000L);
+
+    SimpleDateFormat sdf1 = new SimpleDateFormat("yyyyMMddHHmmssSSS");
+    SimpleDateFormat sdf2 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+
+    // Write to Table 1 with date at end
+    Table table1 = connection.getTable(table1Name);
+    for (int i = 0; i < 6; i++) {
+      List<Put> puts = new ArrayList<>(2);
+
+      // Old data
+      String oldDate = sdf1.format(new Date(oldTime));
+      Put put = new Put(Bytes.toBytes("row_" + i + "_" + oldDate));
+      put.addColumn(FAMILY, Bytes.toBytes("val"), Bytes.toBytes("v" + i));
+      puts.add(put);
+
+      // Recent data
+      String recentDate = sdf1.format(new Date(recordTime));
+      put = new Put(Bytes.toBytes("row_" + (i + 1000) + "_" + recentDate));
+      put.addColumn(FAMILY, Bytes.toBytes("val"), Bytes.toBytes("v" + (i + 
1000)));
+      puts.add(put);
+
+      table1.put(puts);
+      utility.flush(table1Name);
+    }
+    table1.close();
+
+    // Write to Table 2 with date at beginning
+    Table table2 = connection.getTable(table2Name);
+    for (int i = 0; i < 6; i++) {
+      List<Put> puts = new ArrayList<>(2);
+
+      // Old data
+      String oldDate = sdf2.format(new Date(oldTime));
+      Put put = new Put(Bytes.toBytes(oldDate + "_row_" + i));
+      put.addColumn(FAMILY, Bytes.toBytes("val"), Bytes.toBytes("v" + i));
+      puts.add(put);
+
+      // Recent data
+      String recentDate = sdf2.format(new Date(recordTime));
+      put = new Put(Bytes.toBytes(recentDate + "_row_" + (i + 1000)));
+      put.addColumn(FAMILY, Bytes.toBytes("val"), Bytes.toBytes("v" + (i + 
1000)));
+      puts.add(put);
+
+      table2.put(puts);
+      utility.flush(table2Name);
+    }
+    table2.close();
+
+    // First compaction for Table 1
+    long compactionTime1 = System.currentTimeMillis();
+    utility.getAdmin().majorCompact(table1Name);
+    Waiter.waitFor(utility.getConfiguration(), 5000,
+      () -> 
utility.getMiniHBaseCluster().getMaster().getLastMajorCompactionTimestamp(table1Name)
+          > compactionTime1);
+
+    assertEquals(1, utility.getNumHFiles(table1Name, FAMILY));
+
+    
utility.getMiniHBaseCluster().getRegions(table1Name).get(0).getStore(FAMILY).getStorefiles()
+      .forEach(file -> {
+        byte[] rangeBytes = file.getMetadataValue(CUSTOM_TIERING_TIME_RANGE);
+        assertNotNull(rangeBytes);
+        try {
+          TimeRangeTracker timeRangeTracker = 
TimeRangeTracker.parseFrom(rangeBytes);
+          assertEquals(oldTime, timeRangeTracker.getMin());
+          assertEquals(recordTime, timeRangeTracker.getMax());
+        } catch (IOException e) {
+          fail(e.getMessage());
+        }
+      });
+
+    // Second compaction for Table 1
+    long secondCompactionTime1 = System.currentTimeMillis();
+    utility.getAdmin().majorCompact(table1Name);
+    Waiter.waitFor(utility.getConfiguration(), 5000,
+      () -> 
utility.getMiniHBaseCluster().getMaster().getLastMajorCompactionTimestamp(table1Name)
+          > secondCompactionTime1);
+
+    assertEquals(2, utility.getNumHFiles(table1Name, FAMILY));
+
+    
utility.getMiniHBaseCluster().getRegions(table1Name).get(0).getStore(FAMILY).getStorefiles()
+      .forEach(file -> {
+        byte[] rangeBytes = file.getMetadataValue(CUSTOM_TIERING_TIME_RANGE);
+        assertNotNull(rangeBytes);
+        try {
+          TimeRangeTracker timeRangeTracker = 
TimeRangeTracker.parseFrom(rangeBytes);
+          assertEquals(timeRangeTracker.getMin(), timeRangeTracker.getMax());
+        } catch (IOException e) {
+          fail(e.getMessage());
+        }
+      });
+
+    // First compaction for Table 2
+    long compactionTime2 = System.currentTimeMillis();
+    utility.getAdmin().majorCompact(table2Name);
+    Waiter.waitFor(utility.getConfiguration(), 5000,
+      () -> 
utility.getMiniHBaseCluster().getMaster().getLastMajorCompactionTimestamp(table2Name)
+          > compactionTime2);
+
+    assertEquals(1, utility.getNumHFiles(table2Name, FAMILY));
+
+    
utility.getMiniHBaseCluster().getRegions(table2Name).get(0).getStore(FAMILY).getStorefiles()
+      .forEach(file -> {
+        byte[] rangeBytes = file.getMetadataValue(CUSTOM_TIERING_TIME_RANGE);
+        assertNotNull(rangeBytes);
+        try {
+          TimeRangeTracker timeRangeTracker = 
TimeRangeTracker.parseFrom(rangeBytes);
+          // Table 2 uses yyyy-MM-dd HH:mm:ss format, so we need to account 
for second precision
+          // The parsed time will be truncated to second precision (no 
milliseconds)
+          long expectedOldTime = (oldTime / 1000) * 1000;
+          long expectedRecentTime = (recordTime / 1000) * 1000;
+          assertEquals(expectedOldTime, timeRangeTracker.getMin());
+          assertEquals(expectedRecentTime, timeRangeTracker.getMax());
+        } catch (IOException e) {
+          fail(e.getMessage());
+        }
+      });
+
+    // Second compaction for Table 2
+    long secondCompactionTime2 = System.currentTimeMillis();
+    utility.getAdmin().majorCompact(table2Name);
+    Waiter.waitFor(utility.getConfiguration(), 5000,
+      () -> 
utility.getMiniHBaseCluster().getMaster().getLastMajorCompactionTimestamp(table2Name)
+          > secondCompactionTime2);
+
+    assertEquals(2, utility.getNumHFiles(table2Name, FAMILY));
+
+    
utility.getMiniHBaseCluster().getRegions(table2Name).get(0).getStore(FAMILY).getStorefiles()
+      .forEach(file -> {
+        byte[] rangeBytes = file.getMetadataValue(CUSTOM_TIERING_TIME_RANGE);
+        assertNotNull(rangeBytes);
+        try {
+          TimeRangeTracker timeRangeTracker = 
TimeRangeTracker.parseFrom(rangeBytes);
+          assertEquals(timeRangeTracker.getMin(), timeRangeTracker.getMax());
+        } catch (IOException e) {
+          fail(e.getMessage());
+        }
+      });
+  }
 }
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestRowKeyDateTieringValueProvider.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestRowKeyDateTieringValueProvider.java
new file mode 100644
index 00000000000..0123729e1a6
--- /dev/null
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestRowKeyDateTieringValueProvider.java
@@ -0,0 +1,177 @@
+/*
+ * 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.hadoop.hbase.regionserver.compactions;
+
+import static org.junit.Assert.assertEquals;
+
+import java.text.SimpleDateFormat;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ExtendedCell;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ RegionServerTests.class, SmallTests.class })
+public class TestRowKeyDateTieringValueProvider {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestRowKeyDateTieringValueProvider.class);
+
+  private RowKeyDateTieringValueProvider provider;
+  private Configuration conf;
+
+  @Before
+  public void setUp() {
+    conf = new Configuration();
+    provider = new RowKeyDateTieringValueProvider();
+  }
+
+  @After
+  public void tearDown() {
+    provider = null;
+    conf = null;
+  }
+
+  @Test
+  public void testInitWithValidConfig() throws Exception {
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_PATTERN, 
"(\\d{4}-\\d{2}-\\d{2})");
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_FORMAT, 
"yyyy-MM-dd");
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_GROUP, "1");
+    provider.init(conf);
+    assertEquals(provider.getRowKeyPattern().pattern(), 
"(\\d{4}-\\d{2}-\\d{2})");
+    assertEquals(provider.getDateFormat().toPattern(), "yyyy-MM-dd");
+    assertEquals(Integer.valueOf(1), provider.getRowKeyRegexExtractGroup());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInitWithMissingRegexPattern() throws Exception {
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_FORMAT, 
"yyyy-MM-dd");
+    provider.init(conf);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInitWithMissingDateFormat() throws Exception {
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_PATTERN, 
"(\\d{4}-\\d{2}-\\d{2})");
+    provider.init(conf);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInitWithInvalidDateFormat() throws Exception {
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_PATTERN, 
"(\\d{4}-\\d{2}-\\d{2})");
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_FORMAT, 
"invalid-format");
+    provider.init(conf);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInitWithInvalidExtractGroup() throws Exception {
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_PATTERN, 
"(\\d{4}-\\d{2}-\\d{2})");
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_FORMAT, 
"yyyy-MM-dd");
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_GROUP, "-1");
+    provider.init(conf);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInitWithExtractGroupExceedingPatternGroups() throws 
Exception {
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_PATTERN, 
"(\\d{4}-\\d{2}-\\d{2})");
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_FORMAT, 
"yyyy-MM-dd");
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_GROUP, "2"); // 
Only 1 group in
+                                                                          // 
pattern
+    provider.init(conf);
+  }
+
+  @Test
+  public void testGetTieringValue() throws Exception {
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_PATTERN, 
"_(\\d{4}-\\d{2}-\\d{2})_");
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_FORMAT, 
"yyyy-MM-dd");
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_GROUP, "1");
+    provider.init(conf);
+
+    String dateStr = "2023-10-15";
+    SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd");
+    long expectedTimestamp = sdf.parse(dateStr).getTime();
+
+    String rowKeyStr = "order_" + dateStr + "_details";
+    byte[] rowKey = Bytes.toBytes(rowKeyStr);
+    ExtendedCell cell = PrivateCellUtil.createFirstOnRow(rowKey);
+    long actualTimestamp = provider.getTieringValue(cell);
+
+    assertEquals(expectedTimestamp, actualTimestamp);
+  }
+
+  @Test
+  public void testGetTieringValueWithNonMatchingRowKey() throws Exception {
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_PATTERN, 
"_(\\d{4}-\\d{2}-\\d{2})_");
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_FORMAT, 
"yyyy-MM-dd");
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_GROUP, "1");
+    provider.init(conf);
+
+    String rowKeyStr = "order_details_no_date";
+    byte[] rowKey = Bytes.toBytes(rowKeyStr);
+    ExtendedCell cell = PrivateCellUtil.createFirstOnRow(rowKey);
+    long actualTimestamp = provider.getTieringValue(cell);
+
+    assertEquals(Long.MAX_VALUE, actualTimestamp);
+  }
+
+  @Test
+  public void testGetTieringValueWithInvalidDateInRowKey() throws Exception {
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_PATTERN, 
"_(\\d{14})_");
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_FORMAT, 
"yyyyMMddHHmmss");
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_GROUP, "1");
+    provider.init(conf);
+
+    // Invalid Month (14)
+    String rowKeyStr = "order_20151412124556_date";
+    byte[] rowKey = Bytes.toBytes(rowKeyStr);
+    ExtendedCell cell = PrivateCellUtil.createFirstOnRow(rowKey);
+    long actualTimestamp = provider.getTieringValue(cell);
+
+    assertEquals(Long.MAX_VALUE, actualTimestamp);
+  }
+
+  @Test
+  public void testGetTieringValueWithNonUTF8RowKey() throws Exception {
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_PATTERN, 
"_(\\d{8})_");
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_FORMAT, 
"yyyyMMdd");
+    conf.set(RowKeyDateTieringValueProvider.TIERING_KEY_DATE_GROUP, "1");
+    provider.init(conf);
+
+    // Row key with non-UTF-8 bytes (invalid UTF-8 sequence)
+    byte[] rowKey =
+      new byte[] { 0x6F, 0x72, 0x64, 0x65, 0x72, 0x5F, (byte) 0xFF, (byte) 
0xFE, 0x5F };
+    ExtendedCell cell = PrivateCellUtil.createFirstOnRow(rowKey);
+    long timestamp = provider.getTieringValue(cell);
+
+    assertEquals(Long.MAX_VALUE, timestamp);
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testGetTieringValueWithoutInitialization() {
+    String rowKeyStr = "order_9999999999999999_date";
+    byte[] rowKey = Bytes.toBytes(rowKeyStr);
+    ExtendedCell cell = PrivateCellUtil.createFirstOnRow(rowKey);
+    provider.getTieringValue(cell);
+  }
+}

Reply via email to