You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafodion.apache.org by se...@apache.org on 2016/10/26 14:12:48 UTC

incubator-trafodion git commit: [TRAFODION-2306] Trafodion customization using its own configuration file

Repository: incubator-trafodion
Updated Branches:
  refs/heads/master 087d91ec8 -> 9c0e5ab94


[TRAFODION-2306] Trafodion customization using its own configuration file

When sqgen is run, the customization resource file for Trafodion trafodion-site.xml
with pre-defined properties, can be found at $MY_SQROOT/etc.

You can add client side hbase properties with 'trafodion.' prefixed. These properties
become effective in the client Trafodion processes with 'trafodion.' stripped off.

When you modify this file, it needs to be copied to all nodes of Trafodion cluster
and Trafodion restarted.

This closes PR #777


Project: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/commit/9c0e5ab9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/tree/9c0e5ab9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/diff/9c0e5ab9

Branch: refs/heads/master
Commit: 9c0e5ab9442d97ad496c1e2a7b939d2fc06338ce
Parents: 087d91e
Author: selvaganesang <se...@esgyn.com>
Authored: Sun Oct 23 01:42:27 2016 +0000
Committer: selvaganesang <se...@apache.org>
Committed: Wed Oct 26 13:57:32 2016 +0000

----------------------------------------------------------------------
 core/sqf/sql/scripts/install_local_hadoop       |   8 --
 core/sqf/sql/scripts/sqgen                      |  15 ++-
 .../sqf/sql/scripts/traf_coprocessor.properties |  26 -----
 core/sqf/sql/scripts/trafodion-site.xml         |  38 +++++++
 .../java/org/trafodion/dtm/HBaseTxClient.java   |   6 +-
 .../main/java/org/trafodion/dtm/TrafInfo.java   |   4 +-
 .../org/trafodion/sql/TrafConfiguration.java    |   1 +
 .../org/trafodion/sql/CoprocessorUtils.java     | 104 -------------------
 .../java/org/trafodion/sql/HBaseClient.java     |  29 +++---
 .../org/trafodion/sql/TrafConfiguration.java    |  70 +++++++++++++
 .../java/org/trafodion/sql/TrafRegionStats.java |   4 +-
 11 files changed, 144 insertions(+), 161 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9c0e5ab9/core/sqf/sql/scripts/install_local_hadoop
----------------------------------------------------------------------
diff --git a/core/sqf/sql/scripts/install_local_hadoop b/core/sqf/sql/scripts/install_local_hadoop
index 7932793..39b0ba2 100755
--- a/core/sqf/sql/scripts/install_local_hadoop
+++ b/core/sqf/sql/scripts/install_local_hadoop
@@ -1634,14 +1634,6 @@ echo "$MY_LOCAL_SW_DIST/${HBASE_TAR}"
    </property>
    ${HREGION_IMPL}
    ${MASTER_VISBILITY_COPROC}
-   <property>
-    <name>hbase.coprocessor.region.classes</name>
-      <value>
-           org.apache.hadoop.hbase.coprocessor.transactional.TrxRegionObserver,
-           org.apache.hadoop.hbase.coprocessor.transactional.TrxRegionEndpoint,
-           org.apache.hadoop.hbase.coprocessor.AggregateImplementation
-      </value>
-   </property>
 </configuration>
 EOF
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9c0e5ab9/core/sqf/sql/scripts/sqgen
----------------------------------------------------------------------
diff --git a/core/sqf/sql/scripts/sqgen b/core/sqf/sql/scripts/sqgen
index 68c3c03..a761f08 100755
--- a/core/sqf/sql/scripts/sqgen
+++ b/core/sqf/sql/scripts/sqgen
@@ -187,7 +187,20 @@ fi
 
 ./gensqstatem2lenv > $SQETC_DIR/sqstatem2l.env
 
