dev-work-8103 commented on code in PR #7593:
URL: https://github.com/apache/hbase/pull/7593#discussion_r2668100562


##########
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCustomCellTieredCompactor.java:
##########
@@ -145,4 +149,97 @@ public void testCustomCellTieredCompactor() throws 
Exception {
         }
       });
   }
+
+  @Test
+  public void 
TestCustomCellTieredCompactorWithRowKeyDateTieringValueProvider() throws 
Exception {
+    utility.getConfiguration().set(TIERING_VALUE_PROVIDER,
+      RowKeyDateTieringValueProvider.class.getName());
+    
utility.getConfiguration().set(RowKeyDateTieringValueProvider.ROWKEY_REGEX_PATTERN,
+      "(\\d{17})$");
+    
utility.getConfiguration().set(RowKeyDateTieringValueProvider.ROWKEY_DATE_FORMAT,
+      "yyyyMMddHHmmssSSS");

Review Comment:
   I have added two new tests, one has Global configurations for value provider 
and other one has table level configurations with 2 different REGEX patterns 
for 2 different tables.
   Thanks.
   



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RowKeyDateTieringValueProvider.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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: -
+ * `hbase.hstore.datatiering.tieringvalueprovider.regexpattern`: Regex used to 
match the row key. -
+ * `hbase.hstore.datatiering.tieringvalueprovider.dateformat`: 
`java.text.SimpleDateFormat` pattern.
+ * - `hbase.hstore.datatiering.tieringvalueprovider.regexextractgroup`: Regex 
group index to
+ * extract. Behavior: - Requires `init(Configuration)` to be called before 
use. - Returns
+ * `Long.MAX_VALUE` if the row key does not match, extraction fails, or date 
parsing fails. - Uses
+ * strict (non\-lenient) date parsing and validates the extract group against 
the pattern.
+ */
[email protected]
+public class RowKeyDateTieringValueProvider implements 
CustomTieredCompactor.TieringValueProvider {
+  private static final Logger LOG = 
LoggerFactory.getLogger(RowKeyDateTieringValueProvider.class);
+  public static final String ROWKEY_REGEX_PATTERN =
+    "hbase.hstore.datatiering.tieringvalueprovider.regexpattern";
+  public static final String ROWKEY_DATE_FORMAT =
+    "hbase.hstore.datatiering.tieringvalueprovider.dateformat";
+  public static final String ROWKEY_REGEX_EXTRACT_GROUP =
+    "hbase.hstore.datatiering.tieringvalueprovider.regexextractgroup";
+  private Pattern rowKeyPattern;
+  private SimpleDateFormat dateFormat;
+  private Integer rowKeyRegexExtractGroup;
+
+  @Override
+  public void init(Configuration conf) throws Exception {
+    // Initialize regex pattern
+    String regexPatternStr = conf.get(ROWKEY_REGEX_PATTERN);
+    if (regexPatternStr == null || regexPatternStr.isEmpty()) {
+      throw new IllegalArgumentException(
+        "Configuration property '" + ROWKEY_REGEX_PATTERN + "' is required");
+    }
+    rowKeyPattern = Pattern.compile(regexPatternStr);
+
+    // Initialize date format
+    String dateFormatStr = conf.get(ROWKEY_DATE_FORMAT);
+    if (dateFormatStr == null || dateFormatStr.isEmpty()) {
+      throw new IllegalArgumentException(
+        "Configuration property '" + ROWKEY_DATE_FORMAT + "' is required");
+    }
+    try {
+      dateFormat = new SimpleDateFormat(dateFormatStr);
+      dateFormat.setLenient(false);
+    } catch (Exception e) {
+      throw new IllegalArgumentException("Invalid date format for 
Configuration property '"
+        + ROWKEY_DATE_FORMAT + "': " + dateFormatStr, e);
+    }
+
+    // Initialize regex extract group
+    String extractGroupStr = conf.get(ROWKEY_REGEX_EXTRACT_GROUP, "0");
+    try {
+      rowKeyRegexExtractGroup = Integer.parseInt(extractGroupStr);
+    } catch (NumberFormatException e) {
+      throw new IllegalArgumentException(
+        "Configuration property '" + ROWKEY_REGEX_EXTRACT_GROUP + "' must be a 
valid integer", e);
+    }
+    if (rowKeyRegexExtractGroup < 0) {
+      throw new IllegalArgumentException(
+        "Configuration property '" + ROWKEY_REGEX_EXTRACT_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("Row key contains invalid UTF-8 sequences");

Review Comment:
   Done
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to