You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by sp...@apache.org on 2016/09/28 21:18:07 UTC

[1/4] hive git commit: HIVE-13836: DbNotifications giving an error = Invalid state. Transaction has already started (Nachiket Vaidya, reviewd by Sushanth Sowmyan)

Repository: hive
Updated Branches:
  refs/heads/branch-2.1 e045c5a57 -> 878b0e7de


HIVE-13836: DbNotifications giving an error = Invalid state. Transaction has already started (Nachiket Vaidya, reviewd by Sushanth Sowmyan)


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

Branch: refs/heads/branch-2.1
Commit: 2c95d368c5410981959db823557e8a538f9005d1
Parents: e045c5a
Author: Sergio Pena <se...@cloudera.com>
Authored: Wed Sep 28 16:05:20 2016 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Wed Sep 28 16:05:20 2016 -0500

----------------------------------------------------------------------
 .../hive/hcatalog/listener/DbNotificationListener.java    | 10 ++++++++--
 1 file changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2c95d368/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
index 7245b9e..0b3d891 100644
--- a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
@@ -61,6 +61,8 @@ public class DbNotificationListener extends MetaStoreEventListener {
   private static final Logger LOG = LoggerFactory.getLogger(DbNotificationListener.class.getName());
   private static CleanerThread cleaner = null;
 
+  private static final Object NOTIFICATION_TBL_LOCK = new Object();
+
   // This is the same object as super.conf, but it's convenient to keep a copy of it as a
   // HiveConf rather than a Configuration.
   private HiveConf hiveConf;
@@ -252,7 +254,9 @@ public class DbNotificationListener extends MetaStoreEventListener {
 
   private void enqueue(NotificationEvent event) {
     if (rs != null) {
-      rs.addNotificationEvent(event);
+      synchronized(NOTIFICATION_TBL_LOCK) {
+        rs.addNotificationEvent(event);
+      }
     } else {
       LOG.warn("Dropping event " + event + " since notification is not running.");
     }
@@ -274,7 +278,9 @@ public class DbNotificationListener extends MetaStoreEventListener {
     @Override
     public void run() {
       while (true) {
-        rs.cleanNotificationEvents(ttl);
+        synchronized(NOTIFICATION_TBL_LOCK) {
+          rs.cleanNotificationEvents(ttl);
+        }
         LOG.debug("Cleaner thread done");
         try {
           Thread.sleep(sleepTime);


[4/4] hive git commit: HIVE-14137: Hive on Spark throws FileAlreadyExistsException for jobs with multiple empty tables (Sahil Takiar, reviewed by Sergio Pena)

Posted by sp...@apache.org.
HIVE-14137: Hive on Spark throws FileAlreadyExistsException for jobs with multiple empty tables (Sahil Takiar, reviewed by Sergio Pena)


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

Branch: refs/heads/branch-2.1
Commit: 878b0e7de1743059ea70807858a8ce01c360cb18
Parents: a8e101b
Author: Sergio Pena <se...@cloudera.com>
Authored: Wed Sep 28 16:08:40 2016 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Wed Sep 28 16:08:40 2016 -0500

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/exec/Utilities.java   | 18 +++--
 .../hadoop/hive/ql/exec/TestUtilities.java      | 75 ++++++++++++++++++++
 2 files changed, 83 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/878b0e7d/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
index 8f7bbb2..4556301 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
@@ -2944,7 +2944,6 @@ public final class Utilities {
    */
   public static List<Path> getInputPaths(JobConf job, MapWork work, Path hiveScratchDir,
       Context ctx, boolean skipDummy) throws Exception {
-    int sequenceNumber = 0;
 
     Set<Path> pathsProcessed = new HashSet<Path>();
     List<Path> pathsToAdd = new LinkedList<Path>();
@@ -2971,7 +2970,7 @@ public final class Utilities {
           if (!skipDummy
               && isEmptyPath(job, path, ctx)) {
             path = createDummyFileForEmptyPartition(path, job, work,
-                 hiveScratchDir, alias, sequenceNumber++);
+                 hiveScratchDir);
 
           }
           pathsToAdd.add(path);
@@ -2987,8 +2986,7 @@ public final class Utilities {
       // If T is empty and T2 contains 100 rows, the user expects: 0, 100 (2
       // rows)
       if (path == null && !skipDummy) {
-        path = createDummyFileForEmptyTable(job, work, hiveScratchDir,
-            alias, sequenceNumber++);
+        path = createDummyFileForEmptyTable(job, work, hiveScratchDir, alias);
         pathsToAdd.add(path);
       }
     }
@@ -2998,11 +2996,11 @@ public final class Utilities {
   @SuppressWarnings({"rawtypes", "unchecked"})
   private static Path createEmptyFile(Path hiveScratchDir,
       HiveOutputFormat outFileFormat, JobConf job,
-      int sequenceNumber, Properties props, boolean dummyRow)
+      Properties props, boolean dummyRow)
           throws IOException, InstantiationException, IllegalAccessException {
 
     // create a dummy empty file in a new directory
-    String newDir = hiveScratchDir + Path.SEPARATOR + sequenceNumber;
+    String newDir = hiveScratchDir + Path.SEPARATOR + UUID.randomUUID().toString();
     Path newPath = new Path(newDir);
     FileSystem fs = newPath.getFileSystem(job);
     fs.mkdirs(newPath);
@@ -3028,7 +3026,7 @@ public final class Utilities {
 
   @SuppressWarnings("rawtypes")
   private static Path createDummyFileForEmptyPartition(Path path, JobConf job, MapWork work,
-      Path hiveScratchDir, String alias, int sequenceNumber)
+      Path hiveScratchDir)
           throws Exception {
 
     String strPath = path.toString();
@@ -3047,7 +3045,7 @@ public final class Utilities {
     boolean oneRow = partDesc.getInputFileFormatClass() == OneNullRowInputFormat.class;
 
     Path newPath = createEmptyFile(hiveScratchDir, outFileFormat, job,
-        sequenceNumber, props, oneRow);
+        props, oneRow);
 
     if (LOG.isInfoEnabled()) {
       LOG.info("Changed input file " + strPath + " to empty file " + newPath);
@@ -3072,7 +3070,7 @@ public final class Utilities {
 
   @SuppressWarnings("rawtypes")
   private static Path createDummyFileForEmptyTable(JobConf job, MapWork work,
-      Path hiveScratchDir, String alias, int sequenceNumber)
+      Path hiveScratchDir, String alias)
           throws Exception {
 
     TableDesc tableDesc = work.getAliasToPartnInfo().get(alias).getTableDesc();
@@ -3085,7 +3083,7 @@ public final class Utilities {
     HiveOutputFormat outFileFormat = HiveFileFormatUtils.getHiveOutputFormat(job, tableDesc);
 
     Path newPath = createEmptyFile(hiveScratchDir, outFileFormat, job,
-        sequenceNumber, props, false);
+        props, false);
 
     if (LOG.isInfoEnabled()) {
       LOG.info("Changed input file for alias " + alias + " to " + newPath);

http://git-wip-us.apache.org/repos/asf/hive/blob/878b0e7d/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java
index d2060a1..adde59f 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.exec;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 import static org.apache.hadoop.hive.ql.exec.Utilities.getFileExtension;
+import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -30,13 +31,19 @@ import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
+import java.util.LinkedHashMap;
+import java.util.Properties;
+import java.util.UUID;
 
 import org.apache.commons.io.FileUtils;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat;
+import org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx;
@@ -44,17 +51,25 @@ import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
+import org.apache.hadoop.hive.ql.plan.MapWork;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+import org.apache.hadoop.hive.ql.plan.PartitionDesc;
+import org.apache.hadoop.hive.ql.plan.TableDesc;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFFromUtcTimestamp;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.mapred.JobConf;
+
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import com.google.common.io.Files;
 
@@ -246,4 +261,64 @@ public class TestUtilities {
     FileSystem.getLocal(hconf).create(taskOutputPath).close();
     return tempDirPath;
   }
+
+  /**
+   * Check that calling {@link Utilities#getInputPaths(JobConf, MapWork, Path, Context, boolean)}
+   * can process two different empty tables without throwing any exceptions.
+   */
+  @Test
+  public void testGetInputPathsWithEmptyTables() throws Exception {
+    String alias1Name = "alias1";
+    String alias2Name = "alias2";
+
+    MapWork mapWork1 = new MapWork();
+    MapWork mapWork2 = new MapWork();
+    JobConf jobConf = new JobConf();
+
+    String nonExistentPath1 = UUID.randomUUID().toString();
+    String nonExistentPath2 = UUID.randomUUID().toString();
+
+    PartitionDesc mockPartitionDesc = mock(PartitionDesc.class);
+    TableDesc mockTableDesc = mock(TableDesc.class);
+
+    when(mockTableDesc.isNonNative()).thenReturn(false);
+    when(mockTableDesc.getProperties()).thenReturn(new Properties());
+
+    when(mockPartitionDesc.getProperties()).thenReturn(new Properties());
+    when(mockPartitionDesc.getTableDesc()).thenReturn(mockTableDesc);
+    doReturn(HiveSequenceFileOutputFormat.class).when(
+            mockPartitionDesc).getOutputFileFormatClass();
+
+    mapWork1.setPathToAliases(new LinkedHashMap<>(
+            ImmutableMap.of(nonExistentPath1, Lists.newArrayList(alias1Name))));
+    mapWork1.setAliasToWork(new LinkedHashMap<String, Operator<? extends OperatorDesc>>(
+            ImmutableMap.of(alias1Name, (Operator<?>) mock(Operator.class))));
+    mapWork1.setPathToPartitionInfo(new LinkedHashMap<>(
+            ImmutableMap.of(nonExistentPath1, mockPartitionDesc)));
+
+    mapWork2.setPathToAliases(new LinkedHashMap<>(
+            ImmutableMap.of(nonExistentPath2, Lists.newArrayList(alias2Name))));
+    mapWork2.setAliasToWork(new LinkedHashMap<String, Operator<? extends OperatorDesc>>(
+            ImmutableMap.of(alias2Name, (Operator<?>) mock(Operator.class))));
+    mapWork2.setPathToPartitionInfo(new LinkedHashMap<>(
+            ImmutableMap.of(nonExistentPath2, mockPartitionDesc)));
+
+    List<Path> inputPaths = new ArrayList<>();
+    try {
+      Path scratchDir = new Path(HiveConf.getVar(jobConf, HiveConf.ConfVars.LOCALSCRATCHDIR));
+      inputPaths.addAll(Utilities.getInputPaths(jobConf, mapWork1, scratchDir,
+              mock(Context.class), false));
+      inputPaths.addAll(Utilities.getInputPaths(jobConf, mapWork2, scratchDir,
+              mock(Context.class), false));
+      assertEquals(inputPaths.size(), 2);
+    } finally {
+      File file;
+      for (Path path : inputPaths) {
+        file = new File(path.toString());
+        if (file.exists()) {
+          file.delete();
+        }
+      }
+    }
+  }
 }


[3/4] hive git commit: HIVE-14538: beeline throws exceptions with parsing hive config when using !sh statement (Yongzhi Chen, reviewed by Chaoyu Tang)

Posted by sp...@apache.org.
HIVE-14538: beeline throws exceptions with parsing hive config when using !sh statement (Yongzhi Chen, reviewed by Chaoyu Tang)


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

Branch: refs/heads/branch-2.1
Commit: a8e101bbda99b92b8d62a3bbb8d6d56bf00bcce0
Parents: fc902ff
Author: Sergio Pena <se...@cloudera.com>
Authored: Wed Sep 28 16:07:59 2016 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Wed Sep 28 16:07:59 2016 -0500

----------------------------------------------------------------------
 .../java/org/apache/hive/beeline/Commands.java  | 36 +++++++++++++-------
 .../hive/beeline/TestBeeLineWithArgs.java       |  8 +++++
 2 files changed, 31 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/a8e101bb/beeline/src/java/org/apache/hive/beeline/Commands.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/Commands.java b/beeline/src/java/org/apache/hive/beeline/Commands.java
index 3a204c0..7160788 100644
--- a/beeline/src/java/org/apache/hive/beeline/Commands.java
+++ b/beeline/src/java/org/apache/hive/beeline/Commands.java
@@ -742,10 +742,12 @@ public class Commands {
   private Map<String, String> getHiveVariables() {
     Map<String, String> result = new HashMap<>();
     BufferedRows rows = getConfInternal(true);
-    while (rows.hasNext()) {
-      Rows.Row row = (Rows.Row) rows.next();
-      if (!row.isMeta) {
-        result.put(row.values[0], row.values[1]);
+    if (rows != null) {
+      while (rows.hasNext()) {
+        Rows.Row row = (Rows.Row) rows.next();
+        if (!row.isMeta) {
+          result.put(row.values[0], row.values[1]);
+        }
       }
     }
     return result;
@@ -784,13 +786,19 @@ public class Commands {
     Statement stmnt = null;
     BufferedRows rows = null;
     try {
-      boolean hasResults;
-      if (call) {
-        stmnt = beeLine.getDatabaseConnection().getConnection().prepareCall("set");
-        hasResults = ((CallableStatement) stmnt).execute();
-      } else {
-        stmnt = beeLine.createStatement();
-        hasResults = stmnt.execute("set");
+      boolean hasResults = false;
+      DatabaseConnection dbconn = beeLine.getDatabaseConnection();
+      Connection conn = null;
+      if (dbconn != null)
+        conn = dbconn.getConnection();
+      if (conn != null) {
+        if (call) {
+          stmnt = conn.prepareCall("set");
+          hasResults = ((CallableStatement) stmnt).execute();
+        } else {
+          stmnt = beeLine.createStatement();
+          hasResults = stmnt.execute("set");
+        }
       }
       if (hasResults) {
         ResultSet rs = stmnt.getResultSet();
@@ -823,7 +831,8 @@ public class Commands {
       return;
     } else {
       String[] kv = val.split("=", 2);
-      hiveConf.set(kv[0], kv[1]);
+      if (kv.length == 2)
+        hiveConf.set(kv[0], kv[1]);
     }
   }
 
@@ -1088,7 +1097,8 @@ public class Commands {
     }
 
     line = line.substring("sh".length()).trim();
-    line = substituteVariables(getHiveConf(false), line.trim());
+    if (!beeLine.isBeeLine())
+      line = substituteVariables(getHiveConf(false), line.trim());
 
     try {
       ShellCmdExecutor executor = new ShellCmdExecutor(line, beeLine.getOutputStream(),

http://git-wip-us.apache.org/repos/asf/hive/blob/a8e101bb/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
index ae68f62..9159957 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
@@ -864,4 +864,12 @@ public class TestBeeLineWithArgs {
 
     testScriptFile( SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
   }
+
+  @Test
+  public void testBeelineShellCommandWithoutConn() throws Throwable {
+    List<String> argList = new ArrayList<String>();
+    final String SCRIPT_TEXT = "!sh echo hello world";
+    final String EXPECTED_PATTERN = "hello world";
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList,true,false);
+  }
 }


[2/4] hive git commit: HIVE-14296: Session count is not decremented when HS2 clients do not shutdown cleanly. (Naveen Gangam, reviewed by Szehon Ho and Mohit Sabharwal)

Posted by sp...@apache.org.
HIVE-14296: Session count is not decremented when HS2 clients do not shutdown cleanly. (Naveen Gangam, reviewed by Szehon Ho and Mohit Sabharwal)


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

Branch: refs/heads/branch-2.1
Commit: fc902ffe6c57ea1126bf34400de3a38e1e60b9df
Parents: 2c95d36
Author: Sergio Pena <se...@cloudera.com>
Authored: Wed Sep 28 16:06:59 2016 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Wed Sep 28 16:06:59 2016 -0500

----------------------------------------------------------------------
 .../java/org/apache/hive/service/cli/session/SessionManager.java | 4 +++-
 .../org/apache/hive/service/cli/thrift/ThriftCLIService.java     | 3 ---
 2 files changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/fc902ffe/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/session/SessionManager.java b/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
index c2313c0..7bcf3a3 100644
--- a/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
+++ b/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
@@ -351,14 +351,16 @@ public class SessionManager extends CompositeService {
       throw new HiveSQLException("Failed to execute session hooks: " + e.getMessage(), e);
     }
     handleToSession.put(session.getSessionHandle(), session);
+    LOG.info("Session opened, " + session.getSessionHandle() + ", current sessions:" + getOpenSessionCount());
     return session;
   }
 
-  public void closeSession(SessionHandle sessionHandle) throws HiveSQLException {
+  public synchronized void closeSession(SessionHandle sessionHandle) throws HiveSQLException {
     HiveSession session = handleToSession.remove(sessionHandle);
     if (session == null) {
       throw new HiveSQLException("Session does not exist!");
     }
+    LOG.info("Session closed, " + sessionHandle + ", current sessions:" + getOpenSessionCount());
     try {
       session.close();
     } finally {

http://git-wip-us.apache.org/repos/asf/hive/blob/fc902ffe/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
index 24f1a61..e097b79 100644
--- a/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
+++ b/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
@@ -117,7 +117,6 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
   protected CLIService cliService;
   private static final TStatus OK_STATUS = new TStatus(TStatusCode.SUCCESS_STATUS);
   protected static HiveAuthFactory hiveAuthFactory;
-  private static final AtomicInteger sessionCount = new AtomicInteger();
 
   protected int portNum;
   protected InetAddress serverIPAddress;
@@ -320,7 +319,6 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
       if (context != null) {
         context.setSessionHandle(sessionHandle);
       }
-      LOG.info("Opened a session, current sessions: " + sessionCount.incrementAndGet());
     } catch (Exception e) {
       LOG.warn("Error opening session: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
@@ -466,7 +464,6 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
     try {
       SessionHandle sessionHandle = new SessionHandle(req.getSessionHandle());
       cliService.closeSession(sessionHandle);
-      LOG.info("Closed a session, current sessions: " + sessionCount.decrementAndGet());
       resp.setStatus(OK_STATUS);
       ThriftCLIServerContext context =
         (ThriftCLIServerContext)currentServerContext.get();