-cp ./traf_coprocessor.properties $SQETC_DIR/
+if [ -f $SQETC_DIR/trafodion-site.xml ]; then
+    echo "Trafodion customization file $SQETC_DIR/trafodion-site.xml exists."
+    echo "The file will not be copied."
+    echo
+else
+    echo "Copying Trafodion customization file: $SQETC_DIR/trafodion-site.xml"
+    echo
+    cp ./trafodion-site.xml $SQETC_DIR
+    lv_retcode=$?
+    if [[ $lv_retcode != 0 ]]; then 
+	echo "Error $lv_retcode while copying Exiting..."
+	exit $lv_retcode
+    fi
+fi
 
 # Create configuration database tables
 rm -f sqconfig.db

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9c0e5ab9/core/sqf/sql/scripts/traf_coprocessor.properties
----------------------------------------------------------------------
diff --git a/core/sqf/sql/scripts/traf_coprocessor.properties b/core/sqf/sql/scripts/traf_coprocessor.properties
deleted file mode 100644
index d69e2f2..0000000
--- a/core/sqf/sql/scripts/traf_coprocessor.properties
+++ /dev/null
@@ -1,26 +0,0 @@
-#
-# @@@ START COPYRIGHT @@@
-#
-# 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.
-#
-# @@@ END COPYRIGHT @@@
-#
-
-coprocessors=org.apache.hadoop.hbase.coprocessor.transactional.TrxRegionObserver,org.apache.hadoop.hbase.coprocessor.AggregateImplementation
-MVCC=org.apache.hadoop.hbase.coprocessor.transactional.TrxRegionEndpoint
-SSCC=org.apache.hadoop.hbase.coprocessor.transactional.SsccRegionEndpoint

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9c0e5ab9/core/sqf/sql/scripts/trafodion-site.xml
----------------------------------------------------------------------
diff --git a/core/sqf/sql/scripts/trafodion-site.xml b/core/sqf/sql/scripts/trafodion-site.xml
new file mode 100644
index 0000000..fc04716
--- /dev/null
+++ b/core/sqf/sql/scripts/trafodion-site.xml
@@ -0,0 +1,38 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+ <!--
+/**
+ * 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.
+ */
+-->
+
+<!-- Put Trafodion site-specific property overrides in this file. -->
+
+<configuration>
+   <property>
+    <name>hbase.coprocessor.region.classes</name>
+      <value>
+           org.apache.hadoop.hbase.coprocessor.transactional.TrxRegionObserver,
+           org.apache.hadoop.hbase.coprocessor.transactional.TrxRegionEndpoint,
+           org.apache.hadoop.hbase.coprocessor.AggregateImplementation
+      </value>
+   </property>
+   <property>
+    <name>hbase.client.scanner.timeout.period</name>
+    <value>3600000</value>
+   </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9c0e5ab9/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/HBaseTxClient.java
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/HBaseTxClient.java b/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/HBaseTxClient.java
index 1cf3025..77c4808 100644
--- a/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/HBaseTxClient.java
+++ b/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/HBaseTxClient.java
@@ -57,7 +57,6 @@ import org.apache.hadoop.hbase.client.transactional.TransactionMap;
 import org.apache.hadoop.hbase.client.transactional.TransactionalReturn;
 import org.apache.hadoop.hbase.client.transactional.TmDDL;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
@@ -74,6 +73,7 @@ import org.apache.hadoop.hbase.regionserver.transactional.IdTmException;
 import org.apache.hadoop.hbase.regionserver.transactional.IdTmId;
 
 import org.apache.zookeeper.KeeperException;
+import org.trafodion.sql.TrafConfiguration;
 
 import java.util.Map;
 import java.util.HashMap;
