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}