You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by st...@apache.org on 2023/08/21 13:06:31 UTC

[phoenix] branch master updated: PHOENIX-6942: Some config properties do not have phoenix prefix

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

stoty pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/phoenix.git


The following commit(s) were added to refs/heads/master by this push:
     new 928e09977c PHOENIX-6942: Some config properties do not have phoenix prefix
928e09977c is described below

commit 928e09977c60900b2fd413c67ccee56644f8921c
Author: Aron Meszaros <me...@gmail.com>
AuthorDate: Wed Aug 16 11:26:53 2023 +0200

    PHOENIX-6942: Some config properties do not have phoenix prefix
    
    add missing prefixes, and add the old property names as deprecated.
---
 .../apache/phoenix/coprocessor/GlobalIndexRegionScanner.java | 11 +++++++++--
 .../coprocessor/UncoveredGlobalIndexRegionScanner.java       | 11 +++++++++--
 .../org/apache/phoenix/hbase/index/IndexRegionObserver.java  |  3 ---
 .../index/write/AbstractParallelWriterIndexCommitter.java    | 10 ++++++++--
 .../org/apache/phoenix/hbase/index/write/IndexWriter.java    | 11 +++++++++--
 .../index/write/TrackingParallelWriterIndexCommitter.java    | 11 +++++++++--
 .../phoenix/mapreduce/util/PhoenixConfigurationUtil.java     |  8 +++++++-
 .../java/org/apache/phoenix/util/PropertiesUtilTest.java     | 12 ++++++++++++
 8 files changed, 63 insertions(+), 14 deletions(-)

diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GlobalIndexRegionScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GlobalIndexRegionScanner.java
index 4eebee0457..1c59488e05 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GlobalIndexRegionScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GlobalIndexRegionScanner.java
@@ -109,9 +109,9 @@ import static org.apache.phoenix.util.ScanUtil.isDummy;
 public abstract class GlobalIndexRegionScanner extends BaseRegionScanner {
     private static final Logger LOGGER = LoggerFactory.getLogger(GlobalIndexRegionScanner.class);
 
-    public static final String NUM_CONCURRENT_INDEX_VERIFY_THREADS_CONF_KEY = "index.verify.threads.max";
+    public static final String NUM_CONCURRENT_INDEX_VERIFY_THREADS_CONF_KEY = "phoenix.index.verify.threads.max";
     public static final int DEFAULT_CONCURRENT_INDEX_VERIFY_THREADS = 16;
-    public static final String INDEX_VERIFY_ROW_COUNTS_PER_TASK_CONF_KEY = "index.verify.row.count.per.task";
+    public static final String INDEX_VERIFY_ROW_COUNTS_PER_TASK_CONF_KEY = "phoenix.index.verify.row.count.per.task";
     public static final int DEFAULT_INDEX_VERIFY_ROW_COUNTS_PER_TASK = 2048;
     public static final String NO_EXPECTED_MUTATION = "No expected mutation";
     public static final String ACTUAL_MUTATION_IS_NULL_OR_EMPTY = "actualMutationList is null or empty";
@@ -162,6 +162,13 @@ public abstract class GlobalIndexRegionScanner extends BaseRegionScanner {
     protected IndexTool.IndexVerifyType verifyType = IndexTool.IndexVerifyType.NONE;
     protected boolean verify = false;
 
+    // This relies on Hadoop Configuration to handle warning about deprecated configs and
+    // to set the correct non-deprecated configs when an old one shows up.
+    static {
+        Configuration.addDeprecation("index.verify.threads.max", NUM_CONCURRENT_INDEX_VERIFY_THREADS_CONF_KEY);
+        Configuration.addDeprecation("index.verify.row.count.per.task", INDEX_VERIFY_ROW_COUNTS_PER_TASK_CONF_KEY);
+    }
+
     public GlobalIndexRegionScanner(final RegionScanner innerScanner,
                                     final Region region,
                                     final Scan scan,
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UncoveredGlobalIndexRegionScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UncoveredGlobalIndexRegionScanner.java
index 390feeebaa..13907af6ef 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UncoveredGlobalIndexRegionScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UncoveredGlobalIndexRegionScanner.java
@@ -65,9 +65,9 @@ import org.slf4j.LoggerFactory;
 public class UncoveredGlobalIndexRegionScanner extends UncoveredIndexRegionScanner {
     private static final Logger LOGGER =
             LoggerFactory.getLogger(UncoveredGlobalIndexRegionScanner.class);
-    public static final String NUM_CONCURRENT_INDEX_THREADS_CONF_KEY = "index.threads.max";
+    public static final String NUM_CONCURRENT_INDEX_THREADS_CONF_KEY = "phoenix.index.threads.max";
     public static final int DEFAULT_CONCURRENT_INDEX_THREADS = 16;
-    public static final String INDEX_ROW_COUNTS_PER_TASK_CONF_KEY = "index.row.count.per.task";
+    public static final String INDEX_ROW_COUNTS_PER_TASK_CONF_KEY = "phoenix.index.row.count.per.task";
     public static final int DEFAULT_INDEX_ROW_COUNTS_PER_TASK = 2048;
 
     protected byte[][] regionEndKeys;
@@ -77,6 +77,13 @@ public class UncoveredGlobalIndexRegionScanner extends UncoveredIndexRegionScann
     protected String exceptionMessage;
     protected final HTableFactory hTableFactory;
 
+    // This relies on Hadoop Configuration to handle warning about deprecated configs and
+    // to set the correct non-deprecated configs when an old one shows up.
+    static {
+        Configuration.addDeprecation("index.threads.max", NUM_CONCURRENT_INDEX_THREADS_CONF_KEY);
+        Configuration.addDeprecation("index.row.count.per.task", INDEX_ROW_COUNTS_PER_TASK_CONF_KEY);
+    }
+
     public UncoveredGlobalIndexRegionScanner(final RegionScanner innerScanner,
                                              final Region region,
                                              final Scan scan,
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java
index 4738f63042..e8d9cb5280 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java
@@ -292,9 +292,6 @@ public class IndexRegionObserver implements RegionCoprocessor, RegionObserver {
   private ThreadLocal<BatchMutateContext> batchMutateContext =
           new ThreadLocal<BatchMutateContext>();
 
-  /** Configuration key for the {@link IndexBuilder} to use */
-  public static final String INDEX_BUILDER_CONF_KEY = "index.builder";
-
   /**
    * Configuration key for if the indexer should check the version of HBase is running. Generally,
    * you only want to ignore this for testing or for custom versions of HBase.
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/AbstractParallelWriterIndexCommitter.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/AbstractParallelWriterIndexCommitter.java
index 0a295410d8..a90dc473d0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/AbstractParallelWriterIndexCommitter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/AbstractParallelWriterIndexCommitter.java
@@ -51,9 +51,9 @@ import org.apache.phoenix.thirdparty.com.google.common.collect.Multimap;
  */
 public abstract class AbstractParallelWriterIndexCommitter implements IndexCommitter {
 
-    public static final String NUM_CONCURRENT_INDEX_WRITER_THREADS_CONF_KEY = "index.writer.threads.max";
+    public static final String NUM_CONCURRENT_INDEX_WRITER_THREADS_CONF_KEY = "phoenix.index.writer.threads.max";
     private static final int DEFAULT_CONCURRENT_INDEX_WRITER_THREADS = 10;
-    public static final String INDEX_WRITER_KEEP_ALIVE_TIME_CONF_KEY = "index.writer.threads.keepalivetime";
+    public static final String INDEX_WRITER_KEEP_ALIVE_TIME_CONF_KEY = "phoenix.index.writer.threads.keepalivetime";
     private static final Logger LOG = LoggerFactory.getLogger(IndexWriter.class);
 
     protected HTableFactory retryingFactory;
@@ -65,6 +65,12 @@ public abstract class AbstractParallelWriterIndexCommitter implements IndexCommi
     protected TaskBatch<Void> tasks;
     protected boolean disableIndexOnFailure = false;
 
+    // This relies on Hadoop Configuration to handle warning about deprecated configs and
+    // to set the correct non-deprecated configs when an old one shows up.
+    static {
+        Configuration.addDeprecation("index.writer.threads.max", NUM_CONCURRENT_INDEX_WRITER_THREADS_CONF_KEY);
+        Configuration.addDeprecation("index.writer.threads.keepalivetime", INDEX_WRITER_KEEP_ALIVE_TIME_CONF_KEY);
+    }
 
     public AbstractParallelWriterIndexCommitter() {}
 
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriter.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriter.java
index 1676669ca7..5b8a187d0c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriter.java
@@ -47,12 +47,19 @@ import org.apache.phoenix.thirdparty.com.google.common.collect.Multimap;
 public class IndexWriter implements Stoppable {
 
   private static final Logger LOGGER = LoggerFactory.getLogger(IndexWriter.class);
-  public static final String INDEX_COMMITTER_CONF_KEY = "index.writer.commiter.class";
-  public static final String INDEX_FAILURE_POLICY_CONF_KEY = "index.writer.failurepolicy.class";
+  public static final String INDEX_COMMITTER_CONF_KEY = "phoenix.index.writer.commiter.class";
+  public static final String INDEX_FAILURE_POLICY_CONF_KEY = "phoenix.index.writer.failurepolicy.class";
   private AtomicBoolean stopped = new AtomicBoolean(false);
   private IndexCommitter writer;
   private IndexFailurePolicy failurePolicy;
 
+  // This relies on Hadoop Configuration to handle warning about deprecated configs and
+  // to set the correct non-deprecated configs when an old one shows up.
+  static {
+    Configuration.addDeprecation("index.writer.commiter.class", INDEX_COMMITTER_CONF_KEY);
+    Configuration.addDeprecation("index.writer.failurepolicy.class", INDEX_FAILURE_POLICY_CONF_KEY);
+  }
+
   /**
    * @throws IOException if the {@link IndexWriter} or {@link IndexFailurePolicy} cannot be
    *           instantiated
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/TrackingParallelWriterIndexCommitter.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/TrackingParallelWriterIndexCommitter.java
index d35c7a56b2..1405397bb3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/TrackingParallelWriterIndexCommitter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/TrackingParallelWriterIndexCommitter.java
@@ -74,9 +74,9 @@ public class TrackingParallelWriterIndexCommitter implements IndexCommitter {
     private static final Logger LOGGER =
             LoggerFactory.getLogger(TrackingParallelWriterIndexCommitter.class);
 
-    public static final String NUM_CONCURRENT_INDEX_WRITER_THREADS_CONF_KEY = "index.writer.threads.max";
+    public static final String NUM_CONCURRENT_INDEX_WRITER_THREADS_CONF_KEY = "phoenix.index.writer.threads.max";
     private static final int DEFAULT_CONCURRENT_INDEX_WRITER_THREADS = 10;
-    private static final String INDEX_WRITER_KEEP_ALIVE_TIME_CONF_KEY = "index.writer.threads.keepalivetime";
+    private static final String INDEX_WRITER_KEEP_ALIVE_TIME_CONF_KEY = "phoenix.index.writer.threads.keepalivetime";
 
     private TaskRunner pool;
     private HTableFactory retryingFactory;
@@ -86,6 +86,13 @@ public class TrackingParallelWriterIndexCommitter implements IndexCommitter {
     private KeyValueBuilder kvBuilder;
     protected boolean disableIndexOnFailure = false;
 
+    // This relies on Hadoop Configuration to handle warning about deprecated configs and
+    // to set the correct non-deprecated configs when an old one shows up.
+    static {
+        Configuration.addDeprecation("index.writer.threads.max", NUM_CONCURRENT_INDEX_WRITER_THREADS_CONF_KEY);
+        Configuration.addDeprecation("index.writer.threads.keepalivetime", INDEX_WRITER_KEEP_ALIVE_TIME_CONF_KEY);
+    }
+
     // for testing
     public TrackingParallelWriterIndexCommitter(String hbaseVersion) {
         kvBuilder = KeyValueBuilder.get(hbaseVersion);
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java
index 15b90e7127..13c5b70e1d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java
@@ -121,7 +121,7 @@ public final class PhoenixConfigurationUtil {
 
     public static final String MAPREDUCE_INPUT_CLUSTER_QUORUM = "phoenix.mapreduce.input.cluster.quorum";
     
-    public static final String MAPREDUCE_OUTPUT_CLUSTER_QUORUM = "phoneix.mapreduce.output.cluster.quorum";
+    public static final String MAPREDUCE_OUTPUT_CLUSTER_QUORUM = "phoenix.mapreduce.output.cluster.quorum";
 
     public static final String INDEX_DISABLED_TIMESTAMP_VALUE = "phoenix.mr.index.disableTimestamp";
 
@@ -247,6 +247,12 @@ public final class PhoenixConfigurationUtil {
         QUERY
     }
 
+    // This relies on Hadoop Configuration to handle warning about deprecated configs and
+    // to set the correct non-deprecated configs when an old one shows up.
+    static {
+        Configuration.addDeprecation("phoneix.mapreduce.output.cluster.quorum", MAPREDUCE_OUTPUT_CLUSTER_QUORUM);
+    }
+
     private PhoenixConfigurationUtil(){
         
     }
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/PropertiesUtilTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/PropertiesUtilTest.java
index 1dc67dadaa..38ad441ea3 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/PropertiesUtilTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/PropertiesUtilTest.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.util;
 
+import static org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil.MAPREDUCE_OUTPUT_CLUSTER_QUORUM;
 import static org.junit.Assert.assertEquals;
 
 import java.sql.SQLException;
@@ -25,6 +26,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 
+import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
 import org.junit.Test;
 
 public class PropertiesUtilTest {
@@ -77,6 +79,16 @@ public class PropertiesUtilTest {
             Long.toString(HConstants.DEFAULT_HBASE_RPC_TIMEOUT * 10));
     }
 
+    @Test
+    public void testDeprecatedProperties() throws Exception {
+        final Configuration conf = HBaseConfiguration.create();
+        conf.set("phoneix.mapreduce.output.cluster.quorum", "myoverridezookeeperhost");
+        String test = PhoenixConfigurationUtil.getOutputCluster(conf);
+        assertEquals("myoverridezookeeperhost", test);
+        assertEquals("myoverridezookeeperhost",
+                conf.get(MAPREDUCE_OUTPUT_CLUSTER_QUORUM));
+    }
+
     private void verifyValidCopy(Properties props) throws SQLException {
         Properties copy = PropertiesUtil.deepCopy(props);
         copy.containsKey(PhoenixRuntime.TENANT_ID_ATTRIB); //This checks the map and NOT the defaults in java.util.Properties