You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by bb...@apache.org on 2021/01/12 02:38:00 UTC

[geode-native] branch develop updated: GEODE-8806: Refactor Logger code (#718)

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

bbender pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode-native.git


The following commit(s) were added to refs/heads/develop by this push:
     new 29afcad  GEODE-8806: Refactor Logger code (#718)
29afcad is described below

commit 29afcad1466e787a65bbcdf0391b7294cb43ac1e
Author: Blake Bender <bb...@pivotal.io>
AuthorDate: Mon Jan 11 18:37:50 2021 -0800

    GEODE-8806: Refactor Logger code (#718)
    
    Replace all ACE usage by boost or std equivalents
    Fix log rolling code to properly honor file and disk space limits
    Move existing logging test from integration to unit tests, and add new tests
    Clean up global logger variables, and stop allocating/deallocating
    Remove a bunch or redundant code
    
    Co-authored-by: Blake Bender <bb...@vmware.com>
    Co-authored-by: Matthew Reddington <mr...@vmware.com>
---
 cppcache/benchmark/GeodeLoggingBM.cpp              |   6 +-
 cppcache/integration-test/CMakeLists.txt           |   1 -
 cppcache/integration-test/fw_helper.hpp            |   5 +-
 cppcache/integration-test/testLogger.cpp           | 371 --------
 cppcache/integration/test/AuthInitializeTest.cpp   |   3 +-
 cppcache/integration/test/CacheXmlTest.cpp         |   5 +-
 .../integration/test/CqPlusAuthInitializeTest.cpp  |   3 +-
 .../integration/test/FunctionExecutionTest.cpp     |   2 +-
 .../integration/test/PdxInstanceFactoryTest.cpp    |   2 +-
 cppcache/integration/test/RegionPutAllTest.cpp     |   2 +-
 cppcache/integration/test/SslTwoWayTest.cpp        |   3 +-
 cppcache/integration/test/TransactionsTest.cpp     |   2 +-
 .../integration/test/WanDeserializationTest.cpp    |   2 +-
 cppcache/src/CacheXmlParser.cpp                    |   2 +-
 cppcache/src/CqService.cpp                         |  34 +-
 cppcache/src/LocalRegion.cpp                       |   4 +-
 cppcache/src/Log.cpp                               | 998 +++++----------------
 cppcache/src/TcrEndpoint.cpp                       |   4 +-
 cppcache/src/ThinClientDistributionManager.cpp     |   2 +-
 cppcache/src/statistics/GeodeStatisticsFactory.cpp |   7 +-
 cppcache/src/statistics/StatisticsManager.cpp      |  10 +-
 cppcache/src/util/Log.hpp                          | 362 ++------
 cppcache/test/CMakeLists.txt                       |   1 +
 cppcache/test/LoggingTest.cpp                      | 610 +++++++++++++
 24 files changed, 956 insertions(+), 1485 deletions(-)

diff --git a/cppcache/benchmark/GeodeLoggingBM.cpp b/cppcache/benchmark/GeodeLoggingBM.cpp
index 3dd3ff7..19347b0 100644
--- a/cppcache/benchmark/GeodeLoggingBM.cpp
+++ b/cppcache/benchmark/GeodeLoggingBM.cpp
@@ -59,14 +59,14 @@ void GeodeLogStrings(benchmark::State& state) {
   int index = g_iteration++ % STRING_ARRAY_LENGTH;
 
   for (auto _ : state) {
-    Log::debug(logStrings[index]);
+    LOGDEBUG(logStrings[index]);
   }
 }
 
 void GeodeLogInts(benchmark::State& state) {
   std::string intString(std::to_string(g_iteration++));
   for (auto _ : state) {
-    Log::debug(intString.c_str());
+    LOGDEBUG(intString.c_str());
   }
 }
 
@@ -75,7 +75,7 @@ void GeodeLogCombo(benchmark::State& state) {
   std::string comboString = std::string(logStrings[g_iteration % 3]) + " " +
                             std::to_string(g_iteration);
   for (auto _ : state) {
-    Log::debug(comboString.c_str());
+    LOGDEBUG(comboString.c_str());
   }
 }
 
diff --git a/cppcache/integration-test/CMakeLists.txt b/cppcache/integration-test/CMakeLists.txt
index 7a40f75..10ce0b2 100644
--- a/cppcache/integration-test/CMakeLists.txt
+++ b/cppcache/integration-test/CMakeLists.txt
@@ -177,7 +177,6 @@ include_directories(${CMAKE_SOURCE_DIR}/tests/cpp)
 set_tests_properties(testDataOutput
     testFWHelper
     testSystemProperties
-    testLogger
     testLinkage
     testRegionTemplateArgs
     testRegionMap
diff --git a/cppcache/integration-test/fw_helper.hpp b/cppcache/integration-test/fw_helper.hpp
index 11d0ace..1c12762 100644
--- a/cppcache/integration-test/fw_helper.hpp
+++ b/cppcache/integration-test/fw_helper.hpp
@@ -131,10 +131,7 @@ class TestException {
         m_filename(const_cast<char*>(filename)) {}
 
   void print() {
-    char buf[256];
-    apache::geode::client::Log::formatLogLine(
-        buf, apache::geode::client::LogLevel::Error);
-    fprintf(stdout, "--->%sTestException: %s in %s at line %d<---\n", buf,
+    fprintf(stdout, "--->%sTestException: %s in %s at line %d<---\n", apache::geode::client::Log::formatLogLine(apache::geode::client::LogLevel::Error).c_str(),
             m_message.c_str(), m_filename, m_lineno);
   }
   std::string m_message;
diff --git a/cppcache/integration-test/testLogger.cpp b/cppcache/integration-test/testLogger.cpp
deleted file mode 100644
index 5645db4..0000000
--- a/cppcache/integration-test/testLogger.cpp
+++ /dev/null
@@ -1,371 +0,0 @@
-/*
- * 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.
- */
-
-#include "fw_helper.hpp"
-#include <geode/ExceptionTypes.hpp>
-
-#ifndef WIN32
-#include <unistd.h>
-#endif
-
-#define LENGTH_OF_BANNER 16
-
-using apache::geode::client::IllegalStateException;
-using apache::geode::client::Log;
-using apache::geode::client::LogFn;
-using apache::geode::client::LogLevel;
-
-int numOfLinesInFile(const char *fname) {
-  char line[2048];
-  char *read;
-  int ln_cnt = 0;
-  FILE *fp = fopen(fname, "r");
-  if (fp == nullptr) {
-    return -1;
-  }
-  while (!!(read = fgets(line, sizeof line, fp))) {
-    printf("%d:%s", ++ln_cnt, line);
-  }
-
-  if (!feof(fp)) {
-    fclose(fp);
-    return -2;
-  }
-  fclose(fp);
-  return ln_cnt;
-}
-
-void testLogFnError() {
-  LogFn logFn("TestLogger::testLogFnError", LogLevel::Error);
-  Log::error("...");
-}
-
-void testLogFnWarning() {
-  LogFn logFn("TestLogger::testLogFnWarning", LogLevel::Warning);
-  Log::warning("...");
-}
-
-void testLogFnInfo() {
-  LogFn logFn("TestLogger::testLogFnInfo", LogLevel::Info);
-  Log::info("...");
-}
-
-void testLogFnConfig() {
-  LogFn logFn("TestLogger::testLogFnConfig", LogLevel::Config);
-  Log::config("...");
-}
-
-void testLogFnFine() {
-  LogFn logFn("TestLogger::testLogFnFine", LogLevel::Fine);
-  Log::fine("...");
-}
-
-void testLogFnFiner() {
-  LogFn logFn("TestLogger::testLogFnFiner", LogLevel::Finer);
-  Log::finer("...");
-}
-
-void testLogFnFinest() {
-  LogFn logFn("TestLogger::testLogFnFinest");
-  Log::finest("...");
-}
-
-void testLogFnDebug() {
-  LogFn logFn("TestLogger::testLogFnDebug", LogLevel::Debug);
-  Log::debug("...");
-}
-
-int expected(LogLevel level) {
-  int expected = static_cast<int>(level);
-  if (level != LogLevel::None) {
-    expected += LENGTH_OF_BANNER;
-  }
-  if (level >= LogLevel::Default) {
-    expected--;
-  }
-  return expected;
-}
-
-BEGIN_TEST(REINIT)
-  {
-    LOGINFO("Started logging");
-    int exceptiongot = 0;
-    Log::init(LogLevel::Debug, "logfile");
-    try {
-      Log::init(LogLevel::Debug, "logfile1");
-    } catch (IllegalStateException &ex) {
-      printf("Got Illegal state exception while calling init again\n");
-      printf("Exception mesage = %s\n", ex.what());
-      exceptiongot = 1;
-    }
-    ASSERT(exceptiongot == 1, "expected exceptiongot to be 1");
-    Log::close();
-    unlink("logfile.log");
-  }
-END_TEST(REINIT)
-
-BEGIN_TEST(ALL_LEVEL)
-  {
-    for (LogLevel level : {
-             LogLevel::Error,
-             LogLevel::Warning,
-             LogLevel::Info,
-             LogLevel::Default,
-             LogLevel::Config,
-             LogLevel::Fine,
-             LogLevel::Finer,
-             LogLevel::Finest,
-             LogLevel::Debug,
-         }) {
-      Log::init(level, "all_logfile");
-
-      Log::error("Error Message");
-      Log::warning("Warning Message");
-      Log::info("Info Message");
-      Log::config("Config Message");
-      Log::fine("Fine Message");
-      Log::finer("Finer Message");
-      Log::finest("Finest Message");
-      Log::debug("Debug Message");
-
-      int lines = numOfLinesInFile("all_logfile.log");
-
-      printf("lines = %d expected = %d \n", lines, expected(level));
-
-      ASSERT(lines == expected(level), "Wrong number of lines");
-
-      Log::close();
-      unlink("all_logfile.log");
-    }
-  }
-END_TEST(ALL_LEVEL)
-
-BEGIN_TEST(ALL_LEVEL_MACRO)
-  {
-    for (LogLevel level : {
-             LogLevel::Error,
-             LogLevel::Warning,
-             LogLevel::Info,
-             LogLevel::Default,
-             LogLevel::Config,
-             LogLevel::Fine,
-             LogLevel::Finer,
-             LogLevel::Finest,
-             LogLevel::Debug,
-         }) {
-      Log::init(level, "all_logfile");
-
-      LOGERROR("Error Message");
-      LOGWARN("Warning Message");
-      LOGINFO("Info Message");
-      LOGCONFIG("Config Message");
-      LOGFINE("Fine Message");
-      LOGFINER("Finer Message");
-      LOGFINEST("Finest Message");
-      LOGDEBUG("Debug Message");
-
-      int lines = numOfLinesInFile("all_logfile.log");
-
-      printf("lines = %d Level = %d, %d\n", lines, static_cast<int>(level),
-             static_cast<int>(Log::logLevel()));
-
-      ASSERT(lines == expected(level), "Wrong number of lines");
-      Log::close();
-
-      unlink("all_logfile.log");
-    }
-  }
-END_TEST(ALL_LEVEL_MACRO)
-
-BEGIN_TEST(FILE_LIMIT)
-  {
-#ifdef _WIN32
-// Fail to roll file over to timestamp file on windows.
-#else
-    for (LogLevel level : {
-             LogLevel::Error,
-             LogLevel::Warning,
-             LogLevel::Info,
-             LogLevel::Default,
-             LogLevel::Config,
-             LogLevel::Fine,
-             LogLevel::Finer,
-             LogLevel::Finest,
-             LogLevel::Debug,
-         }) {
-      if (level == LogLevel::Default) continue;
-      Log::init(level, "logfile", 1);
-
-      Log::error("Error Message");
-      Log::warning("Warning Message");
-      Log::info("Info Message");
-      Log::config("Config Message");
-      Log::fine("Fine Message");
-      Log::finer("Finer Message");
-      Log::finest("Finest Message");
-      Log::debug("Debug Message");
-
-      int lines = numOfLinesInFile("logfile.log");
-      int expectedLines = static_cast<int>(level) + LENGTH_OF_BANNER -
-                          (level >= LogLevel::Default ? 1 : 0);
-      printf("lines = %d expectedLines = %d level = %d\n", lines, expectedLines,
-             static_cast<int>(level));
-
-      ASSERT(lines == expectedLines, "Wrong number of lines");
-
-      Log::close();
-      unlink("logfile.log");
-    }
-#endif
-  }
-END_TEST(FILE_LIMIT)
-
-BEGIN_TEST(CONFIG_ONWARDS)
-  {
-    Log::init(LogLevel::Config, "logfile");
-
-    Log::debug("Debug Message");
-    Log::config("Config Message");
-    Log::info("Info Message");
-    Log::warning("Warning Message");
-    Log::error("Error Message");
-
-    int lines = numOfLinesInFile("logfile.log");
-    printf("lines = %d\n", lines);
-    // debug should not be printed
-    ASSERT(lines == 4 + LENGTH_OF_BANNER,
-           "Expected 4 + LENGTH_OF_BANNER lines.");
-    Log::close();
-    unlink("logfile.log");
-  }
-END_TEST(CONFIG_ONWARDS)
-
-BEGIN_TEST(INFO_ONWARDS)
-  {
-    Log::init(LogLevel::Info, "logfile");
-
-    Log::debug("Debug Message");
-    Log::config("Config Message");
-    Log::info("Info Message");
-    Log::warning("Warning Message");
-    Log::error("Error Message");
-
-    int lines = numOfLinesInFile("logfile.log");
-    printf("lines = %d\n", lines);
-    // debug, config should not be printed
-    ASSERT(lines == 3 + LENGTH_OF_BANNER,
-           "Expected 3 + LENGTH_OF_BANNER lines.");
-    Log::close();
-    unlink("logfile.log");
-  }
-END_TEST(INFO_ONWARDS)
-
-BEGIN_TEST(WARNING_ONWARDS)
-  {
-    Log::init(LogLevel::Warning, "logfile");
-
-    Log::debug("Debug Message");
-    Log::config("Config Message");
-    Log::info("Info Message");
-    Log::warning("Warning Message");
-    Log::error("Error Message");
-
-    int lines = numOfLinesInFile("logfile.log");
-    printf("lines = %d\n", lines);
-    // debug, config, info should not be printed
-    ASSERT(lines == 2 + LENGTH_OF_BANNER,
-           "Expected 2 + LENGTH_OF_BANNER lines.");
-    Log::close();
-    unlink("logfile.log");
-  }
-END_TEST(WARNING_ONWARDS)
-
-BEGIN_TEST(ERROR_LEVEL)
-  {
-    Log::init(LogLevel::Error, "logfile");
-
-    Log::debug("Debug Message");
-    Log::config("Config Message");
-    Log::info("Info Message");
-    Log::warning("Warning Message");
-    Log::error("Error Message");
-
-    int lines = numOfLinesInFile("logfile.log");
-    printf("lines = %d\n", lines);
-    // debug, config, info and warning should not be printed
-    ASSERT(lines == 1 + LENGTH_OF_BANNER, "Expected 1+LENGTH_OF_BANNER lines.");
-    Log::close();
-    unlink("logfile.log");
-  }
-END_TEST(ERROR_LEVEL)
-
-BEGIN_TEST(NO_LOG)
-  {
-    Log::init(LogLevel::None, "logfile");
-
-    Log::debug("Debug Message");
-    Log::config("Config Message");
-    Log::info("Info Message");
-    Log::warning("Warning Message");
-    Log::error("Error Message");
-
-    int lines = numOfLinesInFile("logfile.log");
-    printf("lines = %d\n", lines);
-    // debug, config, info and warning and even error should not be printed
-    // As the logfile is not there so -1 will be returned.
-    ASSERT(lines == -1 || lines == 0, "Expected 0 or -1 lines.");
-    Log::close();
-    unlink("logfile.log");
-  }
-END_TEST(NO_LOG)
-
-BEGIN_TEST(LOGFN)
-  {
-    for (LogLevel level : {
-             LogLevel::Error,
-             LogLevel::Warning,
-             LogLevel::Info,
-             LogLevel::Default,
-             LogLevel::Config,
-             LogLevel::Fine,
-             LogLevel::Finer,
-             LogLevel::Finest,
-             LogLevel::Debug,
-         }) {
-      Log::init(level, "logfile");
-
-      testLogFnError();
-      testLogFnWarning();
-      testLogFnInfo();
-      testLogFnConfig();
-      testLogFnFine();
-      testLogFnFiner();
-      testLogFnFinest();
-      testLogFnDebug();
-
-      int lines = numOfLinesInFile("logfile.log");
-
-      printf("lines = %d, level = %s\n", lines, Log::levelToChars(level));
-
-      ASSERT(lines == 3 * expected(level) - 2 * LENGTH_OF_BANNER,
-             "Wrong number of lines");
-      Log::close();
-
-      unlink("logfile.log");
-    }
-  }
-END_TEST(LOGFN)
diff --git a/cppcache/integration/test/AuthInitializeTest.cpp b/cppcache/integration/test/AuthInitializeTest.cpp
index 7967571..5526557 100644
--- a/cppcache/integration/test/AuthInitializeTest.cpp
+++ b/cppcache/integration/test/AuthInitializeTest.cpp
@@ -68,8 +68,7 @@ const int32_t CQ_PLUS_AUTH_TEST_REGION_ENTRY_COUNT = 100000;
 
 Cache createCache(std::shared_ptr<SimpleAuthInitialize> auth) {
   auto cache = CacheFactory()
-                   .set("log-level", "debug")
-                   .set("log-file", "geode_native.log")
+                   .set("log-level", "none")
                    .set("statistic-sampling-enabled", "false")
                    .setAuthInitialize(auth)
                    .create();
diff --git a/cppcache/integration/test/CacheXmlTest.cpp b/cppcache/integration/test/CacheXmlTest.cpp
index c70c9f9..99c95a2 100644
--- a/cppcache/integration/test/CacheXmlTest.cpp
+++ b/cppcache/integration/test/CacheXmlTest.cpp
@@ -44,10 +44,9 @@ apache::geode::client::Cache createCacheUsingXmlConfig(
 
   CacheFactory cacheFactory;
 
-  auto cache = cacheFactory.set("log-level", "debug")
-                   .set("log-file", "geode_native.log")
+  auto cache = cacheFactory.set("log-level", "none")
                    .set("statistic-sampling-enabled", "false")
-                   .set("cache-xml-file", xmlFile.c_str())
+                   .set("cache-xml-file", xmlFile)
                    .create();
 
   return cache;
diff --git a/cppcache/integration/test/CqPlusAuthInitializeTest.cpp b/cppcache/integration/test/CqPlusAuthInitializeTest.cpp
index f29b038..b54971d 100644
--- a/cppcache/integration/test/CqPlusAuthInitializeTest.cpp
+++ b/cppcache/integration/test/CqPlusAuthInitializeTest.cpp
@@ -64,8 +64,7 @@ const int32_t CQ_PLUS_AUTH_TEST_REGION_ENTRY_COUNT = 50000;
 
 Cache createCache(std::shared_ptr<SimpleAuthInitialize> auth) {
   auto cache = CacheFactory()
-                   .set("log-level", "debug")
-                   .set("log-file", "geode_native.log")
+                   .set("log-level", "none")
                    .set("statistic-sampling-enabled", "false")
                    .setAuthInitialize(auth)
                    .create();
diff --git a/cppcache/integration/test/FunctionExecutionTest.cpp b/cppcache/integration/test/FunctionExecutionTest.cpp
index f1cc760..73d8c45 100644
--- a/cppcache/integration/test/FunctionExecutionTest.cpp
+++ b/cppcache/integration/test/FunctionExecutionTest.cpp
@@ -301,7 +301,7 @@ TEST(FunctionExecutionTest, OnServersOneServerGoesDown) {
         .execute();
   });
 
-  auto cache = CacheFactory().set("log-level", "debug").create();
+  auto cache = CacheFactory().set("log-level", "none").create();
   auto poolFactory = cache.getPoolManager().createFactory();
 
   cluster.applyLocators(poolFactory);
diff --git a/cppcache/integration/test/PdxInstanceFactoryTest.cpp b/cppcache/integration/test/PdxInstanceFactoryTest.cpp
index 348b96a..bd3168f 100644
--- a/cppcache/integration/test/PdxInstanceFactoryTest.cpp
+++ b/cppcache/integration/test/PdxInstanceFactoryTest.cpp
@@ -41,7 +41,7 @@ const std::string regionName = "my_region";
 
 std::shared_ptr<Cache> createCache() {
   auto cache = CacheFactory()
-                   .set("log-level", "debug")
+                   .set("log-level", "none")
                    .setPdxReadSerialized(true)
                    .create();
   return std::make_shared<Cache>(std::move(cache));
diff --git a/cppcache/integration/test/RegionPutAllTest.cpp b/cppcache/integration/test/RegionPutAllTest.cpp
index c487b4e..7fd7804 100644
--- a/cppcache/integration/test/RegionPutAllTest.cpp
+++ b/cppcache/integration/test/RegionPutAllTest.cpp
@@ -50,7 +50,7 @@ Cache createCache() {
   using apache::geode::client::CacheFactory;
 
   auto cache = CacheFactory()
-                   .set("log-level", "debug")
+                   .set("log-level", "none")
                    .set("statistic-sampling-enabled", "false")
                    .create();
 
diff --git a/cppcache/integration/test/SslTwoWayTest.cpp b/cppcache/integration/test/SslTwoWayTest.cpp
index 877d3ae..eec0122 100644
--- a/cppcache/integration/test/SslTwoWayTest.cpp
+++ b/cppcache/integration/test/SslTwoWayTest.cpp
@@ -84,8 +84,7 @@ TEST_F(SslTwoWayTest, PutGetWithValidSslConfiguration) {
       (clientSslKeysDir /
        boost::filesystem::path("client_truststore_chained_root.pem"));
   auto cache = CacheFactory()
-                   .set("log-level", "debug")
-                   .set("log-file", "./gemfire.log")
+                   .set("log-level", "none")
                    .set("ssl-enabled", "true")
                    .set("ssl-keystore", clientKeystore.string())
                    .set("ssl-keystore-password", certificatePassword)
diff --git a/cppcache/integration/test/TransactionsTest.cpp b/cppcache/integration/test/TransactionsTest.cpp
index f0f5b64..eacaa04 100644
--- a/cppcache/integration/test/TransactionsTest.cpp
+++ b/cppcache/integration/test/TransactionsTest.cpp
@@ -37,7 +37,7 @@ using apache::geode::client::Region;
 using apache::geode::client::RegionShortcut;
 
 std::shared_ptr<Cache> createCache() {
-  auto cache = CacheFactory().set("log-level", "debug").create();
+  auto cache = CacheFactory().set("log-level", "none").create();
   return std::make_shared<Cache>(std::move(cache));
 }
 
diff --git a/cppcache/integration/test/WanDeserializationTest.cpp b/cppcache/integration/test/WanDeserializationTest.cpp
index c34e3ca..0b41d29 100644
--- a/cppcache/integration/test/WanDeserializationTest.cpp
+++ b/cppcache/integration/test/WanDeserializationTest.cpp
@@ -67,7 +67,7 @@ Cache createCache(std::string durableClientId) {
   using apache::geode::client::CacheFactory;
 
   auto cache = CacheFactory()
-                   .set("log-level", "debug")
+                   .set("log-level", "none")
                    .set("statistic-sampling-enabled", "false")
                    .setPdxReadSerialized(true)
                    .set("durable-client-id", durableClientId)
diff --git a/cppcache/src/CacheXmlParser.cpp b/cppcache/src/CacheXmlParser.cpp
index 47c2382..32c4121 100644
--- a/cppcache/src/CacheXmlParser.cpp
+++ b/cppcache/src/CacheXmlParser.cpp
@@ -505,7 +505,7 @@ void CacheXmlParser::create(Cache *cache) {
   }
   cacheCreation_->create(cache);
   delCacheCreation.noDelete();
-  Log::info("Declarative configuration of cache completed successfully");
+  LOGINFO("Declarative configuration of cache completed successfully");
 }
 
 std::string CacheXmlParser::getOptionalAttribute(
diff --git a/cppcache/src/CqService.cpp b/cppcache/src/CqService.cpp
index 57403e9..974a728 100644
--- a/cppcache/src/CqService.cpp
+++ b/cppcache/src/CqService.cpp
@@ -174,7 +174,7 @@ std::shared_ptr<CqQuery> CqService::getCq(const std::string& cqName) {
  * Clears the CQ Query Map.
  */
 void CqService::clearCqQueryMap() {
-  Log::fine("Cleaning clearCqQueryMap.");
+  LOGFINE("Cleaning clearCqQueryMap.");
   m_cqQueryMap.clear();
 }
 
@@ -287,13 +287,13 @@ void CqService::stopCqs(query_container_type& cqs) {
         cqName = cq->getName();
         cq->stop();
       } catch (QueryException& qe) {
-        Log::fine(("Failed to stop the CQ, CqName : " + cqName +
-                   " Error : " + qe.what())
-                      .c_str());
+        LOGFINE(("Failed to stop the CQ, CqName : " + cqName +
+                 " Error : " + qe.what())
+                    .c_str());
       } catch (CqClosedException& cce) {
-        Log::fine(("Failed to stop the CQ, CqName : " + cqName +
-                   " Error : " + cce.what())
-                      .c_str());
+        LOGFINE(("Failed to stop the CQ, CqName : " + cqName +
+                 " Error : " + cce.what())
+                    .c_str());
       }
     }
   }
@@ -316,13 +316,13 @@ void CqService::closeCqs(query_container_type& cqs) {
           cqi->close(false);
         }
       } catch (QueryException& qe) {
-        Log::fine(("Failed to close the CQ, CqName : " + cqName +
-                   " Error : " + qe.what())
-                      .c_str());
+        LOGFINE(("Failed to close the CQ, CqName : " + cqName +
+                 " Error : " + qe.what())
+                    .c_str());
       } catch (CqClosedException& cce) {
-        Log::fine(("Failed to close the CQ, CqName : " + cqName +
-                   " Error : " + cce.what())
-                      .c_str());
+        LOGFINE(("Failed to close the CQ, CqName : " + cqName +
+                 " Error : " + cce.what())
+                    .c_str());
       }
     }
   }
@@ -349,11 +349,11 @@ void CqService::closeCqService() {
   }
 }
 void CqService::closeAllCqs() {
-  Log::fine("closeAllCqs()");
+  LOGFINE("closeAllCqs()");
   query_container_type cqVec = getAllCqs();
-  Log::fine("closeAllCqs() 1");
+  LOGFINE("closeAllCqs() 1");
   auto&& lock = m_cqQueryMap.make_lock();
-  Log::fine("closeAllCqs() 2");
+  LOGFINE("closeAllCqs() 2");
   closeCqs(cqVec);
 }
 
@@ -361,7 +361,7 @@ void CqService::closeAllCqs() {
  * Cleans up the CqService.
  */
 void CqService::cleanup() {
-  Log::fine("Cleaning up CqService.");
+  LOGFINE("Cleaning up CqService.");
 
   // Close All the CQs.
   // Need to take care when Clients are still connected...
diff --git a/cppcache/src/LocalRegion.cpp b/cppcache/src/LocalRegion.cpp
index 8ce50d8..6daf5bb 100644
--- a/cppcache/src/LocalRegion.cpp
+++ b/cppcache/src/LocalRegion.cpp
@@ -717,7 +717,7 @@ void LocalRegion::registerEntryExpiryTask(
       new EntryExpiryHandler(rptr, entry, getEntryExpirationAction(), duration);
   auto id = rptr->getCacheImpl()->getExpiryTaskManager().scheduleExpiryTask(
       handler, duration, std::chrono::seconds::zero());
-  if (Log::finestEnabled()) {
+  if (Log::enabled(LogLevel::Finest)) {
     std::shared_ptr<CacheableKey> key;
     entry->getKeyI(key);
     LOGFINEST(
@@ -2812,7 +2812,7 @@ void LocalRegion::updateAccessAndModifiedTimeForEntry(
     ExpEntryProperties& expProps = ptr->getExpProperties();
     auto currTime = std::chrono::system_clock::now();
     std::string keyStr;
-    if (Log::debugEnabled()) {
+    if (Log::enabled(LogLevel::Debug)) {
       std::shared_ptr<CacheableKey> key;
       ptr->getKeyI(key);
       keyStr = Utils::nullSafeToString(key);
diff --git a/cppcache/src/Log.cpp b/cppcache/src/Log.cpp
index 7b8c2ad..9e7c15a 100644
--- a/cppcache/src/Log.cpp
+++ b/cppcache/src/Log.cpp
@@ -22,52 +22,29 @@
 #include <chrono>
 #include <cinttypes>
 #include <ctime>
+#include <map>
 #include <mutex>
+#include <regex>
+#include <sstream>
 #include <string>
 #include <thread>
 #include <utility>
 #include <vector>
 
-#include <ace/ACE.h>
-#include <ace/Dirent_Selector.h>
+#include <boost/asio/ip/host_name.hpp>
+#include <boost/filesystem.hpp>
 #include <boost/process/environment.hpp>
 
 #include <geode/ExceptionTypes.hpp>
 #include <geode/internal/geode_globals.hpp>
 #include <geode/util/LogLevel.hpp>
 
-#include "../internal/hacks/AceThreadId.h"
 #include "geodeBanner.hpp"
 #include "util/chrono/time_point.hpp"
 
-#if defined(_WIN32)
-#include <io.h>
-#define GF_FILEEXISTS(x) _access_s(x, 00)
-#else
-#include <unistd.h>
-#define GF_FILEEXISTS(x) access(x, F_OK)
-#endif
-
-/*****************************************************************************/
-
-/**
- * The implementation of the Log class
- *
- *
- */
-
-/*****************************************************************************/
-
-namespace apache {
-namespace geode {
-namespace log {
-namespace globals {
-
-static std::string* g_logFile = nullptr;
-static std::string* g_logFileWithExt = nullptr;
+namespace {
 
 static size_t g_bytesWritten = 0;
-static bool g_isLogFileOpened = false;
 
 static size_t g_fileSizeLimit = GEODE_MAX_LOG_FILE_LIMIT;
 static size_t g_diskSpaceLimit = GEODE_MAX_LOG_DISK_LIMIT;
@@ -76,74 +53,18 @@ static std::mutex g_logMutex;
 
 static int g_rollIndex = 0;
 static size_t g_spaceUsed = 0;
-// Make a pair for the filename & its size
-static std::pair<std::string, int64_t> g_fileInfoPair;
-// Vector to hold the fileInformation
-typedef std::vector<std::pair<std::string, int64_t> > g_fileInfo;
+
+static boost::filesystem::path g_fullpath;
+static std::map<int32_t, boost::filesystem::path> g_rollFiles;
 
 static FILE* g_log = nullptr;
-static ACE_utsname g_uname;
-static pid_t g_pid = 0;
 
-}  // namespace globals
-}  // namespace log
-}  // namespace geode
-}  // namespace apache
+static std::string g_hostName;
 
-extern "C" {
-
-static int selector(const dirent* d) {
-  std::string inputname(d->d_name);
-  std::string filebasename =
-      ACE::basename(apache::geode::log::globals::g_logFileWithExt->c_str());
-  size_t actualHyphenPos = filebasename.find_last_of('.');
-  if (strcmp(filebasename.c_str(), d->d_name) == 0) return 1;
-  size_t fileExtPos = inputname.find_last_of('.');
-  std::string extName = inputname.substr(fileExtPos + 1, inputname.length());
-  if (strcmp(extName.c_str(), "log") != 0) return 0;
-  if (fileExtPos != std::string::npos) {
-    std::string tempname = inputname.substr(0, fileExtPos);
-    size_t fileHyphenPos = tempname.find_last_of('-');
-    if (fileHyphenPos != std::string::npos) {
-      std::string buff1 = tempname.substr(0, fileHyphenPos);
-      if (strstr(filebasename.c_str(), buff1.c_str()) == nullptr) {
-        return 0;
-      }
-      if (fileHyphenPos != actualHyphenPos) return 0;
-      std::string buff = tempname.substr(fileHyphenPos + 1,
-                                         tempname.length() - fileHyphenPos - 1);
-      for (std::string::iterator iter = buff.begin(); iter != buff.end();
-           ++iter) {
-        if (*iter < '0' || *iter > '9') {
-          return 0;
-        }
-      }
-      return 1;
-    } else {
-      return 0;
-    }
-  } else {
-    return 0;
-  }
-}
+const int __1K__ = 1024;
+const int __1M__ = (__1K__ * __1K__);
 
-static int comparator(const dirent** d1, const dirent** d2) {
-  if (strlen((*d1)->d_name) < strlen((*d2)->d_name)) {
-    return -1;
-  } else if (strlen((*d1)->d_name) > strlen((*d2)->d_name)) {
-    return 1;
-  }
-
-  int diff = std::strcmp((*d1)->d_name, (*d2)->d_name);
-  if (diff < 0) {
-    return -1;
-  } else if (diff > 0) {
-    return 1;
-  } else {
-    return 0;
-  }
-}
-}
+}  // namespace
 
 namespace apache {
 namespace geode {
@@ -151,21 +72,6 @@ namespace client {
 
 LogLevel Log::s_logLevel = LogLevel::Default;
 
-using apache::geode::log::globals::g_bytesWritten;
-using apache::geode::log::globals::g_diskSpaceLimit;
-using apache::geode::log::globals::g_fileInfo;
-using apache::geode::log::globals::g_fileInfoPair;
-using apache::geode::log::globals::g_fileSizeLimit;
-using apache::geode::log::globals::g_isLogFileOpened;
-using apache::geode::log::globals::g_log;
-using apache::geode::log::globals::g_logFile;
-using apache::geode::log::globals::g_logFileWithExt;
-using apache::geode::log::globals::g_logMutex;
-using apache::geode::log::globals::g_pid;
-using apache::geode::log::globals::g_rollIndex;
-using apache::geode::log::globals::g_spaceUsed;
-using apache::geode::log::globals::g_uname;
-
 /*****************************************************************************/
 
 LogLevel Log::logLevel() { return s_logLevel; }
@@ -175,8 +81,129 @@ LogLevel Log::logLevel() { return s_logLevel; }
  */
 void Log::setLogLevel(LogLevel level) { s_logLevel = level; }
 
+void Log::validateSizeLimits(int64_t fileSizeLimit, int64_t diskSpaceLimit) {
+  if (fileSizeLimit * __1M__ > GEODE_MAX_LOG_FILE_LIMIT) {
+    throw IllegalArgumentException(
+        "Specified file size limit larger than max allowed (1GB)");
+  } else if (fileSizeLimit < 0) {
+    throw IllegalArgumentException("Specified file size limit must be >= 0");
+  }
+
+  if (diskSpaceLimit * __1M__ > GEODE_MAX_LOG_DISK_LIMIT) {
+    throw IllegalArgumentException(
+        "Specified disk space limit larger than max allowed (1TB)");
+  } else if (diskSpaceLimit < 0) {
+    throw IllegalArgumentException("Specified disk space limit must be >= 0");
+  }
+
+  if (fileSizeLimit > diskSpaceLimit && diskSpaceLimit != 0) {
+    throw IllegalArgumentException(
+        "Disk space limit must be larger than file size limit");
+  }
+}
+
+void Log::validateLogFileName(const std::string& filename) {
+  auto nameToCheck = boost::filesystem::path(filename).filename().string();
+  if (!boost::filesystem::portable_file_name(nameToCheck)) {
+    throw IllegalArgumentException("Specified log file (" + nameToCheck +
+                                   ") is not a valid portable name.");
+  }
+}
+
 void Log::init(LogLevel level, const char* logFileName, int32_t logFileLimit,
                int64_t logDiskSpaceLimit) {
+  auto logFileNameString =
+      logFileName ? std::string(logFileName) : std::string();
+  init(level, std::string(logFileNameString), logFileLimit, logDiskSpaceLimit);
+}
+
+void Log::rollLogFile() {
+  if (g_log) {
+    fclose(g_log);
+    g_log = nullptr;
+  }
+
+  auto rollFileName =
+      (g_fullpath.parent_path() /
+       (g_fullpath.stem().string() + "-" + std::to_string(g_rollIndex) +
+        g_fullpath.extension().string()))
+          .string();
+  try {
+    auto rollFile = boost::filesystem::path(rollFileName);
+    boost::filesystem::rename(g_fullpath, rollFile);
+    g_rollFiles[g_rollIndex] = rollFile;
+    g_rollIndex++;
+  } catch (const boost::filesystem::filesystem_error&) {
+    throw IllegalStateException("Failed to roll log file");
+  }
+}
+
+void Log::removeOldestRolledLogFile() {
+  if (g_rollFiles.size()) {
+    auto index = g_rollFiles.begin()->first;
+    auto fileToRemove = g_rollFiles.begin()->second;
+    auto fileSize = boost::filesystem::file_size(fileToRemove);
+    boost::filesystem::remove(fileToRemove);
+    g_rollFiles.erase(index);
+    g_spaceUsed -= fileSize;
+  }
+}
+
+void Log::buildRollFileMapping() {
+  const auto filterstring = g_fullpath.stem().string() + "-(\\d+)\\.log$";
+  const std::regex my_filter(filterstring);
+
+  g_rollFiles.clear();
+
+  boost::filesystem::directory_iterator end_itr;
+  for (boost::filesystem::directory_iterator i(
+           g_fullpath.parent_path().string());
+       i != end_itr; ++i) {
+    if (boost::filesystem::is_regular_file(i->status())) {
+      std::string filename = i->path().filename().string();
+      std::regex testPattern(filterstring);
+      std::match_results<std::string::const_iterator> testMatches;
+      if (std::regex_search(std::string::const_iterator(filename.begin()),
+                            filename.cend(), testMatches, testPattern)) {
+        auto index = std::atoi(
+            std::string(testMatches[1].first, testMatches[1].second).c_str());
+        g_rollFiles[index] = i->path();
+      }
+    }
+  }
+}
+
+void Log::setRollFileIndex() {
+  g_rollIndex = 0;
+  if (g_rollFiles.size()) {
+    g_rollIndex = g_rollFiles.rbegin()->first + 1;
+  }
+}
+
+void Log::setSizeLimits(int32_t logFileLimit, int64_t logDiskSpaceLimit) {
+  validateSizeLimits(logFileLimit, logDiskSpaceLimit);
+
+  // Default to 10MB file limit and 1GB disk limit
+  if (logFileLimit == 0 && logDiskSpaceLimit == 0) {
+    g_fileSizeLimit = 10 * __1M__;
+    g_diskSpaceLimit = 1000 * __1M__;
+  }
+  // disk space specified but file size is defaulted.  Just use a single
+  // log file, i.e. set file limit == disk limit
+  else if (logFileLimit == 0) {
+    g_diskSpaceLimit = logDiskSpaceLimit * __1M__;
+    g_fileSizeLimit = g_diskSpaceLimit;
+  } else if (logDiskSpaceLimit == 0) {
+    g_fileSizeLimit = logFileLimit * __1M__;
+    g_diskSpaceLimit = g_fileSizeLimit;
+  } else {
+    g_fileSizeLimit = logFileLimit * __1M__;
+    g_diskSpaceLimit = logDiskSpaceLimit * __1M__;
+  }
+}
+
+void Log::init(LogLevel level, const std::string& logFileName,
+               int32_t logFileLimit, int64_t logDiskSpaceLimit) {
   if (g_log != nullptr) {
     throw IllegalStateException(
         "The Log has already been initialized. "
@@ -184,248 +211,75 @@ void Log::init(LogLevel level, const char* logFileName, int32_t logFileLimit,
   }
   s_logLevel = level;
 
-  if (logDiskSpaceLimit <
-      0 /*|| logDiskSpaceLimit > GEODE_MAX_LOG_DISK_LIMIT*/) {
-    logDiskSpaceLimit = GEODE_MAX_LOG_DISK_LIMIT;
-  }
-
-  if (logFileLimit < 0 || logFileLimit > GEODE_MAX_LOG_FILE_LIMIT) {
-    logFileLimit = GEODE_MAX_LOG_FILE_LIMIT;
-  }
-
   std::lock_guard<decltype(g_logMutex)> guard(g_logMutex);
 
-  if (logFileName && logFileName[0]) {
-    std::string filename = logFileName;
-    if (g_logFile) {
-      *g_logFile = filename;
-    } else {
-      g_logFile = new std::string(filename);
-    }
+  g_hostName = boost::asio::ip::host_name();
 
-#ifdef _WIN32
-    // replace all '\' with '/' to make everything easier..
-    std::replace(g_logFile->begin(), g_logFile->end(), '\\', '/');
-#endif
+  if (logFileName.length()) {
+    validateLogFileName(logFileName);
+    g_fullpath =
+        boost::filesystem::absolute(boost::filesystem::path(logFileName));
 
-    // Appending a ".log" at the end if it does not exist or file has some other
-    // extension.
-    std::string filebasename = ACE::basename(g_logFile->c_str());
-    auto len = static_cast<int32_t>(filebasename.length());
-    auto fileExtPos = filebasename.find_last_of('.', len);
     // if no extension then add .log extension
-    if (fileExtPos == std::string::npos) {
-      g_logFileWithExt = new std::string(*g_logFile + ".log");
-    } else {
-      std::string extName = filebasename.substr(fileExtPos + 1);
-      // if extension other than .log change it to ext + .log
-      if (extName != "log") {
-        g_logFileWithExt = new std::string(*g_logFile + ".log");
-      }
-      // .log Extension already provided, no need to append any extension.
-      else {
-        g_logFileWithExt = new std::string(*g_logFile);
-      }
-    }
-
-    g_fileSizeLimit = logFileLimit * 1024 * 1024;
-    g_diskSpaceLimit = logDiskSpaceLimit * 1024ll * 1024ll;
-
-    // If FileSizelimit is greater than DiskSpaceLimit & diskspaceLimit is set,
-    // then set DiskSpaceLimit to FileSizelimit
-    if (g_fileSizeLimit > g_diskSpaceLimit && g_diskSpaceLimit != 0) {
-      g_fileSizeLimit = g_diskSpaceLimit;
+    if (g_fullpath.extension().empty() || (g_fullpath.extension() != ".log")) {
+      g_fullpath = g_fullpath.string() + ".log";
     }
 
-    // If only DiskSpaceLimit is specified and no FileSizeLimit specified, then
-    // set DiskSpaceLimit to FileSizelimit.
-    // This helps in getting the file handle that is exceeded the limit.
-    if (g_fileSizeLimit == 0 && g_diskSpaceLimit != 0) {
-      g_fileSizeLimit = g_diskSpaceLimit;
-    }
+    setSizeLimits(logFileLimit, logDiskSpaceLimit);
 
     g_bytesWritten = 0;
     g_spaceUsed = 0;
-    g_rollIndex = 0;
-
-    std::string dirname = ACE::dirname(g_logFile->c_str());
-
-    ACE_Dirent_Selector sds;
-    int status = sds.open(dirname.c_str(), selector, comparator);
-    if (status != -1) {
-      for (int index = 0; index < sds.length(); ++index) {
-        std::string strname = ACE::basename(sds[index]->d_name);
-        fileExtPos = strname.find_last_of('.', strname.length());
-        if (fileExtPos != std::string::npos) {
-          std::string tempname = strname.substr(0, fileExtPos);
-          size_t fileHyphenPos = tempname.find_last_of('-', tempname.length());
-          if (fileHyphenPos != std::string::npos) {
-            std::string buff =
-                tempname.substr(fileHyphenPos + 1, tempname.length());
-            g_rollIndex = std::stoi(buff) + 1;
-          }
-        }  // if loop
-      }    // for loop
-    }
-    sds.close();
-
-    FILE* existingFile = fopen(g_logFileWithExt->c_str(), "r");
-    if (existingFile != nullptr && logFileLimit > 0) {
-      char rollFile[1024] = {0};
-      std::string logsdirname;
-      std::string logsbasename;
-      std::string fnameBeforeExt;
-      std::string extName;
-      std::string newfilestr;
-
-      len = static_cast<int32_t>(g_logFileWithExt->length());
-      int32_t lastPosOfSep = static_cast<int32_t>(
-          g_logFileWithExt->find_last_of(ACE_DIRECTORY_SEPARATOR_CHAR, len));
-      if (lastPosOfSep == -1) {
-        logsdirname = ".";
-      } else {
-        logsdirname = g_logFileWithExt->substr(0, lastPosOfSep);
-      }
-      logsbasename = g_logFileWithExt->substr(lastPosOfSep + 1, len);
-      char logFileExtAfter = '.';
-      int32_t baselen = static_cast<int32_t>(logsbasename.length());
-      int32_t posOfExt = static_cast<int32_t>(
-          logsbasename.find_last_of(logFileExtAfter, baselen));
-      if (posOfExt == -1) {
-        // throw IllegalArgument;
-      } else {
-        fnameBeforeExt = logsbasename.substr(0, posOfExt);
-        extName = logsbasename.substr(posOfExt + 1, baselen);
-      }
-      std::snprintf(rollFile, 1024, "%s%c%s-%d.%s", logsdirname.c_str(),
-                    ACE_DIRECTORY_SEPARATOR_CHAR, fnameBeforeExt.c_str(),
-                    g_rollIndex++, extName.c_str());
-      bool rollFileNameGot = false;
-      while (!rollFileNameGot) {
-        FILE* checkFile = fopen(rollFile, "r");
-        if (checkFile != nullptr) {
-          fclose(checkFile);
-          checkFile = nullptr;
-          std::snprintf(rollFile, 1024, "%s%c%s-%d.%s", logsdirname.c_str(),
-                        ACE_DIRECTORY_SEPARATOR_CHAR, fnameBeforeExt.c_str(),
-                        g_rollIndex++, extName.c_str());
-        } else {
-          rollFileNameGot = true;
-        }
-        /* adongre
-         * CID 28999: Use after free (USE_AFTER_FREE)
-         */
-        if (checkFile != nullptr) fclose(existingFile);
-      }
-      // retry some number of times before giving up when file is busy etc.
-      int renameResult = -1;
-      int maxTries = 10;
-      while (maxTries-- > 0) {
-        renameResult = ACE_OS::rename(g_logFileWithExt->c_str(), rollFile);
-        if (renameResult >= 0) {
-          break;
-        }
-        // continue after some sleep
-        std::this_thread::sleep_for(std::chrono::milliseconds(200));
-      }
-      /* (don't throw exception; try appending to existing file instead)
-      if (renameResult < 0) {
-        std::string msg = "Could not rename: " +
-          *g_logFileWithExt + " to: " + rollFile;
-        throw GeodeIOException(msg.c_str());
-      }
-      */
+
+    // Ensure that directory exists for log files.  We're going to attempt
+    // to iterate through files in that folder, and if it doesn't exist boost
+    // will throw an exception.
+    const auto target_path = g_fullpath.parent_path().string();
+    if (!boost::filesystem::exists(target_path)) {
+      boost::filesystem::create_directories(target_path);
     }
-    if (existingFile != nullptr) {
-      fclose(existingFile);
-      existingFile = nullptr;
+
+    buildRollFileMapping();
+    setRollFileIndex();
+
+    if (boost::filesystem::exists(g_fullpath) && logFileLimit > 0) {
+      rollLogFile();
     }
-  } else if (g_logFile) {
-    delete g_logFile;
-    g_logFile = nullptr;
-    g_logFileWithExt = nullptr;
+    writeBanner();
   }
-  writeBanner();
 }
 
 void Log::close() {
   std::lock_guard<decltype(g_logMutex)> guard(g_logMutex);
 
-  std::string oldfile;
-
-  if (g_logFile) {
-    oldfile = *g_logFile;
-    delete g_logFile;
-    g_logFile = nullptr;
-  }
-  if (g_logFileWithExt) {
-    delete g_logFileWithExt;
-    g_logFileWithExt = nullptr;
-  }
-
   if (g_log) {
     fclose(g_log);
     g_log = nullptr;
   }
+  g_fullpath = "";
 }
 
 void Log::writeBanner() {
-  if (g_logFileWithExt == nullptr) {
-    return;
-  }
-  const char* dirname = ACE::dirname(g_logFileWithExt->c_str());
-  if (GF_FILEEXISTS(dirname) != 0 && ACE_OS::mkdir(dirname) != 0) {
-    std::string msg =
-        "Error in creating directories for: " + std::string(dirname);
-    throw GeodeIOException(msg.c_str());
-  }
-  // retry some number of times before giving up when file is busy etc.
-  int maxTries = 10;
-  while (maxTries-- > 0) {
-    g_log = fopen(g_logFileWithExt->c_str(), "a");
-    if (g_log != nullptr) {
-      break;
-    }
-    int lastError = ACE_OS::last_error();
-    if (lastError != EACCES && lastError != EINTR && lastError != EWOULDBLOCK) {
-      break;
-    }
-    // continue after some sleep
-    std::this_thread::sleep_for(std::chrono::milliseconds(200));
-  }
-  if (!g_log) {
-    g_isLogFileOpened = false;
-    return;
-  } else {
-    g_isLogFileOpened = true;
-  }
-
-  if (s_logLevel == LogLevel::None) {
-    return;
-  }
-  std::string bannertext = geodeBanner::getBanner();
-
-  if (g_logFile == nullptr) {
-    fprintf(stdout, "%s", bannertext.c_str());
-    fflush(stdout);
-    return;
-  }  // else
-
-  if (fprintf(g_log, "%s", bannertext.c_str()) == 0 || ferror(g_log)) {
-    // we should be continue,
-    return;
-  }
+  if (s_logLevel != LogLevel::None) {
+    std::string bannertext = geodeBanner::getBanner();
 
-  int numchars = 0;
-  const char* pch = nullptr;
-  pch = strchr(bannertext.c_str(), '\n');
-  while (pch != nullptr) {
-    pch = strchr(pch + 1, '\n');
-    numchars += 2;
+    // fullpath empty --> we're logging to stdout
+    if (g_fullpath.string().empty()) {
+      fprintf(stdout, "%s", bannertext.c_str());
+      fflush(stdout);
+    } else {
+      if (boost::filesystem::exists(
+              g_fullpath.parent_path().string().c_str()) ||
+          boost::filesystem::create_directories(g_fullpath.parent_path())) {
+        g_log = fopen(g_fullpath.string().c_str(), "a");
+        if (g_log) {
+          if (fprintf(g_log, "%s", bannertext.c_str())) {
+            g_bytesWritten += static_cast<int32_t>(bannertext.length());
+            fflush(g_log);
+          }
+        }
+      }
+    }
   }
-
-  g_bytesWritten += static_cast<int32_t>(bannertext.length() + numchars);
-  fflush(g_log);
 }
 
 const char* Log::levelToChars(LogLevel level) {
@@ -463,9 +317,8 @@ const char* Log::levelToChars(LogLevel level) {
     case LogLevel::All:
       return "all";
   }
-  char buf[64] = {0};
-  std::snprintf(buf, 64, "Unexpected log level: %d", static_cast<int>(level));
-  throw IllegalArgumentException(buf);
+  throw IllegalArgumentException(std::string("Unexpected log level: ") +
+                                 std::to_string(static_cast<int>(level)));
 }
 
 LogLevel Log::charsToLevel(const std::string& chars) {
@@ -502,497 +355,100 @@ LogLevel Log::charsToLevel(const std::string& chars) {
   }
 }
 
-char* Log::formatLogLine(char* buf, LogLevel level) {
-  if (g_pid == 0) {
-    g_pid = boost::this_process::get_id();
-    ACE_OS::uname(&g_uname);
-  }
+std::string Log::formatLogLine(LogLevel level) {
+  std::stringstream msg;
   const size_t MINBUFSIZE = 128;
   auto now = std::chrono::system_clock::now();
   auto secs = std::chrono::system_clock::to_time_t(now);
   auto microseconds = std::chrono::duration_cast<std::chrono::microseconds>(
       now - std::chrono::system_clock::from_time_t(secs));
   auto tm_val = apache::geode::util::chrono::localtime(secs);
-  auto pbuf = buf;
-  pbuf += std::snprintf(pbuf, 15, "[%s ", Log::levelToChars(level));
-  pbuf += std::strftime(pbuf, MINBUFSIZE, "%Y/%m/%d %H:%M:%S", &tm_val);
-  pbuf += std::snprintf(pbuf, 15, ".%06" PRId64 " ",
-                        static_cast<int64_t>(microseconds.count()));
-  pbuf += std::strftime(pbuf, MINBUFSIZE, "%Z ", &tm_val);
 
-  std::snprintf(pbuf, 300, "%s:%d %" PRIu64 "] ", g_uname.nodename, g_pid,
-                hacks::aceThreadId(ACE_OS::thr_self()));
+  msg << "[" << Log::levelToChars(level) << " "
+      << std::put_time(&tm_val, "%Y/%m/%d %H:%M:%S") << '.' << std::setfill('0')
+      << std::setw(6) << microseconds.count() << ' '
+      << std::put_time(&tm_val, "%Z  ") << g_hostName << ":"
+      << boost::this_process::get_id() << " " << std::this_thread::get_id()
+      << "] ";
 
-  return buf;
+  return msg.str();
 }
 
-void Log::put(LogLevel level, const std::string& msg) {
-  put(level, msg.c_str());
+void Log::log(LogLevel level, const std::string& msg) {
+  Log::logInternal(level, msg);
 }
 
-// int g_count = 0;
-void Log::put(LogLevel level, const char* msg) {
+void Log::logInternal(LogLevel level, const std::string& msg) {
   std::lock_guard<decltype(g_logMutex)> guard(g_logMutex);
 
-  g_fileInfo fileInfo;
-
-  char buf[256] = {0};
+  std::string buf;
   char fullpath[512] = {0};
 
-  if (!g_logFile) {
-    fprintf(stdout, "%s%s\n", formatLogLine(buf, level), msg);
+  if (g_fullpath.string().empty()) {
+    fprintf(stdout, "%s%s\n", formatLogLine(level).c_str(), msg.c_str());
     fflush(stdout);
-    // TODO: ignoring for now; probably store the log-lines for possible
-    // future logging if log-file gets initialized properly
-
   } else {
-    if (!g_isLogFileOpened) {
-      g_log = fopen(g_logFileWithExt->c_str(), "a");
-      if (!g_log) {
-        g_isLogFileOpened = false;
-        return;
-      }
-      g_isLogFileOpened = true;
-    } else if (!g_log) {
-      g_log = fopen(g_logFileWithExt->c_str(), "a");
-      if (!g_log) {
-        return;
-      }
+    if (!g_log) {
+      g_log = fopen(g_fullpath.string().c_str(), "a");
     }
 
-    formatLogLine(buf, level);
-    auto numChars = static_cast<int>(std::strlen(buf) + std::strlen(msg));
-    g_bytesWritten +=
-        numChars + 2;  // bcoz we have to count trailing new line (\n)
-
-    if ((g_fileSizeLimit != 0) && (g_bytesWritten >= g_fileSizeLimit)) {
-      char rollFile[1024] = {0};
-      std::string logsdirname;
-      std::string logsbasename;
-      std::string fnameBeforeExt;
-      std::string extName;
-      std::string newfilestr;
-
-      int32_t len = static_cast<int32_t>(g_logFileWithExt->length());
-      int32_t lastPosOfSep = static_cast<int32_t>(
-          g_logFileWithExt->find_last_of(ACE_DIRECTORY_SEPARATOR_CHAR, len));
-      if (lastPosOfSep == -1) {
-        logsdirname = ".";
-      } else {
-        logsdirname = g_logFileWithExt->substr(0, lastPosOfSep);
-      }
-      logsbasename = g_logFileWithExt->substr(lastPosOfSep + 1, len);
-      char logFileExtAfter = '.';
-      int32_t baselen = static_cast<int32_t>(logsbasename.length());
-      int32_t posOfExt = static_cast<int32_t>(
-          logsbasename.find_last_of(logFileExtAfter, baselen));
-      if (posOfExt == -1) {
-        // throw IllegalArgument;
-      } else {
-        fnameBeforeExt = logsbasename.substr(0, posOfExt);
-        extName = logsbasename.substr(posOfExt + 1, baselen);
-      }
-      std::snprintf(rollFile, 1024, "%s%c%s-%d.%s", logsdirname.c_str(),
-                    ACE_DIRECTORY_SEPARATOR_CHAR, fnameBeforeExt.c_str(),
-                    g_rollIndex++, extName.c_str());
-      bool rollFileNameGot = false;
-      while (!rollFileNameGot) {
-        FILE* fp1 = fopen(rollFile, "r");
-        if (fp1 != nullptr) {
-          fclose(fp1);
-          std::snprintf(rollFile, 1024, "%s%c%s-%d.%s", logsdirname.c_str(),
-                        ACE_DIRECTORY_SEPARATOR_CHAR, fnameBeforeExt.c_str(),
-                        g_rollIndex++, extName.c_str());
-        } else {
-          rollFileNameGot = true;
-        }
-      }
-
-      fclose(g_log);
-      g_log = nullptr;
+    if (g_log) {
+      buf = formatLogLine(level);
+      auto numChars = static_cast<int>(buf.length() + msg.length());
+      g_bytesWritten +=
+          numChars + 2;  // bcoz we have to count trailing new line (\n)
 
-      if (ACE_OS::rename(g_logFileWithExt->c_str(), rollFile) < 0) {
-        return;  // no need to throw exception try next time
+      if ((g_fileSizeLimit != 0) && (g_bytesWritten >= g_fileSizeLimit)) {
+        rollLogFile();
+        g_bytesWritten = numChars + 2;  // Account for trailing newline
+        writeBanner();
       }
 
-      g_bytesWritten =
-          numChars + 2;  // bcoz we have to count trailing new line (\n)
-      writeBanner();
-    }
+      g_spaceUsed += numChars + 2;
 
-    g_spaceUsed += g_bytesWritten;
-
-    if ((g_diskSpaceLimit > 0) && (g_spaceUsed >= g_diskSpaceLimit)) {
-      std::string dirname = ACE::dirname(g_logFile->c_str());
-      g_spaceUsed = 0;
-      ACE_stat statBuf = {};
-
-      ACE_Dirent_Selector sds;
-      int status = sds.open(dirname.c_str(), selector, comparator);
-      if (status != -1) {
-        for (int index = 1; index < sds.length(); ++index) {
-          std::snprintf(fullpath, 512, "%s%c%s", dirname.c_str(),
-                        ACE_DIRECTORY_SEPARATOR_CHAR, sds[index]->d_name);
-          ACE_OS::stat(fullpath, &statBuf);
-          g_fileInfoPair = std::make_pair(fullpath, statBuf.st_size);
-          fileInfo.push_back(g_fileInfoPair);
-          g_spaceUsed += fileInfo[index - 1].second;
-        }  // for loop
-        g_spaceUsed += g_bytesWritten;
-        sds.close();
+      // Remove existing rolled log files until we're below the limit
+      while (g_spaceUsed >= g_diskSpaceLimit) {
+        removeOldestRolledLogFile();
       }
-      int fileIndex = 0;
-
-      while ((g_spaceUsed > (g_diskSpaceLimit /*- g_fileSizeLimit*/))) {
-        int64_t fileSize = fileInfo[fileIndex].second;
-        if (ACE_OS::unlink(fileInfo[fileIndex].first.c_str()) == 0) {
-          g_spaceUsed -= fileSize;
-        } else {
-          char printmsg[256];
-          std::snprintf(printmsg, 256, "%s\t%s\n", "Could not delete",
-                        fileInfo[fileIndex].first.c_str());
-          numChars =
-              fprintf(g_log, "%s%s\n", formatLogLine(buf, level), printmsg);
-          g_bytesWritten +=
-              numChars + 2;  // bcoz we have to count trailing new line (\n)
-        }
-        fileIndex++;
-      }
-    }
 
-    if ((numChars = fprintf(g_log, "%s%s\n", buf, msg)) == 0 || ferror(g_log)) {
-      if ((g_diskSpaceLimit > 0)) {
-        g_spaceUsed = g_spaceUsed - (numChars + 2);
-      }
-      if (g_fileSizeLimit > 0) {
-        g_bytesWritten = g_bytesWritten - (numChars + 2);
+      if ((numChars = fprintf(g_log, "%s%s\n", buf.c_str(), msg.c_str())) ==
+              0 ||
+          ferror(g_log)) {
+        // Let's continue without throwing the exception.  It should not cause
+        // process to terminate
+        fclose(g_log);
+        g_log = nullptr;
+      } else {
+        fflush(g_log);
       }
-
-      // lets continue wothout throwing the exception; it should not cause
-      // process to terminate
-      fclose(g_log);
-      g_log = nullptr;
-    } else {
-      fflush(g_log);
     }
   }
 }
 
-void Log::putThrow(LogLevel level, const char* msg, const Exception& ex) {
-  std::string message = "Geode exception " + ex.getName() +
-                        " thrown: " + ex.getMessage() + "\n" + msg;
-  put(level, message);
-}
-
-void Log::putCatch(LogLevel level, const char* msg, const Exception& ex) {
-  std::string message = "Geode exception " + ex.getName() +
-                        " caught: " + ex.getMessage() + "\n" + msg;
-  put(level, message);
-}
-
-void Log::enterFn(LogLevel level, const char* functionName) {
-  enum { MAX_NAME_LENGTH = 1024 };
-  std::string fn = functionName;
-  if (fn.size() > MAX_NAME_LENGTH) {
-    fn = fn.substr(fn.size() - MAX_NAME_LENGTH, MAX_NAME_LENGTH);
-  }
-  char buf[MAX_NAME_LENGTH + 512] = {0};
-  std::snprintf(buf, 1536, "{{{===>>> Entering function %s", fn.c_str());
-  put(level, buf);
-}
-
-void Log::exitFn(LogLevel level, const char* functionName) {
-  enum { MAX_NAME_LENGTH = 1024 };
-  std::string fn = functionName;
-  if (fn.size() > MAX_NAME_LENGTH) {
-    fn = fn.substr(fn.size() - MAX_NAME_LENGTH, MAX_NAME_LENGTH);
-  }
-  char buf[MAX_NAME_LENGTH + 512] = {0};
-  std::snprintf(buf, 1536, "<<<===}}} Exiting function %s", fn.c_str());
-  put(level, buf);
-}
-
-bool Log::enabled(LogLevel level) {
-  return GEODE_HIGHEST_LOG_LEVEL >= level && s_logLevel >= level;
-}
-
-void Log::log(LogLevel level, const char* msg) {
-  if (enabled(level)) put(level, msg);
-}
-
-void Log::logThrow(LogLevel level, const char* msg, const Exception& ex) {
-  if (enabled(level)) putThrow(level, msg, ex);
-}
-
-void Log::logCatch(LogLevel level, const char* msg, const Exception& ex) {
-  if (enabled(level)) putCatch(level, msg, ex);
-}
-
-bool Log::errorEnabled() {
-  return GEODE_HIGHEST_LOG_LEVEL >= LogLevel::Error &&
-         s_logLevel >= LogLevel::Error;
-}
-
-void Log::error(const char* msg) {
-  if (errorEnabled()) put(LogLevel::Error, msg);
-}
-
-void Log::error(const std::string& msg) {
-  if (errorEnabled()) put(LogLevel::Error, msg.c_str());
-}
-
-void Log::errorThrow(const char* msg, const Exception& ex) {
-  if (errorEnabled()) putThrow(LogLevel::Error, msg, ex);
-}
-
-void Log::errorCatch(const char* msg, const Exception& ex) {
-  if (errorEnabled()) putCatch(LogLevel::Error, msg, ex);
-}
-
-bool Log::warningEnabled() {
-  return GEODE_HIGHEST_LOG_LEVEL >= LogLevel::Warning &&
-         s_logLevel >= LogLevel::Warning;
-}
-
-void Log::warning(const char* msg) {
-  if (warningEnabled()) put(LogLevel::Warning, msg);
-}
-
-void Log::warningThrow(const char* msg, const Exception& ex) {
-  if (warningEnabled()) putThrow(LogLevel::Warning, msg, ex);
-}
-
-void Log::warningCatch(const char* msg, const Exception& ex) {
-  if (warningEnabled()) putCatch(LogLevel::Warning, msg, ex);
-}
-
-bool Log::infoEnabled() {
-  return GEODE_HIGHEST_LOG_LEVEL >= LogLevel::Info &&
-         s_logLevel >= LogLevel::Info;
-}
-
-void Log::info(const char* msg) {
-  if (infoEnabled()) put(LogLevel::Info, msg);
-}
-
-void Log::infoThrow(const char* msg, const Exception& ex) {
-  if (infoEnabled()) putThrow(LogLevel::Info, msg, ex);
-}
-
-void Log::infoCatch(const char* msg, const Exception& ex) {
-  if (infoEnabled()) putCatch(LogLevel::Info, msg, ex);
-}
-
-bool Log::configEnabled() {
-  return GEODE_HIGHEST_LOG_LEVEL >= LogLevel::Config &&
-         s_logLevel >= LogLevel::Config;
-}
-
-void Log::config(const char* msg) {
-  if (configEnabled()) put(LogLevel::Config, msg);
-}
-
-void Log::configThrow(const char* msg, const Exception& ex) {
-  if (configEnabled()) putThrow(LogLevel::Config, msg, ex);
-}
-
-void Log::configCatch(const char* msg, const Exception& ex) {
-  if (configEnabled()) putCatch(LogLevel::Config, msg, ex);
-}
-
-bool Log::fineEnabled() {
-  return GEODE_HIGHEST_LOG_LEVEL >= LogLevel::Fine &&
-         s_logLevel >= LogLevel::Fine;
-}
-
-void Log::fine(const char* msg) {
-  if (fineEnabled()) put(LogLevel::Fine, msg);
-}
-
-void Log::fineThrow(const char* msg, const Exception& ex) {
-  if (fineEnabled()) putThrow(LogLevel::Fine, msg, ex);
-}
-
-void Log::fineCatch(const char* msg, const Exception& ex) {
-  if (fineEnabled()) putCatch(LogLevel::Fine, msg, ex);
-}
-
-bool Log::finerEnabled() {
-  return GEODE_HIGHEST_LOG_LEVEL >= LogLevel::Finer &&
-         s_logLevel >= LogLevel::Finer;
-}
-
-void Log::finer(const char* msg) {
-  if (finerEnabled()) put(LogLevel::Finer, msg);
-}
-
-void Log::finerThrow(const char* msg, const Exception& ex) {
-  if (finerEnabled()) putThrow(LogLevel::Finer, msg, ex);
-}
-
-void Log::finerCatch(const char* msg, const Exception& ex) {
-  if (finerEnabled()) putCatch(LogLevel::Finer, msg, ex);
-}
-
-bool Log::finestEnabled() {
-  return GEODE_HIGHEST_LOG_LEVEL >= LogLevel::Finest &&
-         s_logLevel >= LogLevel::Finest;
-}
-
-void Log::finest(const char* msg) {
-  if (finestEnabled()) put(LogLevel::Finest, msg);
-}
-
-void Log::finestThrow(const char* msg, const Exception& ex) {
-  if (finestEnabled()) putThrow(LogLevel::Finest, msg, ex);
-}
-
-void Log::finestCatch(const char* msg, const Exception& ex) {
-  if (finestEnabled()) putCatch(LogLevel::Finest, msg, ex);
-}
-
-bool Log::debugEnabled() {
-  return GEODE_HIGHEST_LOG_LEVEL >= LogLevel::Debug &&
-         s_logLevel >= LogLevel::Debug;
-}
-
-void Log::debug(const char* msg) {
-  if (debugEnabled()) put(LogLevel::Debug, msg);
-}
-
-void Log::debugThrow(const char* msg, const Exception& ex) {
-  if (debugEnabled()) putThrow(LogLevel::Debug, msg, ex);
-}
-
-void Log::debugCatch(const char* msg, const Exception& ex) {
-  if (debugEnabled()) putCatch(LogLevel::Debug, msg, ex);
-}
-
-LogFn::LogFn(const char* functionName, LogLevel level)
-    : m_functionName(functionName), m_level(level) {
-  if (Log::enabled(m_level)) Log::enterFn(m_level, m_functionName);
-}
-
-LogFn::~LogFn() {
-  if (Log::enabled(m_level)) Log::exitFn(m_level, m_functionName);
-}
-
-// var arg logging routines.
-
 #ifdef _WIN32
 #define vsnprintf _vsnprintf
 #endif
 
-void LogVarargs::debug(const char* fmt, ...) {
-  char msg[_GF_MSG_LIMIT] = {0};
-  va_list argp;
-  va_start(argp, fmt);
-  vsnprintf(msg, _GF_MSG_LIMIT, fmt, argp);
-  /* win doesn't guarantee termination */ msg[_GF_MSG_LIMIT - 1] = '\0';
-  Log::put(LogLevel::Debug, msg);
-  va_end(argp);
-}
-
-void LogVarargs::error(const char* fmt, ...) {
-  char msg[_GF_MSG_LIMIT] = {0};
-  va_list argp;
-  va_start(argp, fmt);
-  vsnprintf(msg, _GF_MSG_LIMIT, fmt, argp);
-  /* win doesn't guarantee termination */ msg[_GF_MSG_LIMIT - 1] = '\0';
-  Log::put(LogLevel::Error, msg);
-  va_end(argp);
-}
-
-void LogVarargs::warn(const char* fmt, ...) {
-  char msg[_GF_MSG_LIMIT] = {0};
-  va_list argp;
-  va_start(argp, fmt);
-  vsnprintf(msg, _GF_MSG_LIMIT, fmt, argp);
-  /* win doesn't guarantee termination */ msg[_GF_MSG_LIMIT - 1] = '\0';
-  Log::put(LogLevel::Warning, msg);
-  va_end(argp);
-}
-
-void LogVarargs::info(const char* fmt, ...) {
-  char msg[_GF_MSG_LIMIT] = {0};
-  va_list argp;
-  va_start(argp, fmt);
-  vsnprintf(msg, _GF_MSG_LIMIT, fmt, argp);
-  /* win doesn't guarantee termination */ msg[_GF_MSG_LIMIT - 1] = '\0';
-  Log::put(LogLevel::Info, msg);
-  va_end(argp);
-}
-
-void LogVarargs::config(const char* fmt, ...) {
-  char msg[_GF_MSG_LIMIT] = {0};
-  va_list argp;
-  va_start(argp, fmt);
-  vsnprintf(msg, _GF_MSG_LIMIT, fmt, argp);
-  /* win doesn't guarantee termination */ msg[_GF_MSG_LIMIT - 1] = '\0';
-  Log::put(LogLevel::Config, msg);
-  va_end(argp);
-}
-
-void LogVarargs::fine(const char* fmt, ...) {
-  char msg[_GF_MSG_LIMIT] = {0};
-  va_list argp;
-  va_start(argp, fmt);
-  vsnprintf(msg, _GF_MSG_LIMIT, fmt, argp);
-  /* win doesn't guarantee termination */ msg[_GF_MSG_LIMIT - 1] = '\0';
-  Log::put(LogLevel::Fine, msg);
-  va_end(argp);
-}
-
-void LogVarargs::finer(const char* fmt, ...) {
-  char msg[_GF_MSG_LIMIT] = {0};
-  va_list argp;
-  va_start(argp, fmt);
-  vsnprintf(msg, _GF_MSG_LIMIT, fmt, argp);
-  /* win doesn't guarantee termination */ msg[_GF_MSG_LIMIT - 1] = '\0';
-  Log::put(LogLevel::Finer, msg);
-  va_end(argp);
-}
-
-void LogVarargs::finest(const char* fmt, ...) {
+void Log::log(LogLevel level, const char* fmt, ...) {
   char msg[_GF_MSG_LIMIT] = {0};
   va_list argp;
   va_start(argp, fmt);
   vsnprintf(msg, _GF_MSG_LIMIT, fmt, argp);
   /* win doesn't guarantee termination */ msg[_GF_MSG_LIMIT - 1] = '\0';
-  Log::put(LogLevel::Finest, msg);
+  Log::logInternal(level, std::string(msg));
   va_end(argp);
 }
 
-void LogVarargs::debug(const std::string& message) {
-  Log::put(LogLevel::Debug, message.c_str());
-}
-
-void LogVarargs::error(const std::string& message) {
-  Log::put(LogLevel::Error, message.c_str());
-}
-
-void LogVarargs::warn(const std::string& message) {
-  Log::put(LogLevel::Warning, message.c_str());
-}
-
-void LogVarargs::info(const std::string& message) {
-  Log::put(LogLevel::Info, message.c_str());
-}
-
-void LogVarargs::config(const std::string& message) {
-  Log::put(LogLevel::Config, message.c_str());
-}
-
-void LogVarargs::fine(const std::string& message) {
-  Log::put(LogLevel::Fine, message.c_str());
-}
-
-void LogVarargs::finer(const std::string& message) {
-  Log::put(LogLevel::Finer, message.c_str());
+void Log::logCatch(LogLevel level, const char* msg, const Exception& ex) {
+  if (enabled(level)) {
+    std::string message = "Geode exception " + ex.getName() +
+                          " caught: " + ex.getMessage() + "\n" + msg;
+    log(level, message);
+  }
 }
 
-void LogVarargs::finest(const std::string& message) {
-  Log::put(LogLevel::Finest, message.c_str());
+bool Log::enabled(LogLevel level) {
+  return (level != LogLevel::None && level <= logLevel());
 }
 
 }  // namespace client
diff --git a/cppcache/src/TcrEndpoint.cpp b/cppcache/src/TcrEndpoint.cpp
index 0ef91e2..f5dee10 100644
--- a/cppcache/src/TcrEndpoint.cpp
+++ b/cppcache/src/TcrEndpoint.cpp
@@ -993,9 +993,7 @@ GfErrType TcrEndpoint::sendRequestWithRetry(
                 failReason.c_str());
         if (compareTransactionIds(reqTransId, reply.getTransId(), failReason,
                                   conn)) {
-          if (Log::warningEnabled()) {
-            LOGWARN("Stack trace: %s", ex.getStackTrace().c_str());
-          }
+          LOGWARN("Stack trace: %s", ex.getStackTrace().c_str());
           error = GF_MSG;
           if (useEPPool) {
             m_opConnections.put(conn, false);
diff --git a/cppcache/src/ThinClientDistributionManager.cpp b/cppcache/src/ThinClientDistributionManager.cpp
index a9a30e2..3facc2c 100644
--- a/cppcache/src/ThinClientDistributionManager.cpp
+++ b/cppcache/src/ThinClientDistributionManager.cpp
@@ -79,7 +79,7 @@ void ThinClientDistributionManager::destroy(bool keepAlive) {
   destroyAction();
   // stop the chunk processing thread
   stopChunkProcessor();
-  if (Log::finestEnabled()) {
+  if (Log::enabled(LogLevel::Finest)) {
     std::string endpointStr;
     for (size_t index = 0; index < m_endpoints.size(); ++index) {
       if (index != 0) {
diff --git a/cppcache/src/statistics/GeodeStatisticsFactory.cpp b/cppcache/src/statistics/GeodeStatisticsFactory.cpp
index 8ea2809..900e385 100644
--- a/cppcache/src/statistics/GeodeStatisticsFactory.cpp
+++ b/cppcache/src/statistics/GeodeStatisticsFactory.cpp
@@ -23,6 +23,7 @@
 
 #include <geode/Exception.hpp>
 #include <geode/internal/geode_globals.hpp>
+#include <geode/util/LogLevel.hpp>
 
 #include "../util/Log.hpp"
 #include "AtomicStatisticsImpl.hpp"
@@ -36,6 +37,7 @@ namespace statistics {
 using client::Exception;
 using client::IllegalArgumentException;
 using client::Log;
+using client::LogLevel;
 using client::OutOfMemoryException;
 
 GeodeStatisticsFactory::GeodeStatisticsFactory(StatisticsManager* statMngr) {
@@ -61,13 +63,12 @@ GeodeStatisticsFactory::~GeodeStatisticsFactory() {
     statsTypeMap.clear();
 
   } catch (const Exception& ex) {
-    Log::warningCatch("~GeodeStatisticsFactory swallowing Geode exception", ex);
-
+    Log::logCatch(LogLevel::Warning,
+                  "~GeodeStatisticsFactory swallowing Geode exception", ex);
   } catch (const std::exception& ex) {
     std::string what = "~GeodeStatisticsFactory swallowing std::exception: ";
     what += ex.what();
     LOGWARN(what.c_str());
-
   } catch (...) {
     LOGERROR("~GeodeStatisticsFactory swallowing unknown exception");
   }
diff --git a/cppcache/src/statistics/StatisticsManager.cpp b/cppcache/src/statistics/StatisticsManager.cpp
index 4f7b8d2..480cbeb 100644
--- a/cppcache/src/statistics/StatisticsManager.cpp
+++ b/cppcache/src/statistics/StatisticsManager.cpp
@@ -21,6 +21,7 @@
 
 #include <geode/Exception.hpp>
 #include <geode/internal/geode_globals.hpp>
+#include <geode/util/LogLevel.hpp>
 
 #include "../AdminRegion.hpp"
 #include "../util/Log.hpp"
@@ -35,6 +36,7 @@ namespace statistics {
 
 using client::Exception;
 using client::Log;
+using client::LogLevel;
 
 StatisticsManager::StatisticsManager(
     const char* filePath, const std::chrono::milliseconds sampleInterval,
@@ -95,15 +97,15 @@ StatisticsManager::~StatisticsManager() {
       m_statsList.erase(m_statsList.begin(), m_statsList.end());
     }
   } catch (const Exception& ex) {
-    Log::warningCatch("~StatisticsManager swallowing Geode exception", ex);
-
+    Log::logCatch(LogLevel::Warning,
+                  "~StatisticsManager swallowing Geode exception", ex);
   } catch (const std::exception& ex) {
     std::string what = "~StatisticsManager swallowing std::exception: ";
     what += ex.what();
-    Log::warning(what.c_str());
+    LOGWARN(what.c_str());
 
   } catch (...) {
-    Log::error("~StatisticsManager swallowing unknown exception");
+    LOGERROR("~StatisticsManager swallowing unknown exception");
   }
 }
 
diff --git a/cppcache/src/util/Log.hpp b/cppcache/src/util/Log.hpp
index 6ac930d..ae33fef 100644
--- a/cppcache/src/util/Log.hpp
+++ b/cppcache/src/util/Log.hpp
@@ -144,10 +144,13 @@ class APACHE_GEODE_EXPORT Log {
    * arguments
    */
   static void init
-      // 0 => use maximum value (currently 1G)
+      // 0 => use default value (currently 1GB for file, 1TB for disk)
       (LogLevel level, const char* logFileName, int32_t logFileLimit = 0,
        int64_t logDiskSpaceLimit = 0);
 
+  static void init(LogLevel level, const std::string& logFileName,
+                   int32_t logFileLimit = 0, int64_t logDiskSpaceLimit = 0);
+
   /**
    * closes logging facility (until next init).
    */
@@ -169,9 +172,7 @@ class APACHE_GEODE_EXPORT Log {
   static LogLevel charsToLevel(const std::string& chars);
 
   /**
-   * Fills the provided buffer with formatted log-line given the level
-   * and returns the buffer. This assumes that the buffer has large
-   * enough space left to hold the formatted log-line (around 70 chars).
+   * formats and returns a line for logging.
    *
    * This is provided so that applications wishing to use the same format
    * as Geode log-lines can do so easily. A log-line starts with the prefix
@@ -182,315 +183,96 @@ class APACHE_GEODE_EXPORT Log {
    * When invoking from outside either <init> should have been invoked,
    * or at least the first invocation should be single-threaded.
    */
-  static char* formatLogLine(char* buf, LogLevel level);
-
-  /**
-   * Returns whether log messages at given level are enabled.
-   */
-  static bool enabled(LogLevel level);
+  static std::string formatLogLine(LogLevel level);
 
-  /**
-   * Logs a message at given level.
-   */
-  static void log(LogLevel level, const char* msg);
+  static void log(LogLevel level, const std::string& msg);
 
-  /**
-   * Logs both a message and thrown exception.
-   */
-  static void logThrow(LogLevel level, const char* msg, const Exception& ex);
+  static void log(LogLevel level, const char* fmt, ...);
 
-  /**
-   * Logs both a message and caught exception.
-   */
   static void logCatch(LogLevel level, const char* msg, const Exception& ex);
 
-  /**
-   * Returns whether "error" log messages are enabled.
-   */
-  static bool errorEnabled();
-
-  /**
-   * Logs a message.
-   * The message level is "error".
-   */
-  static void error(const char* msg);
-
-  static void error(const std::string& msg);
-
-  /**
-   * Logs both a message and thrown exception.
-   * The message level is "error".
-   */
-  static void errorThrow(const char* msg, const Exception& ex);
-
-  /**
-   * Writes both a message and caught exception.
-   * The message level is "error".
-   */
-  static void errorCatch(const char* msg, const Exception& ex);
-
-  /**
-   * Returns whether "warning" log messages are enabled.
-   */
-  static bool warningEnabled();
-
-  /**
-   * Logs a message.
-   * The message level is "warning".
-   */
-  static void warning(const char* msg);
-
-  /**
-   * Logs both a message and thrown exception.
-   * The message level is "warning".
-   */
-  static void warningThrow(const char* msg, const Exception& ex);
-
-  /**
-   * Writes both a message and caught exception.
-   * The message level is "warning".
-   */
-  static void warningCatch(const char* msg, const Exception& ex);
-
-  /**
-   * Returns whether "info" log messages are enabled.
-   */
-  static bool infoEnabled();
-
-  /**
-   * Logs a message.
-   * The message level is "info".
-   */
-  static void info(const char* msg);
-
-  /**
-   * Logs both a message and thrown exception.
-   * The message level is "info".
-   */
-  static void infoThrow(const char* msg, const Exception& ex);
-
-  /**
-   * Writes both a message and caught exception.
-   * The message level is "info".
-   */
-  static void infoCatch(const char* msg, const Exception& ex);
-
-  /**
-   * Returns whether "config" log messages are enabled.
-   */
-  static bool configEnabled();
-
-  /**
-   * Logs a message.
-   * The message level is "config".
-   */
-  static void config(const char* msg);
-
-  /**
-   * Logs both a message and thrown exception.
-   * The message level is "config".
-   */
-  static void configThrow(const char* msg, const Exception& ex);
-
-  /**
-   * Writes both a message and caught exception.
-   * The message level is "config".
-   */
-  static void configCatch(const char* msg, const Exception& ex);
-
-  /**
-   * Returns whether "fine" log messages are enabled.
-   */
-  static bool fineEnabled();
-
-  /**
-   * Logs a message.
-   * The message level is "fine".
-   */
-  static void fine(const char* msg);
-
-  /**
-   * Logs both a message and thrown exception.
-   * The message level is "fine".
-   */
-  static void fineThrow(const char* msg, const Exception& ex);
-
-  /**
-   * Writes both a message and caught exception.
-   * The message level is "fine".
-   */
-  static void fineCatch(const char* msg, const Exception& ex);
-
-  /**
-   * Returns whether "finer" log messages are enabled.
-   */
-  static bool finerEnabled();
-
-  /**
-   * Logs a message.
-   * The message level is "finer".
-   */
-  static void finer(const char* msg);
-
-  /**
-   * Logs both a message and thrown exception.
-   * The message level is "finer".
-   */
-  static void finerThrow(const char* msg, const Exception& ex);
-
-  /**
-   * Writes both a message and caught exception.
-   * The message level is "finer".
-   */
-  static void finerCatch(const char* msg, const Exception& ex);
-
-  /**
-   * Returns whether "finest" log messages are enabled.
-   */
-  static bool finestEnabled();
-
-  /**
-   * Logs a message.
-   * The message level is "finest".
-   */
-  static void finest(const char* msg);
-
-  /**
-   * Logs both a message and thrown exception.
-   * The message level is "finest".
-   */
-  static void finestThrow(const char* msg, const Exception& ex);
-
-  /**
-   * Writes both a message and caught exception.
-   * The message level is "finest".
-   */
-  static void finestCatch(const char* msg, const Exception& ex);
-
-  /**
-   * Returns whether "debug" log messages are enabled.
-   */
-  static bool debugEnabled();
-
-  /**
-   * Logs a message.
-   * The message level is "debug".
-   */
-  static void debug(const char* msg);
-
-  /**
-   * Logs both a message and thrown exception.
-   * The message level is "debug".
-   */
-  static void debugThrow(const char* msg, const Exception& ex);
-
-  /**
-   * Writes both a message and caught exception.
-   * The message level is "debug".
-   */
-  static void debugCatch(const char* msg, const Exception& ex);
-
-  static void enterFn(LogLevel level, const char* functionName);
-
-  static void exitFn(LogLevel level, const char* functionName);
+  static bool enabled(LogLevel level);
 
  private:
   static LogLevel s_logLevel;
 
   static void writeBanner();
 
- public:
-  static void put(LogLevel level, const std::string& msg);
+  static void validateSizeLimits(int64_t fileSizeLimit, int64_t diskSpaceLimit);
 
-  static void put(LogLevel level, const char* msg);
+  static void validateLogFileName(const std::string& filename);
 
-  static void putThrow(LogLevel level, const char* msg, const Exception& ex);
+  static void rollLogFile();
 
-  static void putCatch(LogLevel level, const char* msg, const Exception& ex);
-};
+  static void removeOldestRolledLogFile();
 
-class APACHE_GEODE_EXPORT LogFn {
-  const char* m_functionName;
-  LogLevel m_level;
+  static void buildRollFileMapping();
 
- public:
-  explicit LogFn(const char* functionName, LogLevel level = LogLevel::Finest);
+  static void setRollFileIndex();
 
-  ~LogFn();
+  static void setSizeLimits(int32_t logFileLimit, int64_t logDiskSpaceLimit);
 
-  LogFn(const LogFn& rhs) = delete;
-  LogFn& operator=(const LogFn& rhs) = delete;
+  static void logInternal(LogLevel level, const std::string& msg);
 };
 
-class APACHE_GEODE_EXPORT LogVarargs {
- public:
-  static void debug(const char* fmt, ...);
-  static void error(const char* fmt, ...);
-  static void warn(const char* fmt, ...);
-  static void info(const char* fmt, ...);
-  static void config(const char* fmt, ...);
-  static void fine(const char* fmt, ...);
-  static void finer(const char* fmt, ...);
-  static void finest(const char* fmt, ...);
-
-  static void debug(const std::string& message);
-
-  static void error(const std::string& message);
-
-  static void warn(const std::string& message);
-
-  static void info(const std::string& message);
-
-  static void config(const std::string& message);
-
-  static void fine(const std::string& message);
-
-  static void finer(const std::string& message);
-
-  static void finest(const std::string& message);
-};
 }  // namespace client
 }  // namespace geode
 }  // namespace apache
 
-#define LOGDEBUG                                  \
-  if (::apache::geode::client::LogLevel::Debug <= \
-      ::apache::geode::client::Log::logLevel())   \
-  ::apache::geode::client::LogVarargs::debug
-
-#define LOGERROR                                  \
-  if (::apache::geode::client::LogLevel::Error <= \
-      ::apache::geode::client::Log::logLevel())   \
-  ::apache::geode::client::LogVarargs::error
-
-#define LOGWARN                                     \
-  if (::apache::geode::client::LogLevel::Warning <= \
-      ::apache::geode::client::Log::logLevel())     \
-  ::apache::geode::client::LogVarargs::warn
-
-#define LOGINFO                                  \
-  if (::apache::geode::client::LogLevel::Info <= \
-      ::apache::geode::client::Log::logLevel())  \
-  ::apache::geode::client::LogVarargs::info
-
-#define LOGCONFIG                                  \
-  if (::apache::geode::client::LogLevel::Config <= \
-      ::apache::geode::client::Log::logLevel())    \
-  ::apache::geode::client::LogVarargs::config
-
-#define LOGFINE                                  \
-  if (::apache::geode::client::LogLevel::Fine <= \
-      ::apache::geode::client::Log::logLevel())  \
-  ::apache::geode::client::LogVarargs::fine
-
-#define LOGFINER                                  \
-  if (::apache::geode::client::LogLevel::Finer <= \
-      ::apache::geode::client::Log::logLevel())   \
-  ::apache::geode::client::LogVarargs::finer
-
-#define LOGFINEST                                  \
-  if (::apache::geode::client::LogLevel::Finest <= \
-      ::apache::geode::client::Log::logLevel())    \
-  ::apache::geode::client::LogVarargs::finest
+#define LOGERROR(...)                                           \
+  if (::apache::geode::client::Log::enabled(                    \
+          apache::geode::client::LogLevel::Error)) {            \
+    ::apache::geode::client::Log::log(                          \
+        ::apache::geode::client::LogLevel::Error, __VA_ARGS__); \
+  }
+
+#define LOGWARN(...)                                              \
+  if (::apache::geode::client::Log::enabled(                      \
+          apache::geode::client::LogLevel::Warning)) {            \
+    ::apache::geode::client::Log::log(                            \
+        ::apache::geode::client::LogLevel::Warning, __VA_ARGS__); \
+  }
+
+#define LOGINFO(...)                                                           \
+  if (::apache::geode::client::Log::enabled(                                   \
+          apache::geode::client::LogLevel::Info)) {                            \
+    ::apache::geode::client::Log::log(::apache::geode::client::LogLevel::Info, \
+                                      __VA_ARGS__);                            \
+  }
+
+#define LOGCONFIG(...)                                           \
+  if (::apache::geode::client::Log::enabled(                     \
+          apache::geode::client::LogLevel::Config)) {            \
+    ::apache::geode::client::Log::log(                           \
+        ::apache::geode::client::LogLevel::Config, __VA_ARGS__); \
+  }
+
+#define LOGFINE(...)                                                           \
+  if (::apache::geode::client::Log::enabled(                                   \
+          apache::geode::client::LogLevel::Fine)) {                            \
+    ::apache::geode::client::Log::log(::apache::geode::client::LogLevel::Fine, \
+                                      __VA_ARGS__);                            \
+  }
+
+#define LOGFINER(...)                                           \
+  if (::apache::geode::client::Log::enabled(                    \
+          apache::geode::client::LogLevel::Finer)) {            \
+    ::apache::geode::client::Log::log(                          \
+        ::apache::geode::client::LogLevel::Finer, __VA_ARGS__); \
+  }
+
+#define LOGFINEST(...)                                           \
+  if (::apache::geode::client::Log::enabled(                     \
+          apache::geode::client::LogLevel::Finest)) {            \
+    ::apache::geode::client::Log::log(                           \
+        ::apache::geode::client::LogLevel::Finest, __VA_ARGS__); \
+  }
+
+#define LOGDEBUG(...)                                           \
+  if (::apache::geode::client::Log::enabled(                    \
+          apache::geode::client::LogLevel::Debug)) {            \
+    ::apache::geode::client::Log::log(                          \
+        ::apache::geode::client::LogLevel::Debug, __VA_ARGS__); \
+  }
 
 #endif  // GEODE_LOG_H_
diff --git a/cppcache/test/CMakeLists.txt b/cppcache/test/CMakeLists.txt
index d821e32..a2a3e1e 100644
--- a/cppcache/test/CMakeLists.txt
+++ b/cppcache/test/CMakeLists.txt
@@ -42,6 +42,7 @@ add_executable(apache-geode_unittests
   gtest_extensions.h
   InterestResultPolicyTest.cpp
   LocalRegionTest.cpp
+  LoggingTest.cpp
   LRUQueueTest.cpp
   PdxInstanceImplTest.cpp
   PdxTypeTest.cpp
diff --git a/cppcache/test/LoggingTest.cpp b/cppcache/test/LoggingTest.cpp
new file mode 100644
index 0000000..cbeb4d3
--- /dev/null
+++ b/cppcache/test/LoggingTest.cpp
@@ -0,0 +1,610 @@
+/*
+ * 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.
+ */
+
+#include <map>
+#include <regex>
+#include <string>
+#include <util/Log.hpp>
+
+#include <boost/filesystem.hpp>
+
+#include <gtest/gtest.h>
+
+#include <geode/AuthenticatedView.hpp>
+#include <geode/Cache.hpp>
+#include <geode/PoolManager.hpp>
+#include <geode/RegionFactory.hpp>
+#include <geode/RegionShortcut.hpp>
+
+using apache::geode::client::CacheClosedException;
+using apache::geode::client::CacheFactory;
+using apache::geode::client::LogLevel;
+using apache::geode::client::RegionShortcut;
+
+namespace {
+
+const auto __1K__ = 1024;
+const auto __4K__ = 4 * __1K__;
+const auto __1M__ = (__1K__ * __1K__);
+const auto __1G__ = (__1K__ * __1K__ * __1K__);
+
+const auto LENGTH_OF_BANNER = 16;
+
+auto testLogFileName = std::string("LoggingTest.log");
+
+const char* __1KStringLiteral =
+    "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"
+    "AA"
+    "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"
+    "AA"
+    "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"
+    "AA"
+    "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"
+    "AA"
+    "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"
+    "AA"
+    "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"
+    "AA"
+    "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"
+    "AA"
+    "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"
+    "AA"
+    "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"
+    "AA"
+    "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"
+    "AA"
+    "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"
+    "AA"
+    "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"
+    "AA"
+    "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"
+    "AA"
+    "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA";
+
+class LoggingTest : public testing::Test {
+  void scrubTestLogFiles() {
+    // Close logger, just in case
+    apache::geode::client::Log::close();
+
+    if (boost::filesystem::exists(testLogFileName)) {
+      boost::filesystem::remove(testLogFileName);
+    }
+
+    std::map<int32_t, boost::filesystem::path> rolledFiles;
+    LoggingTest::findRolledFiles(boost::filesystem::current_path().string(),
+                                 rolledFiles);
+    for (auto& item : rolledFiles) {
+      boost::filesystem::remove(item.second);
+    }
+  }
+
+  virtual void SetUp() { scrubTestLogFiles(); }
+
+  virtual void TearDown() { scrubTestLogFiles(); }
+
+ public:
+  static void writeRolledLogFile(const boost::filesystem::path& logdir,
+                                 int32_t rollIndex) {
+    auto rolledPath =
+        logdir / boost::filesystem::path("LoggingTest-" +
+                                         std::to_string(rollIndex) + ".log");
+    auto rolledFile = fopen(rolledPath.string().c_str(), "w");
+    fwrite("Test", 1, 4, rolledFile);
+    fclose(rolledFile);
+  }
+
+  static int numOfLinesInFile(const char* fname) {
+    char line[2048];
+    char* read;
+    int ln_cnt = 0;
+    FILE* fp = fopen(fname, "r");
+    if (fp == nullptr) {
+      return 0;
+    }
+    while (!!(read = fgets(line, sizeof line, fp))) {
+      ++ln_cnt;
+    }
+
+    if (!feof(fp)) {
+      fclose(fp);
+      return -2;
+    }
+    fclose(fp);
+    return ln_cnt;
+  }
+
+  static int expected(LogLevel level) {
+    int expected = static_cast<int>(level);
+    if (level >= LogLevel::Default) {
+      expected--;
+    }
+    return expected;
+  }
+
+  static int expectedWithBanner(LogLevel level) {
+    int expected = LoggingTest::expected(level);
+    if (level != LogLevel::None) {
+      expected += LENGTH_OF_BANNER;
+    }
+    return expected;
+  }
+
+  static void verifyLineCountAtLevel(LogLevel level) {
+    apache::geode::client::Log::init(level, testLogFileName);
+
+    LOGERROR("Error Message");
+    LOGWARN("Warning Message");
+    LOGINFO("Info Message");
+    LOGCONFIG("Config Message");
+    LOGFINE("Fine Message");
+    LOGFINER("Finer Message");
+    LOGFINEST("Finest Message");
+    LOGDEBUG("Debug Message");
+
+    int lines = LoggingTest::numOfLinesInFile(testLogFileName.c_str());
+
+    ASSERT_TRUE(lines == LoggingTest::expectedWithBanner(level));
+
+    apache::geode::client::Log::close();
+    boost::filesystem::remove(testLogFileName.c_str());
+  }
+
+  static void findRolledFiles(
+      const std::string& logFilePath,
+      std::map<int32_t, boost::filesystem::path>& rolledFiles) {
+    const auto basePath =
+        boost::filesystem::absolute(boost::filesystem::path(logFilePath)) /
+        testLogFileName;
+    const auto filterstring = basePath.stem().string() + "-(\\d+)\\.log$";
+    const std::regex my_filter(filterstring);
+
+    rolledFiles.clear();
+
+    boost::filesystem::directory_iterator end_itr;
+    for (boost::filesystem::directory_iterator i(
+             basePath.parent_path().string());
+         i != end_itr; ++i) {
+      if (boost::filesystem::is_regular_file(i->status())) {
+        std::string filename = i->path().filename().string();
+        std::regex testPattern(filterstring);
+        std::match_results<std::string::const_iterator> testMatches;
+        if (std::regex_search(std::string::const_iterator(filename.begin()),
+                              filename.cend(), testMatches, testPattern)) {
+          auto index = std::atoi(
+              std::string(testMatches[1].first, testMatches[1].second).c_str());
+          rolledFiles[index] = i->path();
+        }
+      }
+    }
+  }
+};
+
+/**
+ * Verify we can initialize the logger with any combination of level,
+ * filename, file size limit, and disk space limit
+ */
+TEST_F(LoggingTest, logInit) {
+  // Check all valid levels
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::None, testLogFileName.c_str(), 1, 4));
+  apache::geode::client::Log::close();
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Error, testLogFileName.c_str(), 1, 4));
+  apache::geode::client::Log::close();
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Warning, testLogFileName.c_str(), 1, 4));
+  apache::geode::client::Log::close();
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Info, testLogFileName.c_str(), 1, 4));
+  apache::geode::client::Log::close();
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Default, testLogFileName.c_str(), 1, 4));
+  apache::geode::client::Log::close();
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Config, testLogFileName.c_str(), 1, 4));
+  apache::geode::client::Log::close();
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Fine, testLogFileName.c_str(), 1, 4));
+  apache::geode::client::Log::close();
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Finer, testLogFileName.c_str(), 1, 4));
+  apache::geode::client::Log::close();
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Finest, testLogFileName.c_str(), 1, 4));
+  apache::geode::client::Log::close();
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Debug, testLogFileName.c_str(), 1, 4));
+  apache::geode::client::Log::close();
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::All, testLogFileName.c_str(), 1, 4));
+  apache::geode::client::Log::close();
+
+  // Init with valid filename
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Config, "LoggingTest.log"));
+  apache::geode::client::Log::close();
+
+  // Init with invalid filename
+  ASSERT_THROW(apache::geode::client::Log::init(
+                   apache::geode::client::LogLevel::Config, "#?$?%.log"),
+               apache::geode::client::IllegalArgumentException);
+
+  // Specify disk or file limit without a filename
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Config, nullptr, 4));
+  apache::geode::client::Log::close();
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Config, nullptr, 0, 4));
+  apache::geode::client::Log::close();
+
+  // Specify a disk space limit smaller than the file size limit
+  ASSERT_THROW(
+      apache::geode::client::Log::init(apache::geode::client::LogLevel::Config,
+                                       testLogFileName, __1K__, 4),
+      apache::geode::client::IllegalArgumentException);
+
+  // Specify a file size limit above max allowed
+  ASSERT_THROW(
+      apache::geode::client::Log::init(apache::geode::client::LogLevel::Config,
+                                       testLogFileName, __1G__),
+      apache::geode::client::IllegalArgumentException);
+
+  // Specify a disk space limit above max allowed
+  ASSERT_THROW(
+      apache::geode::client::Log::init(apache::geode::client::LogLevel::Config,
+                                       testLogFileName, 1, __1G__),
+      apache::geode::client::IllegalArgumentException);
+
+  // Init twice without closing
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::All, testLogFileName.c_str(), 1, 4));
+  ASSERT_THROW(
+      apache::geode::client::Log::init(apache::geode::client::LogLevel::All,
+                                       testLogFileName.c_str(), 1, 4),
+      apache::geode::client::IllegalStateException);
+  apache::geode::client::Log::close();
+}
+
+TEST_F(LoggingTest, logToFileAtEachLevel) {
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Debug, testLogFileName));
+  LOGDEBUG("This is a debug string");
+  LOGDEBUG("This is a formatted debug string (%d)", __1K__);
+  apache::geode::client::Log::close();
+  ASSERT_TRUE(boost::filesystem::exists(testLogFileName));
+  ASSERT_TRUE(boost::filesystem::file_size(testLogFileName) > 0);
+  boost::filesystem::remove(testLogFileName);
+  ASSERT_FALSE(boost::filesystem::exists(testLogFileName));
+
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Finest, testLogFileName));
+  LOGFINEST("This is a 'finest' string");
+  LOGFINEST("This is a formatted 'finest' string (%d)", __1K__);
+  apache::geode::client::Log::close();
+  ASSERT_TRUE(boost::filesystem::exists(testLogFileName));
+  ASSERT_TRUE(boost::filesystem::file_size(testLogFileName) > 0);
+  boost::filesystem::remove(testLogFileName);
+
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Finer, testLogFileName));
+  LOGFINER("This is a 'finer' string");
+  LOGFINER("This is a formatted 'finer' string (%d)", __1K__);
+  apache::geode::client::Log::close();
+  ASSERT_TRUE(boost::filesystem::exists(testLogFileName));
+  ASSERT_TRUE(boost::filesystem::file_size(testLogFileName) > 0);
+  boost::filesystem::remove(testLogFileName);
+
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Fine, testLogFileName));
+  LOGFINE("This is a 'fine' string");
+  LOGFINE("This is a formatted 'fine' string (%d)", __1K__);
+  apache::geode::client::Log::close();
+  ASSERT_TRUE(boost::filesystem::exists(testLogFileName));
+  ASSERT_TRUE(boost::filesystem::file_size(testLogFileName) > 0);
+  boost::filesystem::remove(testLogFileName);
+
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Config, testLogFileName));
+  LOGCONFIG("This is a 'config' string");
+  LOGCONFIG("This is a formatted 'config' string (%d)", __1K__);
+  apache::geode::client::Log::close();
+  ASSERT_TRUE(boost::filesystem::exists(testLogFileName));
+  ASSERT_TRUE(boost::filesystem::file_size(testLogFileName) > 0);
+  boost::filesystem::remove(testLogFileName);
+
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Info, testLogFileName));
+  LOGINFO("This is a 'finer' string");
+  LOGINFO("This is a formatted 'finer' string (%d)", __1K__);
+  apache::geode::client::Log::close();
+  ASSERT_TRUE(boost::filesystem::exists(testLogFileName));
+  ASSERT_TRUE(boost::filesystem::file_size(testLogFileName) > 0);
+  boost::filesystem::remove(testLogFileName);
+
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Warning, testLogFileName));
+  LOGWARN("This is a 'warning' string");
+  LOGWARN("This is a formatted 'warning' string (%d)", __1K__);
+  apache::geode::client::Log::close();
+  ASSERT_TRUE(boost::filesystem::exists(testLogFileName));
+  ASSERT_TRUE(boost::filesystem::file_size(testLogFileName) > 0);
+  boost::filesystem::remove(testLogFileName);
+
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Error, testLogFileName));
+  LOGERROR("This is a 'error' string");
+  LOGERROR("This is a formatted 'error' string (%d)", __1K__);
+  apache::geode::client::Log::close();
+  ASSERT_TRUE(boost::filesystem::exists(testLogFileName));
+  ASSERT_TRUE(boost::filesystem::file_size(testLogFileName) > 0);
+  boost::filesystem::remove(testLogFileName);
+}
+
+TEST_F(LoggingTest, verifyFileSizeLimit) {
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Debug, testLogFileName, 1, 5));
+  for (auto i = 0; i < 4 * __1K__; i++) {
+    LOGDEBUG(__1KStringLiteral);
+  }
+  apache::geode::client::Log::close();
+
+  // Original file should still be around
+  ASSERT_TRUE(boost::filesystem::exists(testLogFileName));
+
+  // Check for 'rolled' log files.  With a 1MB file size limit and each logged
+  // string having a length of 1K chars, we should have at least one less
+  // rolled log file than the number of strings logged, i.e. 3 rolled files
+  // for 4K strings in this case.  spdlog rolled files look like
+  // <<basename>>.<<#>>.<<extension>>, so for LoggingTest.log we should find
+  // LoggingTest.1.log, LoggingTest.2.log, etc.
+  auto base = boost::filesystem::path(testLogFileName).stem();
+  auto ext = boost::filesystem::path(testLogFileName).extension();
+
+  // File size limit is treated as a "soft" limit.  If the last message in the
+  // log puts the file size over the limit, the file is rolled and the message
+  // is preserved intact, rather than truncated or split across files.  We'll
+  // assume the file size never exceeds 110% of the specified limit.
+  auto adjustedFileSizeLimit =
+      static_cast<uint32_t>(static_cast<uint64_t>(__1M__) * 11 / 10);
+
+  for (auto i = 0; i < 4; i++) {
+    auto rolledLogFileName =
+        base.string() + "-" + std::to_string(i) + ext.string();
+
+    ASSERT_TRUE(boost::filesystem::exists(rolledLogFileName));
+    ASSERT_TRUE(adjustedFileSizeLimit >
+                boost::filesystem::file_size(rolledLogFileName));
+  }
+}
+
+TEST_F(LoggingTest, verifyDiskSpaceLimit) {
+  const int NUMBER_OF_ITERATIONS = 4 * __1K__;
+  const int DISK_SPACE_LIMIT = 2 * __1M__;
+
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Debug, testLogFileName, 1, 2));
+  for (auto i = 0; i < NUMBER_OF_ITERATIONS; i++) {
+    LOGDEBUG(__1KStringLiteral);
+  }
+  apache::geode::client::Log::close();
+
+  // Original file should still be around
+  ASSERT_TRUE(boost::filesystem::exists(testLogFileName));
+
+  auto size = boost::filesystem::file_size(testLogFileName);
+  auto numRolledFilesFound = 0;
+  auto base = boost::filesystem::path(testLogFileName).stem();
+  auto ext = boost::filesystem::path(testLogFileName).extension();
+
+  // We wrote 4x the log file limit, and 2x the disk space limit, so
+  // there should be one 'rolled' file.  Its name should be of the form
+  // <base>-n.log, where n is some reasonable number.
+  std::map<int32_t, boost::filesystem::path> rolledFiles;
+  LoggingTest::findRolledFiles(boost::filesystem::current_path().string(),
+                               rolledFiles);
+  ASSERT_TRUE(rolledFiles.size() == 1);
+
+  auto rolledFile = rolledFiles.begin()->second;
+  size += boost::filesystem::file_size(rolledFile);
+
+  ASSERT_TRUE(size <= DISK_SPACE_LIMIT);
+}
+
+TEST_F(LoggingTest, verifyWithExistingRolledFile) {
+  LoggingTest::writeRolledLogFile(boost::filesystem::current_path(), 11);
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Debug, testLogFileName, 1, 5));
+  for (auto i = 0; i < 2 * __1K__; i++) {
+    LOGDEBUG(__1KStringLiteral);
+  }
+  apache::geode::client::Log::close();
+
+  // Original file should still be around
+  ASSERT_TRUE(boost::filesystem::exists(testLogFileName));
+
+  // Check for 'rolled' log files.  With a 1MB file size limit and each logged
+  // string having a length of 1K chars, we should have at least one less
+  // rolled log file than the number of strings logged, i.e. 3 rolled files
+  // for 4K strings in this case.  spdlog rolled files look like
+  // <<basename>>.<<#>>.<<extension>>, so for LoggingTest.log we should find
+  // LoggingTest.1.log, LoggingTest.2.log, etc.
+  auto base = boost::filesystem::path(testLogFileName).stem();
+  auto ext = boost::filesystem::path(testLogFileName).extension();
+
+  // File size limit is treated as a "soft" limit.  If the last message in the
+  // log puts the file size over the limit, the file is rolled and the message
+  // is preserved intact, rather than truncated or split across files.  We'll
+  // assume the file size never exceeds 110% of the specified limit.
+  auto adjustedFileSizeLimit =
+      static_cast<uint32_t>(static_cast<uint64_t>(__1M__) * 11 / 10);
+
+  auto rolledLogFileName =
+      base.string() + "-" + std::to_string(12) + ext.string();
+
+  ASSERT_TRUE(boost::filesystem::exists(rolledLogFileName));
+  ASSERT_TRUE(adjustedFileSizeLimit >
+              boost::filesystem::file_size(rolledLogFileName));
+}
+
+void verifyWithPath(const boost::filesystem::path& path, int32_t fileSizeLimit,
+                    int64_t diskSpaceLimit) {
+  auto relativePath = path / boost::filesystem::path(testLogFileName);
+
+  ASSERT_NO_THROW(apache::geode::client::Log::init(
+      apache::geode::client::LogLevel::Debug, relativePath.string(),
+      fileSizeLimit, diskSpaceLimit));
+  for (auto i = 0; i < ((3 * fileSizeLimit) / 2) * __1K__; i++) {
+    LOGDEBUG(__1KStringLiteral);
+  }
+  apache::geode::client::Log::close();
+
+  // Original file should still be around
+  ASSERT_TRUE(boost::filesystem::exists(relativePath));
+
+  // Check for 'rolled' log files.  With a 1MB file size limit and each logged
+  // string having a length of 1K chars, we should have at least one less
+  // rolled log file than the number of strings logged, i.e. 3 rolled files
+  // for 4K strings in this case.  spdlog rolled files look like
+  // <<basename>>.<<#>>.<<extension>>, so for LoggingTest.log we should find
+  // LoggingTest.1.log, LoggingTest.2.log, etc.
+  auto base = boost::filesystem::path(relativePath).stem();
+  auto ext = boost::filesystem::path(relativePath).extension();
+
+  // File size limit is treated as a "soft" limit.  If the last message in the
+  // log puts the file size over the limit, the file is rolled and the message
+  // is preserved intact, rather than truncated or split across files.  We'll
+  // assume the file size never exceeds 110% of the specified limit.
+  auto adjustedFileSizeLimit = static_cast<uint32_t>(
+      static_cast<uint64_t>(__1M__ * fileSizeLimit) * 11 / 10);
+
+  auto rolledLogFileName =
+      relativePath.parent_path() /
+      boost::filesystem::path(base.string() + "-" + std::to_string(0) +
+                              ext.string());
+
+  if (fileSizeLimit == diskSpaceLimit) {
+    // If the limits are equal, we should *never* roll logs, just delete the
+    // current file and start over
+    ASSERT_FALSE(boost::filesystem::exists(rolledLogFileName));
+  } else {
+    ASSERT_TRUE(boost::filesystem::exists(rolledLogFileName));
+    ASSERT_TRUE(adjustedFileSizeLimit >
+                boost::filesystem::file_size(rolledLogFileName));
+  }
+  ASSERT_TRUE(adjustedFileSizeLimit >
+              boost::filesystem::file_size(relativePath));
+}
+
+TEST_F(LoggingTest, verifyWithRelativePathFromCWD) {
+  auto relativePath = boost::filesystem::path("foo/bar");
+
+  verifyWithPath(relativePath, 1, 5);
+
+  boost::filesystem::remove_all(boost::filesystem::path("foo"));
+}
+
+TEST_F(LoggingTest, verifyWithAbsolutePath) {
+  auto absolutePath =
+      boost::filesystem::absolute(boost::filesystem::path("foo/bar"));
+
+  verifyWithPath(absolutePath, 1, 5);
+
+  boost::filesystem::remove_all(boost::filesystem::path("foo"));
+}
+
+TEST_F(LoggingTest, setLimitsEqualAndRoll) {
+  verifyWithPath(boost::filesystem::path(), 1, 1);
+}
+
+// Logger is supposed to tack the '.log' extension on any file that doesn't
+// already have it.
+TEST_F(LoggingTest, verifyExtension) {
+  apache::geode::client::Log::init(LogLevel::All, "foo");
+  LOGINFO("...");
+  apache::geode::client::Log::close();
+  ASSERT_TRUE(LoggingTest::numOfLinesInFile("foo.log") > 0);
+  boost::filesystem::remove("foo.log");
+
+  apache::geode::client::Log::init(LogLevel::All, "foo.txt");
+  LOGINFO("...");
+  apache::geode::client::Log::close();
+  ASSERT_TRUE(LoggingTest::numOfLinesInFile("foo.txt.log") > 0);
+  boost::filesystem::remove("foo.txt.log");
+}
+
+// Old version of logger didn't distinguish between rolled log file and
+// filename containing '-', so would crash in an atoi() call if you used
+// '-' in your log file name.
+TEST_F(LoggingTest, verifyFilenameWithDash) {
+  apache::geode::client::Log::init(LogLevel::All, "foo-bar.log");
+  LOGINFO("...");
+  apache::geode::client::Log::close();
+  ASSERT_TRUE(LoggingTest::numOfLinesInFile("foo-bar.log") > 0);
+  boost::filesystem::remove("foo-bar.log");
+}
+
+TEST_F(LoggingTest, countLinesAllLevels) {
+  for (LogLevel level : {
+           LogLevel::Error,
+           LogLevel::Warning,
+           LogLevel::Info,
+           LogLevel::Default,
+           LogLevel::Config,
+           LogLevel::Fine,
+           LogLevel::Finer,
+           LogLevel::Finest,
+           LogLevel::Debug,
+       }) {
+    apache::geode::client::Log::init(level, testLogFileName);
+
+    LOGERROR("Error Message");
+    LOGWARN("Warning Message");
+    LOGINFO("Info Message");
+    LOGCONFIG("Config Message");
+    LOGFINE("Fine Message");
+    LOGFINER("Finer Message");
+    LOGFINEST("Finest Message");
+    LOGDEBUG("Debug Message");
+
+    int lines = LoggingTest::numOfLinesInFile(testLogFileName.c_str());
+
+    ASSERT_TRUE(lines == LoggingTest::expectedWithBanner(level));
+
+    apache::geode::client::Log::close();
+    boost::filesystem::remove(testLogFileName);
+  }
+}
+
+TEST_F(LoggingTest, countLinesConfigOnwards) {
+  verifyLineCountAtLevel(LogLevel::Config);
+}
+
+TEST_F(LoggingTest, countLinesInfoOnwards) {
+  verifyLineCountAtLevel(LogLevel::Info);
+}
+
+TEST_F(LoggingTest, countLinesWarningOnwards) {
+  verifyLineCountAtLevel(LogLevel::Warning);
+}
+
+TEST_F(LoggingTest, countLinesErrorOnly) {
+  verifyLineCountAtLevel(LogLevel::Error);
+}
+
+TEST_F(LoggingTest, countLinesNone) { verifyLineCountAtLevel(LogLevel::None); }
+
+}  // namespace