You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2016/09/12 20:24:35 UTC

[03/31] hive git commit: HIVE-14710 : unify DB product type treatment in directsql and txnhandler (Sergey Shelukhin, reviewed by Alan Gates)

HIVE-14710 : unify DB product type treatment in directsql and txnhandler (Sergey Shelukhin, reviewed by Alan Gates)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/8cec20d9
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/8cec20d9
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/8cec20d9

Branch: refs/heads/hive-14535
Commit: 8cec20d97c8c61e41d38b52f435c7ed55b1d1761
Parents: cd6c3cd
Author: Sergey Shelukhin <se...@apache.org>
Authored: Wed Sep 7 19:03:29 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Wed Sep 7 19:03:29 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/metastore/DatabaseProduct.java  | 75 ++++++++++++++++++++
 .../hive/metastore/MetaStoreDirectSql.java      | 58 ++++++---------
 .../hadoop/hive/metastore/txn/TxnHandler.java   | 52 ++++----------
 .../hadoop/hive/metastore/txn/TestTxnUtils.java |  7 +-
 4 files changed, 112 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8cec20d9/metastore/src/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java b/metastore/src/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java
new file mode 100644
index 0000000..33abbb2
--- /dev/null
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java
@@ -0,0 +1,75 @@
+/**
+ * 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.hive.metastore;
+
+import java.sql.SQLException;
+import java.sql.SQLTransactionRollbackException;
+
+/** Database product infered via JDBC. */
+public enum DatabaseProduct {
+  DERBY, MYSQL, POSTGRES, ORACLE, SQLSERVER, OTHER;
+
+  /**
+   * Determine the database product type
+   * @param conn database connection
+   * @return database product type
+   */
+  public static DatabaseProduct determineDatabaseProduct(String productName) throws SQLException {
+    if (productName == null) {
+      return OTHER;
+    }
+    productName = productName.toLowerCase();
+    if (productName.contains("derby")) {
+      return DERBY;
+    } else if (productName.contains("microsoft sql server")) {
+      return SQLSERVER;
+    } else if (productName.contains("mysql")) {
+      return MYSQL;
+    } else if (productName.contains("oracle")) {
+      return ORACLE;
+    } else if (productName.contains("postgresql")) {
+      return POSTGRES;
+    } else {
+      return OTHER;
+    }
+  }
+
+  public static boolean isDeadlock(DatabaseProduct dbProduct, SQLException e) {
+    return e instanceof SQLTransactionRollbackException
+        || ((dbProduct == MYSQL || dbProduct == POSTGRES || dbProduct == SQLSERVER)
+            && e.getSQLState().equals("40001"))
+        || (dbProduct == POSTGRES && e.getSQLState().equals("40P01"))
+        || (dbProduct == ORACLE && (e.getMessage().contains("deadlock detected")
+            || e.getMessage().contains("can't serialize access for this transaction")));
+  }
+
+  /**
+   * Whether the RDBMS has restrictions on IN list size (explicit, or poor perf-based).
+   */
+  public static boolean needsInBatching(DatabaseProduct dbType) {
+    return dbType == ORACLE || dbType == SQLSERVER;
+  }
+
+  /**
+   * Whether the RDBMS has a bug in join and filter operation order described in DERBY-6358.
+   */
+  public static boolean hasJoinOperationOrderBug(DatabaseProduct dbType) {
+    return dbType == DERBY || dbType == ORACLE;
+  }
+};

