001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.regionserver.compactions; 019 020import java.text.ParseException; 021import java.text.SimpleDateFormat; 022import java.util.regex.Matcher; 023import java.util.regex.Pattern; 024import org.apache.hadoop.conf.Configuration; 025import org.apache.hadoop.hbase.ExtendedCell; 026import org.apache.hadoop.hbase.util.Bytes; 027import org.apache.yetus.audience.InterfaceAudience; 028import org.slf4j.Logger; 029import org.slf4j.LoggerFactory; 030 031/** 032 * Provides a tiering value for compactions by extracting and parsing a date from the row key. This 033 * implementation uses a configurable regex and date format to locate and parse a date substring 034 * from the row key and returns the parsed epoch time in milliseconds. Configuration properties can 035 * be set at globally or at table level. 036 */ 037@InterfaceAudience.Private 038public class RowKeyDateTieringValueProvider implements CustomTieredCompactor.TieringValueProvider { 039 private static final Logger LOG = LoggerFactory.getLogger(RowKeyDateTieringValueProvider.class); 040 public static final String TIERING_KEY_DATE_PATTERN = "TIERING_KEY_DATE_PATTERN"; 041 public static final String TIERING_KEY_DATE_FORMAT = "TIERING_KEY_DATE_FORMAT"; 042 public static final String TIERING_KEY_DATE_GROUP = "TIERING_KEY_DATE_GROUP"; 043 private Pattern rowKeyPattern; 044 private SimpleDateFormat dateFormat; 045 private Integer rowKeyRegexExtractGroup; 046 047 @Override 048 public void init(Configuration conf) throws Exception { 049 // Initialize regex pattern 050 String regexPatternStr = conf.get(TIERING_KEY_DATE_PATTERN); 051 if (regexPatternStr == null || regexPatternStr.isEmpty()) { 052 throw new IllegalArgumentException( 053 "Configuration property '" + TIERING_KEY_DATE_PATTERN + "' is required"); 054 } 055 rowKeyPattern = Pattern.compile(regexPatternStr); 056 057 // Initialize date format 058 String dateFormatStr = conf.get(TIERING_KEY_DATE_FORMAT); 059 if (dateFormatStr == null || dateFormatStr.isEmpty()) { 060 throw new IllegalArgumentException( 061 "Configuration property '" + TIERING_KEY_DATE_FORMAT + "' is required"); 062 } 063 try { 064 dateFormat = new SimpleDateFormat(dateFormatStr); 065 dateFormat.setLenient(false); 066 } catch (Exception e) { 067 throw new IllegalArgumentException("Invalid date format for Configuration property '" 068 + TIERING_KEY_DATE_FORMAT + "': " + dateFormatStr, e); 069 } 070 071 // Initialize regex extract group 072 String extractGroupStr = conf.get(TIERING_KEY_DATE_GROUP, "0"); 073 try { 074 rowKeyRegexExtractGroup = Integer.parseInt(extractGroupStr); 075 } catch (NumberFormatException e) { 076 throw new IllegalArgumentException( 077 "Configuration property '" + TIERING_KEY_DATE_GROUP + "' must be a valid integer", e); 078 } 079 if (rowKeyRegexExtractGroup < 0) { 080 throw new IllegalArgumentException( 081 "Configuration property '" + TIERING_KEY_DATE_GROUP + "' must be non-negative"); 082 } 083 // Validate extract group exists in pattern 084 int groupCount = rowKeyPattern.matcher("").groupCount(); 085 if (rowKeyRegexExtractGroup > groupCount) { 086 throw new IllegalArgumentException( 087 "Extract group " + rowKeyRegexExtractGroup + " exceeds pattern group count " + groupCount); 088 } 089 090 LOG.info("Initialized RowKeyDateTieringValueProvider with regex='{}', dateFormat='{}' ", 091 regexPatternStr, dateFormat); 092 } 093 094 @Override 095 public long getTieringValue(ExtendedCell cell) { 096 if (rowKeyPattern == null || dateFormat == null || rowKeyRegexExtractGroup == null) { 097 throw new IllegalStateException("RowKeyDateTieringValueProvider not initialized properly"); 098 } 099 byte[] rowArray = new byte[cell.getRowLength()]; 100 System.arraycopy(cell.getRowArray(), cell.getRowOffset(), rowArray, 0, cell.getRowLength()); 101 String rowKeyStr; 102 try { 103 rowKeyStr = Bytes.toString(rowArray); 104 // Validate UTF-8 encoding 105 if (rowKeyStr.contains("\ufffd")) { 106 LOG.debug("Failed to extract the date from row key due to invalid UTF-8 encoding"); 107 return Long.MAX_VALUE; 108 } 109 } catch (Exception e) { 110 LOG.debug("Failed to convert row key to string", e); 111 return Long.MAX_VALUE; 112 } 113 114 Matcher matcher = rowKeyPattern.matcher(rowKeyStr); 115 if (!matcher.find()) { 116 LOG.debug("Row key '{}' does not match the regex pattern", rowKeyStr); 117 return Long.MAX_VALUE; 118 } 119 120 String extractedValue = null; 121 try { 122 extractedValue = matcher.group(rowKeyRegexExtractGroup); 123 if (extractedValue == null || extractedValue.isEmpty()) { 124 LOG.debug("No value extracted from row key '{}' using regex pattern", rowKeyStr); 125 return Long.MAX_VALUE; 126 } 127 return dateFormat.parse(extractedValue).getTime(); 128 } catch (ParseException e) { 129 LOG.debug("Error parsing date value '{}' extracted from row key '{}'", extractedValue, 130 rowKeyStr, e); 131 } catch (Exception e) { 132 LOG.debug("Unexpected error while parsing date from row key '{}'", rowKeyStr, e); 133 } 134 return Long.MAX_VALUE; 135 } 136 137 public Pattern getRowKeyPattern() { 138 return rowKeyPattern; 139 } 140 141 public SimpleDateFormat getDateFormat() { 142 return dateFormat; 143 } 144 145 public Integer getRowKeyRegexExtractGroup() { 146 return rowKeyRegexExtractGroup; 147 } 148}