PHOENIX-3553 Zookeeper connection should be closed immediately after DefaultStatisticsCollector's collecting stats done (Yeonseop Kim)
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/eb72fa89 Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/eb72fa89 Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/eb72fa89 Branch: refs/heads/calcite Commit: eb72fa892ac105251e21355eb8b9fefb35e92819 Parents: f5de28b Author: James Taylor <jamestay...@apache.org> Authored: Fri Jan 13 11:12:48 2017 -0800 Committer: James Taylor <jamestay...@apache.org> Committed: Fri Jan 13 11:12:48 2017 -0800 ---------------------------------------------------------------------- .../stats/DefaultStatisticsCollector.java | 32 ++++++++++++++------ 1 file changed, 23 insertions(+), 9 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/phoenix/blob/eb72fa89/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java index 17010ac..47ee39b 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java @@ -23,6 +23,8 @@ import java.util.List; import java.util.Map; import java.util.Set; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; @@ -70,6 +72,7 @@ class DefaultStatisticsCollector implements StatisticsCollector { private long guidePostDepth; private long maxTimeStamp = MetaDataProtocol.MIN_TABLE_TIMESTAMP; + private static final Log LOG = LogFactory.getLog(DefaultStatisticsCollector.class); DefaultStatisticsCollector(RegionCoprocessorEnvironment env, String tableName, long clientTimeStamp, byte[] family, byte[] gp_width_bytes, byte[] gp_per_region_bytes) throws IOException { @@ -119,16 +122,27 @@ class DefaultStatisticsCollector implements StatisticsCollector { this.guidePostDepth = StatisticsUtil.getGuidePostDepth(guidepostPerRegion, guidepostWidth, env.getRegion().getTableDesc()); } else { - // Next check for GUIDE_POST_WIDTH on table - HTableInterface htable = env.getTable( - SchemaUtil.getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration())); - Get get = new Get(ptableKey); - get.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH_BYTES); - Result result = htable.get(get); long guidepostWidth = -1; - if (!result.isEmpty()) { - Cell cell = result.listCells().get(0); - guidepostWidth = PLong.INSTANCE.getCodec().decodeLong(cell.getValueArray(), cell.getValueOffset(), SortOrder.getDefault()); + HTableInterface htable = null; + try { + // Next check for GUIDE_POST_WIDTH on table + htable = env.getTable( + SchemaUtil.getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration())); + Get get = new Get(ptableKey); + get.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH_BYTES); + Result result = htable.get(get); + if (!result.isEmpty()) { + Cell cell = result.listCells().get(0); + guidepostWidth = PLong.INSTANCE.getCodec().decodeLong(cell.getValueArray(), cell.getValueOffset(), SortOrder.getDefault()); + } + } finally { + if (htable != null) { + try { + htable.close(); + } catch (IOException e) { + LOG.warn("Failed to close " + htable.getName(), e); + } + } } if (guidepostWidth >= 0) { this.guidePostDepth = guidepostWidth;