http://git-wip-us.apache.org/repos/asf/hive/blob/8cec20d9/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
index 8eeb1c4..561f3e3 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
@@ -88,14 +88,6 @@ import com.google.common.collect.Lists;
  * to SQL stores only. There's always a way to do without direct SQL.
  */
 class MetaStoreDirectSql {
-  private static enum DB {
-    MYSQL,
-    ORACLE,
-    MSSQL,
-    DERBY,
-    OTHER
-  }
-
   private static final int NO_BATCHING = -1, DETECT_BATCHING = 0;
 
   private static final Logger LOG = LoggerFactory.getLogger(MetaStoreDirectSql.class);
@@ -109,7 +101,7 @@ class MetaStoreDirectSql {
    *
    * Use sparingly, we don't want to devolve into another DataNucleus...
    */
-  private final DB dbType;
+  private final DatabaseProduct dbType;
   private final int batchSize;
   private final boolean convertMapNullsToEmptyStrings;
   private final String defaultPartName;
@@ -123,10 +115,17 @@ class MetaStoreDirectSql {
 
   public MetaStoreDirectSql(PersistenceManager pm, Configuration conf) {
     this.pm = pm;
-    this.dbType = determineDbType();
+    DatabaseProduct dbType = null;
+    try {
+      dbType = DatabaseProduct.determineDatabaseProduct(getProductName());
+    } catch (SQLException e) {
+      LOG.warn("Cannot determine database product; assuming OTHER", e);
+      dbType = DatabaseProduct.OTHER;
+    }
+    this.dbType = dbType;
     int batchSize = HiveConf.getIntVar(conf, ConfVars.METASTORE_DIRECT_SQL_PARTITION_BATCH_SIZE);
     if (batchSize == DETECT_BATCHING) {
-      batchSize = (dbType == DB.ORACLE || dbType == DB.MSSQL) ? 1000 : NO_BATCHING;
+      batchSize = DatabaseProduct.needsInBatching(dbType) ? 1000 : NO_BATCHING;
     }
     this.batchSize = batchSize;
 
@@ -136,7 +135,7 @@ class MetaStoreDirectSql {
 
     String jdoIdFactory = HiveConf.getVar(conf, ConfVars.METASTORE_IDENTIFIER_FACTORY);
     if (! ("datanucleus1".equalsIgnoreCase(jdoIdFactory))){
-      LOG.warn("Underlying metastore does not use 'datanuclues1' for its ORM naming scheme."
+      LOG.warn("Underlying metastore does not use 'datanucleus1' for its ORM naming scheme."
           + " Disabling directSQL as it uses hand-hardcoded SQL with that assumption.");
       isCompatibleDatastore = false;
     } else {
@@ -146,30 +145,13 @@ class MetaStoreDirectSql {
       }
     }
 
-    isAggregateStatsCacheEnabled = HiveConf.getBoolVar(conf, ConfVars.METASTORE_AGGREGATE_STATS_CACHE_ENABLED);
+    isAggregateStatsCacheEnabled = HiveConf.getBoolVar(
+        conf, ConfVars.METASTORE_AGGREGATE_STATS_CACHE_ENABLED);
     if (isAggregateStatsCacheEnabled) {
       aggrStatsCache = AggregateStatsCache.getInstance(conf);
     }
   }
 
-  private DB determineDbType() {
-    DB dbType = DB.OTHER;
-    String productName = getProductName();
-    if (productName != null) {
-      productName = productName.toLowerCase();
-      if (productName.contains("mysql")) {
-        dbType = DB.MYSQL;
-      } else if (productName.contains("oracle")) {
-        dbType = DB.ORACLE;
-      } else if (productName.contains("microsoft sql server")) {
-        dbType = DB.MSSQL;
-      } else if (productName.contains("derby")) {
-        dbType = DB.DERBY;
-      }
-    }
-    return dbType;
-  }
-
   private String getProductName() {
     JDOConnection jdoConn = pm.getDataStoreConnection();
     try {
@@ -391,7 +373,7 @@ class MetaStoreDirectSql {
   public boolean generateSqlFilterForPushdown(
       Table table, ExpressionTree tree, SqlFilterForPushdown result) throws MetaException {
     // Derby and Oracle do not interpret filters ANSI-properly in some cases and need a workaround.
-    boolean dbHasJoinCastBug = (dbType == DB.DERBY || dbType == DB.ORACLE);
+    boolean dbHasJoinCastBug = DatabaseProduct.hasJoinOperationOrderBug(dbType);
     result.table = table;
     result.filter = PartitionFilterGenerator.generateSqlFilter(
         table, tree, result.params, result.joins, dbHasJoinCastBug, defaultPartName, dbType);
@@ -960,10 +942,10 @@ class MetaStoreDirectSql {
     private final List<String> joins;
     private final boolean dbHasJoinCastBug;
     private final String defaultPartName;
-    private final DB dbType;
+    private final DatabaseProduct dbType;
 
     private PartitionFilterGenerator(Table table, List<Object> params, List<String> joins,
-        boolean dbHasJoinCastBug, String defaultPartName, DB dbType) {
+        boolean dbHasJoinCastBug, String defaultPartName, DatabaseProduct dbType) {
       this.table = table;
       this.params = params;
       this.joins = joins;
@@ -981,8 +963,8 @@ class MetaStoreDirectSql {
      * @return the string representation of the expression tree
      */
     private static String generateSqlFilter(Table table, ExpressionTree tree, List<Object> params,
-        List<String> joins, boolean dbHasJoinCastBug, String defaultPartName, DB dbType)
-            throws MetaException {
+        List<String> joins, boolean dbHasJoinCastBug, String defaultPartName,
+        DatabaseProduct dbType) throws MetaException {
       assert table != null;
       if (tree == null) {
         // consistent with other APIs like makeExpressionTree, null is returned to indicate that
@@ -1129,7 +1111,7 @@ class MetaStoreDirectSql {
         if (colType == FilterType.Integral) {
           tableValue = "cast(" + tableValue + " as decimal(21,0))";
         } else if (colType == FilterType.Date) {
-          if (dbType == DB.ORACLE) {
+          if (dbType == DatabaseProduct.ORACLE) {
             // Oracle requires special treatment... as usual.
             tableValue = "TO_DATE(" + tableValue + ", 'YYYY-MM-DD')";
           } else {
@@ -1756,7 +1738,7 @@ class MetaStoreDirectSql {
    * effect will apply to the connection that is executing the queries otherwise.
    */
   public void prepareTxn() throws MetaException {
-    if (dbType != DB.MYSQL) return;
+    if (dbType != DatabaseProduct.MYSQL) return;
     try {
       assert pm.currentTransaction().isActive(); // must be inside tx together with queries
       executeNoResult("SET @@session.sql_mode=ANSI_QUOTES");

http://git-wip-us.apache.org/repos/asf/hive/blob/8cec20d9/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index a7a1cf9..fe16ebd 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -27,6 +27,7 @@ import org.apache.commons.lang.NotImplementedException;
 import org.apache.hadoop.hive.common.ServerUtils;
 import org.apache.hadoop.hive.common.classification.InterfaceAudience;
 import org.apache.hadoop.hive.common.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
 import org.apache.hadoop.hive.metastore.HouseKeeperService;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.slf4j.Logger;
@@ -1875,12 +1876,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
       if(dbProduct == null) {
         throw new IllegalStateException("DB Type not determined yet.");
       }
-      if (e instanceof SQLTransactionRollbackException ||
-        ((dbProduct == DatabaseProduct.MYSQL || dbProduct == DatabaseProduct.POSTGRES ||
-          dbProduct == DatabaseProduct.SQLSERVER) && e.getSQLState().equals("40001")) ||
-        (dbProduct == DatabaseProduct.POSTGRES && e.getSQLState().equals("40P01")) ||
-        (dbProduct == DatabaseProduct.ORACLE && (e.getMessage().contains("deadlock detected")
-          || e.getMessage().contains("can't serialize access for this transaction")))) {
+      if (DatabaseProduct.isDeadlock(dbProduct, e)) {
         if (deadlockCnt++ < ALLOWED_REPEATED_DEADLOCKS) {
           long waitInterval = deadlockRetryInterval * deadlockCnt;
           LOG.warn("Deadlock detected in " + caller + ". Will wait " + waitInterval +
@@ -1985,44 +1981,22 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
     return identifierQuoteString;
   }
 
-  protected enum DatabaseProduct { DERBY, MYSQL, POSTGRES, ORACLE, SQLSERVER}
 
-  /**
-   * Determine the database product type
-   * @param conn database connection
-   * @return database product type
-   */
-  private DatabaseProduct determineDatabaseProduct(Connection conn) {
-    if (dbProduct == null) {
-      try {
-        String s = conn.getMetaData().getDatabaseProductName();
-        if (s == null) {
-          String msg = "getDatabaseProductName returns null, can't determine database product";
-          LOG.error(msg);
-          throw new IllegalStateException(msg);
-        } else if (s.equals("Apache Derby")) {
-          dbProduct = DatabaseProduct.DERBY;
-        } else if (s.equals("Microsoft SQL Server")) {
-          dbProduct = DatabaseProduct.SQLSERVER;
-        } else if (s.equals("MySQL")) {
-          dbProduct = DatabaseProduct.MYSQL;
-        } else if (s.equals("Oracle")) {
-          dbProduct = DatabaseProduct.ORACLE;
-        } else if (s.equals("PostgreSQL")) {
-          dbProduct = DatabaseProduct.POSTGRES;
-        } else {
-          String msg = "Unrecognized database product name <" + s + ">";
-          LOG.error(msg);
-          throw new IllegalStateException(msg);
-        }
-
-      } catch (SQLException e) {
-        String msg = "Unable to get database product name: " + e.getMessage();
+  private void determineDatabaseProduct(Connection conn) {
+    if (dbProduct != null) return;
+    try {
+      String s = conn.getMetaData().getDatabaseProductName();
+      dbProduct = DatabaseProduct.determineDatabaseProduct(s);
+      if (dbProduct == DatabaseProduct.OTHER) {
+        String msg = "Unrecognized database product name <" + s + ">";
         LOG.error(msg);
         throw new IllegalStateException(msg);
       }
+    } catch (SQLException e) {
+      String msg = "Unable to get database product name";
+      LOG.error(msg, e);
+      throw new IllegalStateException(msg, e);
     }
-    return dbProduct;
   }
 
   private static class LockInfo {

http://git-wip-us.apache.org/repos/asf/hive/blob/8cec20d9/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnUtils.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnUtils.java b/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnUtils.java
index ebcbaff..8fada2c 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnUtils.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnUtils.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hive.metastore.txn;
 
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -118,7 +119,7 @@ public class TestTxnUtils {
   public void testSQLGenerator() throws Exception {
     //teseted on Oracle Database 11g Express Edition Release 11.2.0.2.0 - 64bit Production
     TxnHandler.SQLGenerator sqlGenerator =
-      new TxnHandler.SQLGenerator(TxnHandler.DatabaseProduct.ORACLE, conf);
+      new TxnHandler.SQLGenerator(DatabaseProduct.ORACLE, conf);
     List<String> rows = new ArrayList<>();
     rows.add("'yellow', 1");
     List<String> sql = sqlGenerator.createInsertValuesStmt("colors(name, category)", rows);
@@ -140,7 +141,7 @@ public class TestTxnUtils {
     Assert.assertEquals("Wrong stmt", "insert all into colors(name, category) values('G',997) into colors(name, category) values('G',998) into colors(name, category) values('G',999) select * from dual", sql.get(1));
     
     sqlGenerator =
-      new TxnHandler.SQLGenerator(TxnHandler.DatabaseProduct.MYSQL, conf);
+      new TxnHandler.SQLGenerator(DatabaseProduct.MYSQL, conf);
     rows.clear();
     rows.add("'yellow', 1");
     sql = sqlGenerator.createInsertValuesStmt("colors(name, category)", rows);
@@ -159,7 +160,7 @@ public class TestTxnUtils {
     Assert.assertEquals("Wrong stmt", "insert into colors(name, category) values('yellow', 1),('red', 2),('orange', 3),('G',0),('G',1),('G',2),('G',3),('G',4),('G',5),('G',6),('G',7),('G',8),('G',9),('G',10),('G',11),('G',12),('G',13),('G',14),('G',15),('G',16),('G',17),('G',18),('G',19),('G',20),('G',21),('G',22),('G',23),('G',24),('G',25),('G',26),('G',27),('G',28),('G',29),('G',30),('G',31),('G',32),('G',33),('G',34),('G',35),('G',36),('G',37),('G',38),('G',39),('G',40),('G',41),('G',42),('G',43),('G',44),('G',45),('G',46),('G',47),('G',48),('G',49),('G',50),('G',51),('G',52),('G',53),('G',54),('G',55),('G',56),('G',57),('G',58),('G',59),('G',60),('G',61),('G',62),('G',63),('G',64),('G',65),('G',66),('G',67),('G',68),('G',69),('G',70),('G',71),('G',72),('G',73),('G',74),('G',75),('G',76),('G',77),('G',78),('G',79),('G',80),('G',81),('G',82),('G',83),('G',84),('G',85),('G',86),('G',87),('G',88),('G',89),('G',90),('G',91),('G',92),('G',93),('G',94),('G',95),('G',96),('G',97),('G',9
 8),('G',99),('G',100),('G',101),('G',102),('G',103),('G',104),('G',105),('G',106),('G',107),('G',108),('G',109),('G',110),('G',111),('G',112),('G',113),('G',114),('G',115),('G',116),('G',117),('G',118),('G',119),('G',120),('G',121),('G',122),('G',123),('G',124),('G',125),('G',126),('G',127),('G',128),('G',129),('G',130),('G',131),('G',132),('G',133),('G',134),('G',135),('G',136),('G',137),('G',138),('G',139),('G',140),('G',141),('G',142),('G',143),('G',144),('G',145),('G',146),('G',147),('G',148),('G',149),('G',150),('G',151),('G',152),('G',153),('G',154),('G',155),('G',156),('G',157),('G',158),('G',159),('G',160),('G',161),('G',162),('G',163),('G',164),('G',165),('G',166),('G',167),('G',168),('G',169),('G',170),('G',171),('G',172),('G',173),('G',174),('G',175),('G',176),('G',177),('G',178),('G',179),('G',180),('G',181),('G',182),('G',183),('G',184),('G',185),('G',186),('G',187),('G',188),('G',189),('G',190),('G',191),('G',192),('G',193),('G',194),('G',195),('G',196),('G',197),('G',
 198),('G',199),('G',200),('G',201),('G',202),('G',203),('G',204),('G',205),('G',206),('G',207),('G',208),('G',209),('G',210),('G',211),('G',212),('G',213),('G',214),('G',215),('G',216),('G',217),('G',218),('G',219),('G',220),('G',221),('G',222),('G',223),('G',224),('G',225),('G',226),('G',227),('G',228),('G',229),('G',230),('G',231),('G',232),('G',233),('G',234),('G',235),('G',236),('G',237),('G',238),('G',239),('G',240),('G',241),('G',242),('G',243),('G',244),('G',245),('G',246),('G',247),('G',248),('G',249),('G',250),('G',251),('G',252),('G',253),('G',254),('G',255),('G',256),('G',257),('G',258),('G',259),('G',260),('G',261),('G',262),('G',263),('G',264),('G',265),('G',266),('G',267),('G',268),('G',269),('G',270),('G',271),('G',272),('G',273),('G',274),('G',275),('G',276),('G',277),('G',278),('G',279),('G',280),('G',281),('G',282),('G',283),('G',284),('G',285),('G',286),('G',287),('G',288),('G',289),('G',290),('G',291),('G',292),('G',293),('G',294),('G',295),('G',296),('G',297),('
 G',298),('G',299),('G',300),('G',301),('G',302),('G',303),('G',304),('G',305),('G',306),('G',307),('G',308),('G',309),('G',310),('G',311),('G',312),('G',313),('G',314),('G',315),('G',316),('G',317),('G',318),('G',319),('G',320),('G',321),('G',322),('G',323),('G',324),('G',325),('G',326),('G',327),('G',328),('G',329),('G',330),('G',331),('G',332),('G',333),('G',334),('G',335),('G',336),('G',337),('G',338),('G',339),('G',340),('G',341),('G',342),('G',343),('G',344),('G',345),('G',346),('G',347),('G',348),('G',349),('G',350),('G',351),('G',352),('G',353),('G',354),('G',355),('G',356),('G',357),('G',358),('G',359),('G',360),('G',361),('G',362),('G',363),('G',364),('G',365),('G',366),('G',367),('G',368),('G',369),('G',370),('G',371),('G',372),('G',373),('G',374),('G',375),('G',376),('G',377),('G',378),('G',379),('G',380),('G',381),('G',382),('G',383),('G',384),('G',385),('G',386),('G',387),('G',388),('G',389),('G',390),('G',391),('G',392),('G',393),('G',394),('G',395),('G',396),('G',397)
 ,('G',398),('G',399),('G',400),('G',401),('G',402),('G',403),('G',404),('G',405),('G',406),('G',407),('G',408),('G',409),('G',410),('G',411),('G',412),('G',413),('G',414),('G',415),('G',416),('G',417),('G',418),('G',419),('G',420),('G',421),('G',422),('G',423),('G',424),('G',425),('G',426),('G',427),('G',428),('G',429),('G',430),('G',431),('G',432),('G',433),('G',434),('G',435),('G',436),('G',437),('G',438),('G',439),('G',440),('G',441),('G',442),('G',443),('G',444),('G',445),('G',446),('G',447),('G',448),('G',449),('G',450),('G',451),('G',452),('G',453),('G',454),('G',455),('G',456),('G',457),('G',458),('G',459),('G',460),('G',461),('G',462),('G',463),('G',464),('G',465),('G',466),('G',467),('G',468),('G',469),('G',470),('G',471),('G',472),('G',473),('G',474),('G',475),('G',476),('G',477),('G',478),('G',479),('G',480),('G',481),('G',482),('G',483),('G',484),('G',485),('G',486),('G',487),('G',488),('G',489),('G',490),('G',491),('G',492),('G',493),('G',494),('G',495),('G',496),('G',4
 97),('G',498),('G',499),('G',500),('G',501),('G',502),('G',503),('G',504),('G',505),('G',506),('G',507),('G',508),('G',509),('G',510),('G',511),('G',512),('G',513),('G',514),('G',515),('G',516),('G',517),('G',518),('G',519),('G',520),('G',521),('G',522),('G',523),('G',524),('G',525),('G',526),('G',527),('G',528),('G',529),('G',530),('G',531),('G',532),('G',533),('G',534),('G',535),('G',536),('G',537),('G',538),('G',539),('G',540),('G',541),('G',542),('G',543),('G',544),('G',545),('G',546),('G',547),('G',548),('G',549),('G',550),('G',551),('G',552),('G',553),('G',554),('G',555),('G',556),('G',557),('G',558),('G',559),('G',560),('G',561),('G',562),('G',563),('G',564),('G',565),('G',566),('G',567),('G',568),('G',569),('G',570),('G',571),('G',572),('G',573),('G',574),('G',575),('G',576),('G',577),('G',578),('G',579),('G',580),('G',581),('G',582),('G',583),('G',584),('G',585),('G',586),('G',587),('G',588),('G',589),('G',590),('G',591),('G',592),('G',593),('G',594),('G',595),('G',596),('G
 ',597),('G',598),('G',599),('G',600),('G',601),('G',602),('G',603),('G',604),('G',605),('G',606),('G',607),('G',608),('G',609),('G',610),('G',611),('G',612),('G',613),('G',614),('G',615),('G',616),('G',617),('G',618),('G',619),('G',620),('G',621),('G',622),('G',623),('G',624),('G',625),('G',626),('G',627),('G',628),('G',629),('G',630),('G',631),('G',632),('G',633),('G',634),('G',635),('G',636),('G',637),('G',638),('G',639),('G',640),('G',641),('G',642),('G',643),('G',644),('G',645),('G',646),('G',647),('G',648),('G',649),('G',650),('G',651),('G',652),('G',653),('G',654),('G',655),('G',656),('G',657),('G',658),('G',659),('G',660),('G',661),('G',662),('G',663),('G',664),('G',665),('G',666),('G',667),('G',668),('G',669),('G',670),('G',671),('G',672),('G',673),('G',674),('G',675),('G',676),('G',677),('G',678),('G',679),('G',680),('G',681),('G',682),('G',683),('G',684),('G',685),('G',686),('G',687),('G',688),('G',689),('G',690),('G',691),('G',692),('G',693),('G',694),('G',695),('G',696),
 ('G',697),('G',698),('G',699),('G',700),('G',701),('G',702),('G',703),('G',704),('G',705),('G',706),('G',707),('G',708),('G',709),('G',710),('G',711),('G',712),('G',713),('G',714),('G',715),('G',716),('G',717),('G',718),('G',719),('G',720),('G',721),('G',722),('G',723),('G',724),('G',725),('G',726),('G',727),('G',728),('G',729),('G',730),('G',731),('G',732),('G',733),('G',734),('G',735),('G',736),('G',737),('G',738),('G',739),('G',740),('G',741),('G',742),('G',743),('G',744),('G',745),('G',746),('G',747),('G',748),('G',749),('G',750),('G',751),('G',752),('G',753),('G',754),('G',755),('G',756),('G',757),('G',758),('G',759),('G',760),('G',761),('G',762),('G',763),('G',764),('G',765),('G',766),('G',767),('G',768),('G',769),('G',770),('G',771),('G',772),('G',773),('G',774),('G',775),('G',776),('G',777),('G',778),('G',779),('G',780),('G',781),('G',782),('G',783),('G',784),('G',785),('G',786),('G',787),('G',788),('G',789),('G',790),('G',791),('G',792),('G',793),('G',794),('G',795),('G',79
 6),('G',797),('G',798),('G',799),('G',800),('G',801),('G',802),('G',803),('G',804),('G',805),('G',806),('G',807),('G',808),('G',809),('G',810),('G',811),('G',812),('G',813),('G',814),('G',815),('G',816),('G',817),('G',818),('G',819),('G',820),('G',821),('G',822),('G',823),('G',824),('G',825),('G',826),('G',827),('G',828),('G',829),('G',830),('G',831),('G',832),('G',833),('G',834),('G',835),('G',836),('G',837),('G',838),('G',839),('G',840),('G',841),('G',842),('G',843),('G',844),('G',845),('G',846),('G',847),('G',848),('G',849),('G',850),('G',851),('G',852),('G',853),('G',854),('G',855),('G',856),('G',857),('G',858),('G',859),('G',860),('G',861),('G',862),('G',863),('G',864),('G',865),('G',866),('G',867),('G',868),('G',869),('G',870),('G',871),('G',872),('G',873),('G',874),('G',875),('G',876),('G',877),('G',878),('G',879),('G',880),('G',881),('G',882),('G',883),('G',884),('G',885),('G',886),('G',887),('G',888),('G',889),('G',890),('G',891),('G',892),('G',893),('G',894),('G',895),('G'
 ,896),('G',897),('G',898),('G',899),('G',900),('G',901),('G',902),('G',903),('G',904),('G',905),('G',906),('G',907),('G',908),('G',909),('G',910),('G',911),('G',912),('G',913),('G',914),('G',915),('G',916),('G',917),('G',918),('G',919),('G',920),('G',921),('G',922),('G',923),('G',924),('G',925),('G',926),('G',927),('G',928),('G',929),('G',930),('G',931),('G',932),('G',933),('G',934),('G',935),('G',936),('G',937),('G',938),('G',939),('G',940),('G',941),('G',942),('G',943),('G',944),('G',945),('G',946),('G',947),('G',948),('G',949),('G',950),('G',951),('G',952),('G',953),('G',954),('G',955),('G',956),('G',957),('G',958),('G',959),('G',960),('G',961),('G',962),('G',963),('G',964),('G',965),('G',966),('G',967),('G',968),('G',969),('G',970),('G',971),('G',972),('G',973),('G',974),('G',975),('G',976),('G',977),('G',978),('G',979),('G',980),('G',981),('G',982),('G',983),('G',984),('G',985),('G',986),('G',987),('G',988),('G',989),('G',990),('G',991),('G',992),('G',993),('G',994),('G',995),(
 'G',996)", sql.get(0));
     Assert.assertEquals("Wrong stmt", "insert into colors(name, category) values('G',997),('G',998),('G',999)", sql.get(1));
 
-    sqlGenerator = new TxnHandler.SQLGenerator(TxnHandler.DatabaseProduct.SQLSERVER, conf);
+    sqlGenerator = new TxnHandler.SQLGenerator(DatabaseProduct.SQLSERVER, conf);
     String modSql = sqlGenerator.addForUpdateClause("select nl_next from NEXT_LOCK_ID");
     Assert.assertEquals("select nl_next from NEXT_LOCK_ID with (updlock)", modSql);
     modSql = sqlGenerator.addForUpdateClause("select MT_COMMENT from AUX_TABLE where MT_KEY1='CheckLock' and MT_KEY2=0");