You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by jb...@apache.org on 2018/03/15 13:14:12 UTC

[geode-native] branch develop updated: GEODE-4843: Removes remaining host stats files and globals. (#237)

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

jbarrett 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 f56c472  GEODE-4843: Removes remaining host stats files and globals. (#237)
f56c472 is described below

commit f56c4722266f71e998f37fe4d99f355b61937334
Author: Jacob Barrett <jb...@pivotal.io>
AuthorDate: Thu Mar 15 06:14:10 2018 -0700

    GEODE-4843: Removes remaining host stats files and globals. (#237)
---
 cppcache/src/statistics/GeodeStatisticsFactory.cpp |   1 -
 cppcache/src/statistics/HostStatHelper.cpp         | 110 --------
 cppcache/src/statistics/HostStatHelper.hpp         |  88 ------
 cppcache/src/statistics/HostStatHelperLinux.hpp    |  61 -----
 cppcache/src/statistics/HostStatHelperNull.cpp     |  18 --
 cppcache/src/statistics/HostStatHelperNull.hpp     |  40 ---
 cppcache/src/statistics/HostStatHelperSolaris.hpp  |  76 -----
 cppcache/src/statistics/HostStatHelperWin.hpp      | 305 ---------------------
 cppcache/src/statistics/HostStatSampler.cpp        |  10 +-
 cppcache/src/statistics/HostStatSampler.hpp        |  16 +-
 cppcache/src/statistics/NullProcessStats.cpp       |  42 ---
 cppcache/src/statistics/NullProcessStats.hpp       |  61 -----
 cppcache/src/statistics/PoolStatsSampler.cpp       |   3 -
 13 files changed, 2 insertions(+), 829 deletions(-)

diff --git a/cppcache/src/statistics/GeodeStatisticsFactory.cpp b/cppcache/src/statistics/GeodeStatisticsFactory.cpp
index ebad568..98e97b0 100644
--- a/cppcache/src/statistics/GeodeStatisticsFactory.cpp
+++ b/cppcache/src/statistics/GeodeStatisticsFactory.cpp
@@ -30,7 +30,6 @@
 #include "../util/Log.hpp"
 #include "AtomicStatisticsImpl.hpp"
 #include "OsStatisticsImpl.hpp"
-#include "HostStatHelper.hpp"
 
 namespace apache {
 namespace geode {
diff --git a/cppcache/src/statistics/HostStatHelper.cpp b/cppcache/src/statistics/HostStatHelper.cpp
deleted file mode 100644
index df76a14..0000000
--- a/cppcache/src/statistics/HostStatHelper.cpp
+++ /dev/null
@@ -1,110 +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 <ace/OS_NS_sys_utsname.h>
-
-#include <geode/internal/geode_globals.hpp>
-
-#include "config.h"
-#include "HostStatHelper.hpp"
-#include "GeodeStatisticsFactory.hpp"
-#include "../Assert.hpp"
-
-namespace apache {
-namespace geode {
-namespace statistics {
-
-int32_t HostStatHelper::PROCESS_STAT_FLAG = 1;
-int32_t HostStatHelper::SYSTEM_STAT_FLAG = 2;
-GFS_OSTYPES HostStatHelper::osCode =
-    static_cast<GFS_OSTYPES>(0);  // Default OS is Linux
-ProcessStats* HostStatHelper::processStats = nullptr;
-
-void HostStatHelper::initOSCode() {
-  ACE_utsname u;
-  ACE_OS::uname(&u);
-  std::string osName(u.sysname);
-
-  if (osName == "Linux") {
-    osCode = GFS_OSTYPE_LINUX;
-  } else if ((osName == "Windows") || (osName == "Win32")) {
-    osCode = GFS_OSTYPE_WINDOWS;
-  } else if (osName == "SunOS") {
-    osCode = GFS_OSTYPE_SOLARIS;
-  } else if (osName == "Darwin") {
-    osCode = GFS_OSTYPE_MACOSX;
-  } else {
-    char buf[1024] = {0};
-    ACE_OS::snprintf(buf, 1024,
-                     "HostStatHelper::initOSTypes:unhandled os type: %s",
-                     osName.c_str());
-    throw IllegalArgumentException(buf);
-  }
-}
-
-void HostStatHelper::refresh() {
-  if (processStats != nullptr) {
-    HostStatHelperNull::refreshProcess(processStats);
-  }
-}
-
-void HostStatHelper::newProcessStats(GeodeStatisticsFactory* statisticsFactory,
-                                     int64_t pid, const char* name) {
-  // Init OsCode
-  initOSCode();
-
-  processStats = new NullProcessStats(pid, name);
-  GF_D_ASSERT(processStats != nullptr);
-}
-
-void HostStatHelper::close() {
-  if (processStats) {
-    processStats->close();
-  }
-}
-
-void HostStatHelper::cleanup() {
-  if (processStats) {
-    delete processStats;
-    processStats = nullptr;
-  }
-}
-
-int32_t HostStatHelper::getCpuUsage() {
-  if (HostStatHelper::processStats != nullptr) {
-    return HostStatHelper::processStats->getCpuUsage();
-  }
-  return 0;
-}
-
-int64_t HostStatHelper::getCpuTime() {
-  if (HostStatHelper::processStats != nullptr) {
-    return HostStatHelper::processStats->getAllCpuTime();
-  }
-  return 0;
-}
-
-int32_t HostStatHelper::getNumThreads() {
-  if (HostStatHelper::processStats != nullptr) {
-    return HostStatHelper::processStats->getNumThreads();
-  }
-  return 0;
-}
-
-}  // namespace statistics
-}  // namespace geode
-}  // namespace apache
diff --git a/cppcache/src/statistics/HostStatHelper.hpp b/cppcache/src/statistics/HostStatHelper.hpp
deleted file mode 100644
index 0522244..0000000
--- a/cppcache/src/statistics/HostStatHelper.hpp
+++ /dev/null
@@ -1,88 +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.
- */
-
-#pragma once
-
-#ifndef GEODE_STATISTICS_HOSTSTATHELPER_H_
-#define GEODE_STATISTICS_HOSTSTATHELPER_H_
-
-#include <string>
-
-#include <geode/internal/geode_globals.hpp>
-
-#include "StatisticDescriptorImpl.hpp"
-#include "StatisticsType.hpp"
-#include "Statistics.hpp"
-#include "StatisticDescriptor.hpp"
-#include "ProcessStats.hpp"
-#include "StatisticsFactory.hpp"
-#include "OsStatisticsImpl.hpp"
-#include "StatsDef.hpp"
-#include "HostStatHelperWin.hpp"
-#include "HostStatHelperLinux.hpp"
-#include "HostStatHelperSolaris.hpp"
-#include "HostStatHelperNull.hpp"
-#include "NullProcessStats.hpp"
-
-// TODO refactor - conditionally include os specific impl headers.
-
-/** @file
- */
-
-namespace apache {
-namespace geode {
-namespace statistics {
-
-class GeodeStatisticsFactory;
-
-/**
- * Provides native methods which fetch operating system statistics.
- * accessed by calling {@link #getInstance()}.
- */
-
-class _GEODE_EXPORT HostStatHelper {
- private:
-  static int32_t PROCESS_STAT_FLAG;
-
-  static int32_t SYSTEM_STAT_FLAG;
-
-  static GFS_OSTYPES osCode;
-
-  static ProcessStats* processStats;
-
-  static void initOSCode();
-
- public:
-  static int32_t getCpuUsage();
-  static int64_t getCpuTime();
-
-  static int32_t getNumThreads();
-
-  static void refresh();
-
-  static void newProcessStats(GeodeStatisticsFactory* statisticsFactory,
-                              int64_t pid, const char* name);
-
-  static void close();
-
-  static void cleanup();
-};
-}  // namespace statistics
-}  // namespace geode
-}  // namespace apache
-
-#endif  // GEODE_STATISTICS_HOSTSTATHELPER_H_
diff --git a/cppcache/src/statistics/HostStatHelperLinux.hpp b/cppcache/src/statistics/HostStatHelperLinux.hpp
deleted file mode 100644
index 4bed14f..0000000
--- a/cppcache/src/statistics/HostStatHelperLinux.hpp
+++ /dev/null
@@ -1,61 +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.
- */
-
-#pragma once
-
-#ifndef GEODE_STATISTICS_HOSTSTATHELPERLINUX_H_
-#define GEODE_STATISTICS_HOSTSTATHELPERLINUX_H_
-
-#include "config.h"
-
-#if defined(_LINUX)
-
-#include <string>
-
-#include <sys/sysinfo.h>
-
-#include <geode/internal/geode_globals.hpp>
-
-#include "ProcessStats.hpp"
-
-/** @file
- */
-
-namespace apache {
-namespace geode {
-namespace statistics {
-
-/**
- * Linux Implementation to fetch operating system stats.
- *
- */
-
-class HostStatHelperLinux {
- public:
-  static void refreshProcess(ProcessStats* processStats);
-  // static refreeshSystem(Statistics* stats);
-
- private:
-  static uint8_t m_logStatErrorCountDown;
-};
-}  // namespace statistics
-}  // namespace geode
-}  // namespace apache
-
-#endif  // if def(_LINUX)
-
-#endif  // GEODE_STATISTICS_HOSTSTATHELPERLINUX_H_
diff --git a/cppcache/src/statistics/HostStatHelperNull.cpp b/cppcache/src/statistics/HostStatHelperNull.cpp
deleted file mode 100644
index 6ef35ef..0000000
--- a/cppcache/src/statistics/HostStatHelperNull.cpp
+++ /dev/null
@@ -1,18 +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 <geode/internal/geode_globals.hpp>
diff --git a/cppcache/src/statistics/HostStatHelperNull.hpp b/cppcache/src/statistics/HostStatHelperNull.hpp
deleted file mode 100644
index 41c1fc7..0000000
--- a/cppcache/src/statistics/HostStatHelperNull.hpp
+++ /dev/null
@@ -1,40 +0,0 @@
-#pragma once
-
-#ifndef GEODE_STATISTICS_HOSTSTATHELPERNULL_H_
-#define GEODE_STATISTICS_HOSTSTATHELPERNULL_H_
-
-/*
- * 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 <geode/internal/geode_globals.hpp>
-
-/** @file
- */
-
-namespace apache {
-namespace geode {
-namespace statistics {
-
-class HostStatHelperNull {
- public:
-  static void refreshProcess(ProcessStats* processStats) {}
-};
-}  // namespace client
-}  // namespace geode
-}  // namespace apache
-
-#endif  // GEODE_STATISTICS_HOSTSTATHELPERNULL_H_
diff --git a/cppcache/src/statistics/HostStatHelperSolaris.hpp b/cppcache/src/statistics/HostStatHelperSolaris.hpp
deleted file mode 100644
index ebf6d72..0000000
--- a/cppcache/src/statistics/HostStatHelperSolaris.hpp
+++ /dev/null
@@ -1,76 +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.
- */
-
-#pragma once
-
-#ifndef GEODE_STATISTICS_HOSTSTATHELPERSOLARIS_H_
-#define GEODE_STATISTICS_HOSTSTATHELPERSOLARIS_H_
-
-#include "config.h"
-
-#if defined(_SOLARIS)
-
-#include <string>
-#include <sys/sysinfo.h>
-#include <kstat.h>
-
-#include <geode/internal/geode_globals.hpp>
-
-#include "ProcessStats.hpp"
-
-/*
- * CPU_USAGE_STAT_THRESHOLD sets how much time must pass between samples
- * before a new cpu utilization is calculated.
- * Units are in 100ths of a second.
- * if set too low you will get divide by zero overflows or scewed data
- * due to rounding errors.
- * This is likely unnecesary with our stat sampling interval being 1 second.
- */
-#define CPU_USAGE_STAT_THRESHOLD 10
-
-/** @file
- */
-
-namespace apache {
-namespace geode {
-namespace statistics {
-
-/**
- * Solaris Implementation to fetch operating system stats.
- *
- */
-
-class HostStatHelperSolaris {
- public:
-  static void refreshProcess(ProcessStats* processStats);
-  static void closeHostStatHelperSolaris();
-  // static refreeshSystem(Statistics* stats);
-
- private:
-  static uint8_t m_logStatErrorCountDown;
-  static kstat_ctl_t* m_kstat;
-  static uint32_t m_cpuUtilPrev[CPU_STATES];
-  static bool m_initialized;
-  static void getKernelStats(uint32_t*);
-};
-}  // namespace statistics
-}  // namespace geode
-}  // namespace apache
-
-#endif  // if def(_SOLARIS)
-
-#endif  // GEODE_STATISTICS_HOSTSTATHELPERSOLARIS_H_
diff --git a/cppcache/src/statistics/HostStatHelperWin.hpp b/cppcache/src/statistics/HostStatHelperWin.hpp
deleted file mode 100644
index bd308dc..0000000
--- a/cppcache/src/statistics/HostStatHelperWin.hpp
+++ /dev/null
@@ -1,305 +0,0 @@
-#pragma once
-
-#ifndef GEODE_STATISTICS_HOSTSTATHELPERWIN_H_
-#define GEODE_STATISTICS_HOSTSTATHELPERWIN_H_
-
-/*
- * 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.
- */
-
-#if defined(_WIN32)
-
-#include <geode/internal/geode_globals.hpp>
-#include <string>
-#include <Windows.h>
-#include <WinPerf.h>
-
-#include <cstdio>
-#include <cstdlib>
-#include <string>
-#include "ProcessStats.hpp"
-
-/** @file
- */
-
-namespace apache {
-namespace geode {
-namespace statistics {
-
-/**
- * Windows2000 Implementation of code to fetch operating system stats.
- */
-
-class HostStatHelperWin {
- private:
-#define LEVEL1_QUERY_STRING "2 4 230 238 260"
-  static PPERF_DATA_BLOCK PerfData;
-  static PPERF_OBJECT_TYPE ProcessObj;
-  static PPERF_OBJECT_TYPE ProcessorObj;
-  static PPERF_OBJECT_TYPE MemoryObj;
-  static PPERF_OBJECT_TYPE SystemObj;
-  static PPERF_OBJECT_TYPE ObjectsObj;
-  static DWORD BufferSize;
-  static int32_t pidCtrOffset;
-
-  /* #define NTDBG 1 */
-
-  enum {
-    SYSTEM_OBJ_ID = 2,
-    MEMORY_OBJ_ID = 4,
-    PROCESS_OBJ_ID = 230,
-    PROCESSOR_OBJ_ID = 238,
-    OBJECTS_OBJ_ID = 260
-  };
-
-  enum {
-    PID_ID = 784,
-    PROCESSORTIME_ID = 6,
-    USERTIME_ID = 142,
-    PRIVILEGEDTIME_ID = 144,
-    VIRTUALBYTESPEAK_ID = 172,
-    VIRTUALBYTES_ID = 174,
-    PAGEFAULTS_ID = 28,
-    WORKINGSETPEAK_ID = 178,
-    WORKINGSET_ID = 180,
-    PAGEFILEBYTESPEAK_ID = 182,
-    PAGEFILEBYTES_ID = 184,
-    PRIVATEBYTES_ID = 186,
-    THREADCOUNT_ID = 680,
-    PRIORITYBASE_ID = 682,
-    /* ELAPSEDTIME_ID = 684, */
-    POOLPAGEDBYTES_ID = 56,
-    POOLNONPAGEDBYTES_ID = 58,
-    HANDLECOUNT_ID = 952
-  };
-  enum {
-    PROCESSORTIME_IDX = 0,
-    USERTIME_IDX,
-    PRIVILEGEDTIME_IDX,
-    VIRTUALBYTESPEAK_IDX,
-    VIRTUALBYTES_IDX,
-    PAGEFAULTS_IDX,
-    WORKINGSETPEAK_IDX,
-    WORKINGSET_IDX,
-    PAGEFILEBYTESPEAK_IDX,
-    PAGEFILEBYTES_IDX,
-    PRIVATEBYTES_IDX,
-    THREADCOUNT_IDX,
-    PRIORITYBASE_IDX,
-    POOLPAGEDBYTES_IDX,
-    POOLNONPAGEDBYTES_IDX,
-    HANDLECOUNT_IDX,
-    MAX_PROCESS_CTRS_COLLECTED
-  };
-  static PERF_COUNTER_DEFINITION processCtrCache[MAX_PROCESS_CTRS_COLLECTED];
-
-  enum {
-    /* PROCESSORTIME_ID = 6, */
-    /* USERTIME_ID = 142, */
-    /* PRIVILEGEDTIME_ID = 144, */
-    INTERRUPTS_ID = 148,
-    INTERRUPTTIME_ID = 698
-  };
-
-  enum {
-    TOTALPROCESSORTIME_IDX = 0,
-    TOTALUSERTIME_IDX,
-    TOTALPRIVILEGEDTIME_IDX,
-    INTERRUPTS_IDX,
-    INTERRUPTTIME_IDX,
-    MAX_PROCESSOR_CTRS_COLLECTED
-  };
-
-  static PERF_COUNTER_DEFINITION
-      processorCtrCache[MAX_PROCESSOR_CTRS_COLLECTED];
-
-  enum {
-    TOTALFILEREADOPS_ID = 10,
-    TOTALFILEWRITEOPS_ID = 12,
-    TOTALFILECONTROLOPS_ID = 14,
-    TOTALFILEREADKBYTES_ID = 16,
-    TOTALFILEWRITEKBYTES_ID = 18,
-    TOTALFILECONTROLKBYTES_ID = 20,
-    TOTALCONTEXTSWITCHES_ID = 146,
-    TOTALSYSTEMCALLS_ID = 150,
-    TOTALFILEDATAOPS_ID = 406,
-    /* SYSTEMUPTIME_ID = 674, */
-    PROCESSORQUEUELENGTH_ID = 44,
-    ALIGNMENTFIXUPS_ID = 686,
-    EXCEPTIONDISPATCHES_ID = 688,
-    FLOATINGEMULATIONS_ID = 690,
-    REGISTRYQUOTAINUSE_ID = 1350
-  };
-  enum {
-    TOTALFILEREADOPS_IDX = 0,
-    TOTALFILEWRITEOPS_IDX,
-    TOTALFILECONTROLOPS_IDX,
-    TOTALFILEREADKBYTES_IDX,
-    TOTALFILEWRITEKBYTES_IDX,
-    TOTALFILECONTROLKBYTES_IDX,
-    TOTALCONTEXTSWITCHES_IDX,
-    TOTALSYSTEMCALLS_IDX,
-    TOTALFILEDATAOPS_IDX,
-    PROCESSORQUEUELENGTH_IDX,
-    ALIGNMENTFIXUPS_IDX,
-    EXCEPTIONDISPATCHES_IDX,
-    FLOATINGEMULATIONS_IDX,
-    REGISTRYQUOTAINUSE_IDX,
-    MAX_SYSTEM_CTRS_COLLECTED
-  };
-  static PERF_COUNTER_DEFINITION systemCtrCache[MAX_SYSTEM_CTRS_COLLECTED];
-
-  enum {
-    AVAILABLEBYTES_ID = 24,
-    COMMITTEDBYTES_ID = 26,
-    COMMITLIMIT_ID = 30,
-    TOTALPAGEFAULTS_ID = 28,
-    WRITECOPIES_ID = 32,
-
-    TRANSITIONFAULTS_ID = 34,
-    CACHEFAULTS_ID = 36,
-    DEMANDZEROFAULTS_ID = 38,
-    PAGES_ID = 40,
-    PAGESINPUT_ID = 822,
-    PAGEREADS_ID = 42,
-    PAGESOUTPUT_ID = 48,
-    PAGEWRITES_ID = 50,
-    TOTALPOOLPAGEDBYTES_ID = 56,
-    TOTALPOOLNONPAGEDBYTES_ID = 58,
-    POOLPAGEDALLOCS_ID = 60,
-    POOLNONPAGEDALLOCS_ID = 64,
-    FREESYSTEMPAGETABLEENTRIES_ID = 678,
-    CACHEBYTES_ID = 818,
-    CACHEBYTESPEAK_ID = 820,
-    POOLPAGEDRESIDENTBYTES_ID = 66,
-    SYSTEMCODETOTALBYTES_ID = 68,
-    SYSTEMCODERESIDENTBYTES_ID = 70,
-    SYSTEMDRIVERTOTALBYTES_ID = 72,
-    SYSTEMDRIVERRESIDENTBYTES_ID = 74,
-    SYSTEMCACHERESIDENTBYTES_ID = 76,
-    COMMITTEDBYTESINUSE_ID = 1406
-  };
-  enum {
-    AVAILABLEBYTES_IDX = 0,
-    COMMITTEDBYTES_IDX,
-    COMMITLIMIT_IDX,
-    TOTALPAGEFAULTS_IDX,
-    WRITECOPIES_IDX,
-    TRANSITIONFAULTS_IDX,
-    CACHEFAULTS_IDX,
-    DEMANDZEROFAULTS_IDX,
-    PAGES_IDX,
-    PAGESINPUT_IDX,
-    PAGEREADS_IDX,
-    PAGESOUTPUT_IDX,
-    PAGEWRITES_IDX,
-    TOTALPOOLPAGEDBYTES_IDX,
-    TOTALPOOLNONPAGEDBYTES_IDX,
-    POOLPAGEDALLOCS_IDX,
-    POOLNONPAGEDALLOCS_IDX,
-    FREESYSTEMPAGETABLEENTRIES_IDX,
-    CACHEBYTES_IDX,
-    CACHEBYTESPEAK_IDX,
-    POOLPAGEDRESIDENTBYTES_IDX,
-    SYSTEMCODETOTALBYTES_IDX,
-    SYSTEMCODERESIDENTBYTES_IDX,
-    SYSTEMDRIVERTOTALBYTES_IDX,
-    SYSTEMDRIVERRESIDENTBYTES_IDX,
-    SYSTEMCACHERESIDENTBYTES_IDX,
-    COMMITTEDBYTESINUSE_IDX,
-    MAX_MEMORY_CTRS_COLLECTED
-  };
-  static PERF_COUNTER_DEFINITION memoryCtrCache[MAX_MEMORY_CTRS_COLLECTED];
-
-  enum {
-    PROCESSES_ID = 248,
-    THREADS_ID = 250,
-    EVENTS_ID = 252,
-    SEMAPHORES_ID = 254,
-    MUTEXES_ID = 256,
-    SECTIONS_ID = 258
-  };
-  enum {
-    PROCESSES_IDX = 0,
-    THREADS_IDX,
-    EVENTS_IDX,
-    SEMAPHORES_IDX,
-    MUTEXES_IDX,
-    SECTIONS_IDX,
-    MAX_OBJECTS_CTRS_COLLECTED
-  };
-  static PERF_COUNTER_DEFINITION objectsCtrCache[MAX_OBJECTS_CTRS_COLLECTED];
-
-  struct FetchDataSType {
-    uint32_t perfTimeMs;
-    int64_t usertime;
-    int64_t systime;
-    int64_t idletime;
-    int64_t inttime;
-    uint32_t interrupts;
-  };
-
-  static FetchDataSType lastFetchData;
-  static FetchDataSType currentFetchData;
-
- private:
-  static void HostStatsFetchData();
-
-  static int32_t getPid(int32_t pidCtrOffset, PPERF_COUNTER_BLOCK PerfCntrBlk);
-
-  static uint32_t getInt32Value(PPERF_COUNTER_DEFINITION PerfCntr,
-                                PPERF_COUNTER_BLOCK PerfCntrBlk);
-
-  static int64_t getInt64Value(PPERF_COUNTER_DEFINITION PerfCntr,
-                               PPERF_COUNTER_BLOCK PerfCntrBlk,
-                               bool convertMS = true);
-
-  static PPERF_OBJECT_TYPE FirstObject(PPERF_DATA_BLOCK PerfData);
-
-  static PPERF_OBJECT_TYPE NextObject(PPERF_OBJECT_TYPE PerfObj);
-
-  static PPERF_INSTANCE_DEFINITION FirstInstance(PPERF_OBJECT_TYPE PerfObj);
-
-  static PPERF_INSTANCE_DEFINITION NextInstance(
-      PPERF_COUNTER_BLOCK PerfCntrBlk);
-
-  static PPERF_COUNTER_DEFINITION FirstCounter(PPERF_OBJECT_TYPE PerfObj);
-
-  static PPERF_COUNTER_DEFINITION NextCounter(
-      PPERF_COUNTER_DEFINITION PerfCntr);
-
-  static char* getInstIdStr(PPERF_INSTANCE_DEFINITION PerfInst, char* prefix);
-
-  static int calculateCpuUsage(PPERF_COUNTER_BLOCK& ctrBlk);
-
- public:
-  static void initHostStatHelperWin();
-
-  static void refreshProcess(ProcessStats* processStats);
-
-  static void closeHostStatHelperWin();
-
-  // static refreeshSystem(Statistics* stats);
-
-};  // class
-
-}  // namespace statistics
-}  // namespace geode
-}  // namespace apache
-
-#endif  // (_WIN32)
-
-#endif  // GEODE_STATISTICS_HOSTSTATHELPERWIN_H_
diff --git a/cppcache/src/statistics/HostStatSampler.cpp b/cppcache/src/statistics/HostStatSampler.cpp
index 5064684..97346a9 100644
--- a/cppcache/src/statistics/HostStatSampler.cpp
+++ b/cppcache/src/statistics/HostStatSampler.cpp
@@ -34,7 +34,6 @@
 #include <geode/SystemProperties.hpp>
 
 #include "HostStatSampler.hpp"
-#include "HostStatHelper.hpp"
 #include "StatArchiveWriter.hpp"
 #include "GeodeStatisticsFactory.hpp"
 #include "../util/Log.hpp"
@@ -513,16 +512,11 @@ int32_t HostStatSampler::rollArchive(std::string filename) {
 
 void HostStatSampler::initSpecialStats() {
   // After Special categories are decided initialize them here
-  HostStatHelper::newProcessStats(m_statMngr->getStatisticsFactory(), m_pid,
-                                  "ProcessStats");
 }
 
-void HostStatSampler::sampleSpecialStats() { HostStatHelper::refresh(); }
+void HostStatSampler::sampleSpecialStats() {  }
 
 void HostStatSampler::closeSpecialStats() {
-  ACE_Guard<ACE_Recursive_Thread_Mutex> guard(m_statMngr->getListMutex());
-  HostStatHelper::close();
-  HostStatHelper::cleanup();
 }
 
 void HostStatSampler::checkListeners() {}
@@ -581,8 +575,6 @@ void HostStatSampler::putStatsInAdminRegion() {
               puts += creates;
             }
           }
-          numThreads = HostStatHelper::getNumThreads();
-          cpuTime = HostStatHelper::getCpuTime();
         }
         static int numCPU = ACE_OS::num_processors();
         auto obj = ClientHealthStats::create(gets, puts, misses, numListeners,
diff --git a/cppcache/src/statistics/HostStatSampler.hpp b/cppcache/src/statistics/HostStatSampler.hpp
index b120ac4..eef4758 100644
--- a/cppcache/src/statistics/HostStatSampler.hpp
+++ b/cppcache/src/statistics/HostStatSampler.hpp
@@ -58,25 +58,11 @@ using std::chrono::system_clock;
 
 class StatArchiveWriter;
 class StatisticsManager;
+
 /**
  * HostStatSampler implements a thread which will monitor, sample and archive
  * statistics. It only has the common functionalities which any sampler needs.
  */
-
-/* adongre
- * CID 28733: Other violation (MISSING_COPY)
- * Class "apache::geode::statistics::OsStatisticsImpl" owns resources that are
- * managed in its constructor and destructor but has no user-written copy
- * constructor.
- *
- * CID 28719: Other violation (MISSING_ASSIGN) Class
- * "apache::geode::statistics::HostStatSampler"
- * owns resources that are managed in its constructor and destructor but has no
- * user-written assignment operator.
- *
- * FIX : Make the class NonCopyable
- */
-
 class _GEODE_EXPORT HostStatSampler : public ACE_Task_Base,
                                       private NonCopyable,
                                       private NonAssignable {
diff --git a/cppcache/src/statistics/NullProcessStats.cpp b/cppcache/src/statistics/NullProcessStats.cpp
deleted file mode 100644
index 8b0bd5c..0000000
--- a/cppcache/src/statistics/NullProcessStats.cpp
+++ /dev/null
@@ -1,42 +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 <geode/internal/geode_globals.hpp>
-#include <ace/Thread_Mutex.h>
-#include <ace/Singleton.h>
-#include "NullProcessStats.hpp"
-#include "GeodeStatisticsFactory.hpp"
-#include "HostStatHelperNull.hpp"
-using namespace apache::geode::statistics;
-
-/**
- * <P>This class provides the interface for statistics about a
- * Null operating system process that is using a Geode system.
- *
- */
-
-NullProcessStats::NullProcessStats(int64_t pid, const char* name) {}
-
-int64_t NullProcessStats::getProcessSize() { return 0; }
-
-int32_t NullProcessStats::getCpuUsage() { return 0; }
-int64_t NullProcessStats::getCPUTime() { return 0; }
-int32_t NullProcessStats::getNumThreads() { return 0; }
-int64_t NullProcessStats::getAllCpuTime() { return 0; }
-
-void NullProcessStats::close() {}
-
-NullProcessStats::~NullProcessStats() {}
diff --git a/cppcache/src/statistics/NullProcessStats.hpp b/cppcache/src/statistics/NullProcessStats.hpp
deleted file mode 100644
index 35c0487..0000000
--- a/cppcache/src/statistics/NullProcessStats.hpp
+++ /dev/null
@@ -1,61 +0,0 @@
-#pragma once
-
-#ifndef GEODE_STATISTICS_NULLPROCESSSTATS_H_
-#define GEODE_STATISTICS_NULLPROCESSSTATS_H_
-
-/*
- * 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 <geode/internal/geode_globals.hpp>
-
-#include "Statistics.hpp"
-#include "StatisticsType.hpp"
-#include "StatisticDescriptor.hpp"
-#include "ProcessStats.hpp"
-#include "HostStatHelper.hpp"
-
-using namespace apache::geode::client;
-
-/** @file
- */
-
-namespace apache {
-namespace geode {
-namespace statistics {
-/**
- * <P>This class provides the interface for statistics about a
- * Null operating system process that is using a Geode system.
- *
- */
-
-class _GEODE_EXPORT NullProcessStats : public ProcessStats {
- public:
-  NullProcessStats(int64_t pid, const char* name);
-  ~NullProcessStats();
-
-  int64_t getProcessSize();
-  int32_t getCpuUsage();
-  int64_t getCPUTime();
-  int32_t getNumThreads();
-  int64_t getAllCpuTime();
-  void close();
-};  // Class NullProcessStats
-}  // namespace client
-}  // namespace geode
-}  // namespace apache
-
-#endif  // GEODE_STATISTICS_NULLPROCESSSTATS_H_
diff --git a/cppcache/src/statistics/PoolStatsSampler.cpp b/cppcache/src/statistics/PoolStatsSampler.cpp
index d75b480..79b816d 100644
--- a/cppcache/src/statistics/PoolStatsSampler.cpp
+++ b/cppcache/src/statistics/PoolStatsSampler.cpp
@@ -21,7 +21,6 @@
 
 #include "PoolStatsSampler.hpp"
 #include "GeodeStatisticsFactory.hpp"
-#include "HostStatHelper.hpp"
 #include "../ReadWriteLock.hpp"
 #include "../CacheImpl.hpp"
 #include "../ThinClientPoolDM.hpp"
@@ -111,8 +110,6 @@ void PoolStatsSampler::putStatsInAdminRegion() {
             puts += creates;
           }
         }
-        numThreads = HostStatHelper::getNumThreads();
-        cpuTime = HostStatHelper::getCpuTime();
       }
       static int numCPU = ACE_OS::num_processors();
       auto obj = ClientHealthStats::create(gets, puts, misses, numListeners,

-- 
To stop receiving notification emails like this one, please contact
jbarrett@apache.org.