mirror of
https://github.com/apache/impala.git
synced 2025-12-25 02:03:09 -05:00
IMPALA-9858: Fix wrong partition metrics in LocalCatalog profile
The hits and requests metrics of partitions are overcounted due to using
an updated map. This patch fixes it and adds test coverage on partition
metrics.
Tests
- Run CatalogdMetaProviderTest
Change-Id: I10cabce2908f1d252b90390978e679d31003e89d
Reviewed-on: http://gerrit.cloudera.org:8080/16080
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
(cherry picked from commit ee70df2e90)
This commit is contained in:
@@ -836,8 +836,8 @@ public class CatalogdMetaProvider implements MetaProvider {
|
||||
storePartitionsInCache(refImpl, hostIndex, fromCatalogd);
|
||||
}
|
||||
sw.stop();
|
||||
addStatsToProfile(PARTITIONS_STATS_CATEGORY, refToMeta.size(), numMisses, sw);
|
||||
LOG.trace("Request for partitions of {}: hit {}/{}", table, refToMeta.size(),
|
||||
addStatsToProfile(PARTITIONS_STATS_CATEGORY, numHits, numMisses, sw);
|
||||
LOG.trace("Request for partitions of {}: hit {}/{}", table, numHits,
|
||||
partitionRefs.size());
|
||||
|
||||
// Convert the returned map to be by-name instead of by-ref.
|
||||
|
||||
@@ -20,7 +20,6 @@ package org.apache.impala.catalog.local;
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
@@ -43,6 +42,7 @@ import org.apache.impala.testutil.TestUtils;
|
||||
import org.apache.impala.thrift.TBackendGflags;
|
||||
import org.apache.impala.thrift.TCatalogObject;
|
||||
import org.apache.impala.thrift.TCatalogObjectType;
|
||||
import org.apache.impala.thrift.TCounter;
|
||||
import org.apache.impala.thrift.TDatabase;
|
||||
import org.apache.impala.thrift.TFunction;
|
||||
import org.apache.impala.thrift.TFunctionName;
|
||||
@@ -58,11 +58,13 @@ import org.slf4j.LoggerFactory;
|
||||
import com.google.common.base.Stopwatch;
|
||||
import com.google.common.cache.CacheStats;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Maps;
|
||||
|
||||
public class CatalogdMetaProviderTest {
|
||||
|
||||
private final static Logger LOG = LoggerFactory.getLogger(
|
||||
CatalogdMetaProviderTest.class);
|
||||
private final static ListMap<TNetworkAddress> HOST_INDEX = new ListMap<>();
|
||||
|
||||
private final CatalogdMetaProvider provider_;
|
||||
private final TableMetaRef tableRef_;
|
||||
@@ -113,33 +115,36 @@ public class CatalogdMetaProviderTest {
|
||||
public void testCachePartitionsByRef() throws Exception {
|
||||
List<PartitionRef> allRefs = provider_.loadPartitionList(tableRef_);
|
||||
List<PartitionRef> partialRefs = allRefs.subList(3, 8);
|
||||
ListMap<TNetworkAddress> hostIndex = new ListMap<>();
|
||||
CacheStats stats = diffStats();
|
||||
|
||||
// Should get no hits on the initial load of partitions.
|
||||
Map<String, PartitionMetadata> partMap = provider_.loadPartitionsByRefs(
|
||||
tableRef_, /* partitionColumnNames unused by this impl */null, hostIndex,
|
||||
partialRefs);
|
||||
Map<String, PartitionMetadata> partMap = loadPartitions(tableRef_, partialRefs);
|
||||
assertEquals(partialRefs.size(), partMap.size());
|
||||
stats = diffStats();
|
||||
assertEquals(0, stats.hitCount());
|
||||
|
||||
// Load the same partitions again and we should get a hit for each partition.
|
||||
Map<String, PartitionMetadata> partMapHit = provider_.loadPartitionsByRefs(
|
||||
tableRef_, /* partitionColumnNames unused by this impl */null, hostIndex,
|
||||
partialRefs);
|
||||
Map<String, PartitionMetadata> partMapHit = loadPartitions(tableRef_, partialRefs);
|
||||
stats = diffStats();
|
||||
assertEquals(stats.hitCount(), partMapHit.size());
|
||||
|
||||
// Load all of the partitions: we should get some hits and some misses.
|
||||
Map<String, PartitionMetadata> allParts = provider_.loadPartitionsByRefs(
|
||||
tableRef_, /* partitionColumnNames unused by this impl */null, hostIndex,
|
||||
allRefs);
|
||||
Map<String, PartitionMetadata> allParts = loadPartitions(tableRef_, allRefs);
|
||||
assertEquals(allRefs.size(), allParts.size());
|
||||
stats = diffStats();
|
||||
assertEquals(stats.hitCount(), partMapHit.size());
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper method for loading partitions by refs.
|
||||
*/
|
||||
private Map<String, PartitionMetadata> loadPartitions(TableMetaRef tableMetaRef,
|
||||
List<PartitionRef> partRefs) throws Exception {
|
||||
return provider_.loadPartitionsByRefs(
|
||||
tableMetaRef, /* partitionColumnNames unused by this impl */null, HOST_INDEX,
|
||||
partRefs);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCacheColumnStats() throws Exception {
|
||||
ImmutableList<String> colNames = ImmutableList.of("month", "id");
|
||||
@@ -240,19 +245,35 @@ public class CatalogdMetaProviderTest {
|
||||
public void testProfile() throws Exception {
|
||||
FrontendProfile profile;
|
||||
try (FrontendProfile.Scope scope = FrontendProfile.createNewWithScope()) {
|
||||
// This table has been loaded in the constructor. Hit cache.
|
||||
provider_.loadTable("functional", "alltypes");
|
||||
// Load all partition ids. This will create a PartitionLists miss.
|
||||
List<PartitionRef> allRefs = provider_.loadPartitionList(tableRef_);
|
||||
// Load all partitions. This will create one partition miss per partition.
|
||||
loadPartitions(tableRef_, allRefs);
|
||||
profile = FrontendProfile.getCurrent();
|
||||
}
|
||||
TRuntimeProfileNode prof = profile.emitAsThrift();
|
||||
assertEquals(4, prof.counters.size());
|
||||
Collections.sort(prof.counters);
|
||||
assertEquals("TCounter(name:CatalogFetch.Tables.Hits, unit:NONE, value:1)",
|
||||
prof.counters.get(0).toString());
|
||||
assertEquals("TCounter(name:CatalogFetch.Tables.Misses, unit:NONE, value:0)",
|
||||
prof.counters.get(1).toString());
|
||||
assertEquals("TCounter(name:CatalogFetch.Tables.Requests, unit:NONE, value:1)",
|
||||
prof.counters.get(2).toString());
|
||||
assertEquals("CatalogFetch.Tables.Time", prof.counters.get(3).name);
|
||||
Map<String, TCounter> counters = Maps.uniqueIndex(prof.counters, TCounter::getName);
|
||||
assertEquals(prof.counters.toString(), 16, counters.size());
|
||||
assertEquals(1, counters.get("CatalogFetch.Tables.Hits").getValue());
|
||||
assertEquals(0, counters.get("CatalogFetch.Tables.Misses").getValue());
|
||||
assertEquals(1, counters.get("CatalogFetch.Tables.Requests").getValue());
|
||||
assertTrue(counters.containsKey("CatalogFetch.Tables.Time"));
|
||||
assertEquals(0, counters.get("CatalogFetch.PartitionLists.Hits").getValue());
|
||||
assertEquals(1, counters.get("CatalogFetch.PartitionLists.Misses").getValue());
|
||||
assertEquals(1, counters.get("CatalogFetch.PartitionLists.Requests").getValue());
|
||||
assertTrue(counters.containsKey("CatalogFetch.PartitionLists.Time"));
|
||||
assertEquals(0, counters.get("CatalogFetch.Partitions.Hits").getValue());
|
||||
assertEquals(24, counters.get("CatalogFetch.Partitions.Misses").getValue());
|
||||
assertEquals(24, counters.get("CatalogFetch.Partitions.Requests").getValue());
|
||||
assertTrue(counters.containsKey("CatalogFetch.Partitions.Time"));
|
||||
assertTrue(counters.containsKey("CatalogFetch.RPCs.Bytes"));
|
||||
assertTrue(counters.containsKey("CatalogFetch.RPCs.Time"));
|
||||
// 2 RPCs: one for fetching partition list, the other one for fetching partitions.
|
||||
assertEquals(2, counters.get("CatalogFetch.RPCs.Requests").getValue());
|
||||
// Should contains StorageLoad.Time since we have loaded partitions from catalogd.
|
||||
assertTrue(counters.containsKey("CatalogFetch.StorageLoad.Time"));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
||||
Reference in New Issue
Block a user