@@ -124,7 +124,7 @@ public class HBaseTxClient {
       if (LOG.isDebugEnabled()) LOG.debug("Enter init, hBasePath:" + hBasePath);
       if (LOG.isTraceEnabled()) LOG.trace("mapTransactionStates " + mapTransactionStates + " entries " + mapTransactionStates.size());
       if (config == null) {
-         config = HBaseConfiguration.create();
+         config = TrafConfiguration.create();
          connection = ConnectionFactory.createConnection(config);
       }
       config.set("hbase.zookeeper.quorum", zkServers);
@@ -203,7 +203,7 @@ public class HBaseTxClient {
       setupLog4j();
       if (LOG.isDebugEnabled()) LOG.debug("Enter init(" + dtmid + ")");
       if (config == null) {
-         config = HBaseConfiguration.create();
+         config = TrafConfiguration.create();
          connection = ConnectionFactory.createConnection(config);
       }
       config.set("hbase.hregion.impl", "org.apache.hadoop.hbase.regionserver.transactional.TransactionalRegion");

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9c0e5ab9/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/TrafInfo.java
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/TrafInfo.java b/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/TrafInfo.java
index 777eaea..495ad46 100644
--- a/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/TrafInfo.java
+++ b/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/dtm/TrafInfo.java
@@ -30,7 +30,6 @@ import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -38,6 +37,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
 //H98import org.apache.hadoop.hbase.ipc.TransactionalRegionInterface;
 //H98import org.apache.hadoop.hbase.ipc.HRegionInterface;
 
+import org.trafodion.sql.TrafConfiguration;
 
 public class TrafInfo {
 
@@ -50,7 +50,7 @@ public class TrafInfo {
     }
 
     public void init() throws IOException {
-        this.config = HBaseConfiguration.create();
+        this.config = TrafConfiguration.create();
         this.connection = ConnectionFactory.createConnection(config);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9c0e5ab9/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/sql/TrafConfiguration.java
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/sql/TrafConfiguration.java b/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/sql/TrafConfiguration.java
new file mode 120000
index 0000000..811c58c
--- /dev/null
+++ b/core/sqf/src/seatrans/tm/hbasetmlib2/src/main/java/org/trafodion/sql/TrafConfiguration.java
@@ -0,0 +1 @@
+../../../../../../../../../../../sql/src/main/java/org/trafodion/sql/TrafConfiguration.java
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9c0e5ab9/core/sql/src/main/java/org/trafodion/sql/CoprocessorUtils.java
----------------------------------------------------------------------
diff --git a/core/sql/src/main/java/org/trafodion/sql/CoprocessorUtils.java b/core/sql/src/main/java/org/trafodion/sql/CoprocessorUtils.java
deleted file mode 100644
index 9a3d75c..0000000
--- a/core/sql/src/main/java/org/trafodion/sql/CoprocessorUtils.java
+++ /dev/null
@@ -1,104 +0,0 @@
-// @@@ START COPYRIGHT @@@
-// //
-// // 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.
-// //
-// // @@@ END COPYRIGHT @@@
-
-package org.trafodion.sql;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.configuration.Configuration;
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.commons.configuration.PropertiesConfiguration;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.log4j.Logger;
-
-public class CoprocessorUtils {
-    private static Logger logger = Logger.getLogger(CoprocessorUtils.class.getName());
-    private static List<String> coprocessors = new ArrayList<String>();
-    private static String MVCC = null;
-    private static String SSCC = null;
-
-    static {
-        init();
-    }
-
-    private static void init() {
-        Configuration config = null;
-        try {
-            String path = System.getenv("MY_SQROOT") + "/etc/traf_coprocessor.properties";
-            config = new PropertiesConfiguration(path);
-        } catch (ConfigurationException e) {
-            logger.error("error when finding trafcoprocess.properties");
-            e.printStackTrace();
-        }
-
-        if (config != null) {
-            for (String coprocessor : config.getStringArray("coprocessors")) {
-                coprocessors.add(coprocessor);
-            }
-            MVCC = config.getString("MVCC");
-            SSCC = config.getString("SSCC");
-        }
-    }
-
-    //boolean as return ,to make sure whether changes take place in HTableDescriptor
-    public static boolean addCoprocessor(String currentAllClassName, HTableDescriptor desc, boolean isMVCC) throws IOException {
-        boolean retVal = false; 
-        if (coprocessors == null) {
-            return retVal;
-        }
-        for (String coprocess : coprocessors) {
-            if ((currentAllClassName == null || !currentAllClassName.contains(coprocess)) && !desc.hasCoprocessor(coprocess)) {
-                desc.addCoprocessor(coprocess);
-                retVal = true;
-            }
-        }
-        
-        if (isMVCC && (currentAllClassName == null || !currentAllClassName.contains(MVCC)) && !desc.hasCoprocessor(MVCC)) {
-            desc.addCoprocessor(MVCC);
-            retVal = true;
-        } else if (!isMVCC && (currentAllClassName == null || !currentAllClassName.contains(SSCC)) && !desc.hasCoprocessor(SSCC)) {
-            desc.addCoprocessor(SSCC);
-            retVal = true;
-        }
-
-        return retVal;
-    }
-
-    public static boolean addCoprocessor(String currentAllClassName, HTableDescriptor desc) throws IOException {
-        return addCoprocessor(currentAllClassName, desc, true);
-    }
-    public static void main(String[] args) throws IOException {
-        System.out.println("================CoprocessorUtils.main======================");
-        String currentAllClassName = "";
-        HTableDescriptor desc = new HTableDescriptor();
-        boolean isMVCC = true;
-        addCoprocessor(currentAllClassName, desc, isMVCC);
-
-        List<String> list = desc.getCoprocessors();
-
-        for (String string : list) {
-            System.out.println(string);
-        }
-        System.out.println("================CoprocessorUtils.main======================");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9c0e5ab9/core/sql/src/main/java/org/trafodion/sql/HBaseClient.java
----------------------------------------------------------------------
diff --git a/core/sql/src/main/java/org/trafodion/sql/HBaseClient.java b/core/sql/src/main/java/org/trafodion/sql/HBaseClient.java
index 5ab258c..c007f74 100644
--- a/core/sql/src/main/java/org/trafodion/sql/HBaseClient.java
+++ b/core/sql/src/main/java/org/trafodion/sql/HBaseClient.java
@@ -37,7 +37,6 @@ import java.net.URISyntaxException;
 import org.apache.log4j.PropertyConfigurator;
 import org.apache.log4j.Logger;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Admin;
@@ -73,6 +72,7 @@ import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy;
 import org.apache.hadoop.hbase.client.Durability;
 import org.trafodion.sql.HTableClient;
+import org.trafodion.sql.TrafConfiguration;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.RegionLoad;
 import org.apache.hadoop.hbase.client.HTable;
@@ -101,7 +101,7 @@ import com.google.protobuf.ServiceException;
 public class HBaseClient {
 
     static Logger logger = Logger.getLogger(HBaseClient.class.getName());
-    private static Configuration config = HBaseConfiguration.create();
+    private static Configuration config = null;
     private RMInterface table = null;
 
     // variables used for getRegionStats() and getClusterStats()
@@ -155,21 +155,11 @@ public class HBaseClient {
     		confFile = System.getenv("MY_SQROOT") + "/conf/log4j.hdfs.config";
     	}
     	PropertyConfigurator.configure(confFile);
+        config = TrafConfiguration.create();
     }
 
     
     static public Connection getConnection() throws IOException {
-        // On some distributions, the hbase.client.scanner.timeout.period setting is
-        // too small, resulting in annoying SocketTimeoutExceptions during operations
-        // such as UPDATE STATISTICS on very large tables. On CDH 5.4.5 in particular
-        // we have seen this. Unfortunately Cloudera Manager does not allow us to 
-        // change this setting, and setting it manually in hbase-site.xml doesn't work
-        // because a later Cloudera Manager deploy would just overwrite it. So, we
-        // programmatically check the setting here and insure it is at least 1 hour.
-        long configuredTimeout = config.getLong("hbase.client.scanner.timeout.period",0);
-        if (configuredTimeout < 3600000 /* 1 hour */)
-          config.setLong("hbase.client.scanner.timeout.period",3600000);    
- 
         if (connection == null) 
               connection = ConnectionFactory.createConnection(config);
         return connection;
@@ -185,13 +175,22 @@ public class HBaseClient {
         table = new RMInterface(connection);
         return true;
     }
+
+    private void addCoprocessor(HTableDescriptor desc) throws IOException {
+        String[] coprocessors = config.getStrings("hbase.coprocessor.region.classes");
+        if (coprocessors != null) {
+           for (int i = 0; i < coprocessors.length ; i++) {
+               desc.addCoprocessor(coprocessors[i].trim());
+           }
+        }
+    }
  
     public boolean create(String tblName, Object[]  colFamNameList,
                           boolean isMVCC) 
         throws IOException, MasterNotRunningException {
             if (logger.isDebugEnabled()) logger.debug("HBaseClient.create(" + tblName + ") called, and MVCC is " + isMVCC + ".");
             HTableDescriptor desc = new HTableDescriptor(tblName);
-            CoprocessorUtils.addCoprocessor(config.get("hbase.coprocessor.region.classes"), desc, isMVCC);
+            addCoprocessor(desc);
             for (int i = 0; i < colFamNameList.length ; i++) {
 		String  colFam = (String)colFamNameList[i];
                 HColumnDescriptor colDesc = new HColumnDescriptor(colFam);
@@ -442,7 +441,7 @@ public class HBaseClient {
             if (logger.isDebugEnabled()) logger.debug("HBaseClient.createk(" + tblName + ") called.");
             String trueStr = "TRUE";
             HTableDescriptor desc = new HTableDescriptor(tblName);
-            CoprocessorUtils.addCoprocessor(config.get("hbase.coprocessor.region.classes"), desc, isMVCC);
+            addCoprocessor(desc);
             int defaultVersionsValue = 0;
             if (isMVCC)
                 defaultVersionsValue = DtmConst.MVCC_MAX_VERSION;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9c0e5ab9/core/sql/src/main/java/org/trafodion/sql/TrafConfiguration.java
----------------------------------------------------------------------
diff --git a/core/sql/src/main/java/org/trafodion/sql/TrafConfiguration.java b/core/sql/src/main/java/org/trafodion/sql/TrafConfiguration.java
new file mode 100644
index 0000000..0d7e44a
--- /dev/null
+++ b/core/sql/src/main/java/org/trafodion/sql/TrafConfiguration.java
@@ -0,0 +1,70 @@
+/**
+ * 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.trafodion.sql;
+
+import org.apache.log4j.Logger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.fs.Path;
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ * Adds Trafodion configuration files to a Configuration
+ */
+public class TrafConfiguration extends HBaseConfiguration {
+
+  static Logger logger = Logger.getLogger(TrafConfiguration.class.getName());
+
+  public static Configuration addTrafResources(Configuration conf) {
+    Configuration lv_conf = new Configuration();
+    String trafSiteXml = new String(System.getenv("MY_SQROOT") + "/etc/trafodion-site.xml");
+    Path fileRes = new Path(trafSiteXml);
+    lv_conf.addResource(fileRes);
+    Iterator<Map.Entry<String,String>> iter = lv_conf.iterator();
+    String key;
+    while (iter.hasNext()) {
+       Map.Entry<String,String> entry = iter.next();
+       key = entry.getKey();
+       if (key.startsWith("trafodion."))
+          key = key.substring(10); // 10 - length of trafodion.
+       conf.set(key, entry.getValue());
+    }
+    return conf;
+  }
+
+  /**
+   * Creates a Configuration with Trafodion resources
+   * @return a Configuration with Trafodion and HBase resources
+   */
+  public static Configuration create() {
+    Configuration conf = HBaseConfiguration.create();
+    return addTrafResources(conf);
+  }
+
+  /**
+   * @param that Configuration to clone.
+   * @return a Configuration created with the trafodion-site.xml files plus
+   * the given configuration.
+   */
+  public static Configuration create(final Configuration that) {
+    Configuration conf = create();
+    merge(conf, that);
+    return conf;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9c0e5ab9/core/sql/src/main/java/org/trafodion/sql/TrafRegionStats.java
----------------------------------------------------------------------
diff --git a/core/sql/src/main/java/org/trafodion/sql/TrafRegionStats.java b/core/sql/src/main/java/org/trafodion/sql/TrafRegionStats.java
index 89fa90f..42a9fba 100644
--- a/core/sql/src/main/java/org/trafodion/sql/TrafRegionStats.java
+++ b/core/sql/src/main/java/org/trafodion/sql/TrafRegionStats.java
@@ -24,7 +24,6 @@ package org.trafodion.sql;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.RegionLoad;
@@ -33,6 +32,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.trafodion.sql.TrafConfiguration;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -158,7 +158,7 @@ public class TrafRegionStats {
 
 
     public TrafRegionStats () throws IOException {
-        Configuration config = HBaseConfiguration.create();
+        Configuration config = TrafConfiguration.create();
 
         hbAdmin = new HBaseAdmin(config);
     }