[ 
https://issues.apache.org/jira/browse/IMPALA-9733?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

h.s updated IMPALA-9733:
------------------------
    Description: 
a table default.test (5 column, 91 partitions, about 256k files per partitions, 
3.2k per file, totally 23340537 files)

when query 'describe default.test' catalog log throw oom like
{quote}I0507 21:03:10.629343 4618 HdfsTable.java:1235] Fetching partition 
metadata from the Metastore: default.test
 I0507 21:03:10.811630 4618 HdfsTable.java:1239] Fetched partition metadata 
from the Metastore: default.test
 I0507 21:03:11.136696 4618 HdfsTable.java:862] Loading file and block metadata 
for 91 paths for table default.test using a thread pool of size 50
 I0507 21:05:03.368173 4618 HdfsTable.java:903] Loaded file and block metadata 
for default.test
 I0507 21:05:03.391232 4618 TableLoader.java:97] Loaded metadata for: 
default.test
 E0507 21:05:16.206063 51284 catalog-server.cc:377] OutOfMemoryError: null
 E0507 21:05:28.912606 51284 catalog-server.cc:377] OutOfMemoryError: null
{quote}
i do debug catalog use idea trace the oom stack like
{quote}java.io.ByteArrayOutputStream.hugeCapacity(ByteArrayOutputStream.java:123)
 java.io.ByteArrayOutputStream.grow(ByteArrayOutputStream.java:117)
 java.io.ByteArrayOutputStream.ensureCapacity(ByteArrayOutputStream.java:93)
 java.io.ByteArrayOutputStream.write(ByteArrayOutputStream.java:153)
 
org.apache.thrift.transport.TIOStreamTransport.write(TIOStreamTransport.java:145)
 
org.apache.thrift.protocol.TBinaryProtocol.writeBinary(TBinaryProtocol.java:211)
 
org.apache.impala.thrift.THdfsFileDesc$THdfsFileDescStandardScheme.write(THdfsFileDesc.java:369)
 
org.apache.impala.thrift.THdfsFileDesc$THdfsFileDescStandardScheme.write(THdfsFileDesc.java:332)
 org.apache.impala.thrift.THdfsFileDesc.write(THdfsFileDesc.java:283)
 
org.apache.impala.thrift.THdfsPartition$THdfsPartitionStandardScheme.write(THdfsPartition.java:2137)
 
org.apache.impala.thrift.THdfsPartition$THdfsPartitionStandardScheme.write(THdfsPartition.java:1862)
 org.apache.impala.thrift.THdfsPartition.write(THdfsPartition.java:1677)
 
org.apache.impala.thrift.THdfsTable$THdfsTableStandardScheme.write(THdfsTable.java:1249)
 
org.apache.impala.thrift.THdfsTable$THdfsTableStandardScheme.write(THdfsTable.java:1076)
 org.apache.impala.thrift.THdfsTable.write(THdfsTable.java:939)
 org.apache.impala.thrift.TTable$TTableStandardScheme.write(TTable.java:1632)
 org.apache.impala.thrift.TTable$TTableStandardScheme.write(TTable.java:1403)
 org.apache.impala.thrift.TTable.write(TTable.java:1212)
 
org.apache.impala.thrift.TCatalogObject$TCatalogObjectStandardScheme.write(TCatalogObject.java:1245)
 
org.apache.impala.thrift.TCatalogObject$TCatalogObjectStandardScheme.write(TCatalogObject.java:1102)
 org.apache.impala.thrift.TCatalogObject.write(TCatalogObject.java:942)
 org.apache.thrift.TSerializer.serialize(TSerializer.java:79)
 
org.apache.impala.catalog.CatalogServiceCatalog$GetCatalogDeltaContext.addCatalogObject(CatalogServiceCatalog.java:591)
 
org.apache.impala.catalog.CatalogServiceCatalog.addTableToCatalogDeltaHelper(CatalogServiceCatalog.java:1028)
 
org.apache.impala.catalog.CatalogServiceCatalog.addTableToCatalogDelta(CatalogServiceCatalog.java:974)
 
org.apache.impala.catalog.CatalogServiceCatalog.addDatabaseToCatalogDelta(CatalogServiceCatalog.java:880)
 
org.apache.impala.catalog.CatalogServiceCatalog.getCatalogDelta(CatalogServiceCatalog.java:671)
 org.apache.impala.service.JniCatalog.getCatalogDelta(JniCatalog.java:135)
{quote}
and the code stack
{quote}grow:119, ByteArrayOutputStream (java.io)
 ensureCapacity:93, ByteArrayOutputStream (java.io)
 write:153, ByteArrayOutputStream (java.io)
 write:145, TIOStreamTransport (org.apache.thrift.transport)
 writeBinary:211, TBinaryProtocol (org.apache.thrift.protocol)
 write:369, THdfsFileDesc$THdfsFileDescStandardScheme (org.apache.impala.thrift)
 write:332, THdfsFileDesc$THdfsFileDescStandardScheme (org.apache.impala.thrift)
 write:283, THdfsFileDesc (org.apache.impala.thrift)
 write:2137, THdfsPartition$THdfsPartitionStandardScheme 
(org.apache.impala.thrift)
 write:1862, THdfsPartition$THdfsPartitionStandardScheme 
(org.apache.impala.thrift)
 write:1677, THdfsPartition (org.apache.impala.thrift)
 write:1249, THdfsTable$THdfsTableStandardScheme (org.apache.impala.thrift)
 write:1076, THdfsTable$THdfsTableStandardScheme (org.apache.impala.thrift)
 write:939, THdfsTable (org.apache.impala.thrift)
 write:1632, TTable$TTableStandardScheme (org.apache.impala.thrift)
 write:1403, TTable$TTableStandardScheme (org.apache.impala.thrift)
 write:1212, TTable (org.apache.impala.thrift)
 write:1245, TCatalogObject$TCatalogObjectStandardScheme 
(org.apache.impala.thrift)
 write:1102, TCatalogObject$TCatalogObjectStandardScheme 
(org.apache.impala.thrift)
 write:942, TCatalogObject (org.apache.impala.thrift)
 serialize:79, TSerializer (org.apache.thrift)
 addCatalogObject:591, CatalogServiceCatalog$GetCatalogDeltaContext 
(org.apache.impala.catalog)
 addTableToCatalogDeltaHelper:1028, CatalogServiceCatalog 
(org.apache.impala.catalog)
 addTableToCatalogDelta:974, CatalogServiceCatalog (org.apache.impala.catalog)
 addDatabaseToCatalogDelta:880, CatalogServiceCatalog 
(org.apache.impala.catalog)
 getCatalogDelta:671, CatalogServiceCatalog (org.apache.impala.catalog)
 getCatalogDelta:135, JniCatalog (org.apache.impala.service)
{quote}
{{and in file ByteArrayOutputStream.java (Integer.MAX_VALUE=2147483647) code 
like}}
 private static int hugeCapacity(int minCapacity)

{ if (minCapacity < 0) // overflow throw new OutOfMemoryError(); return 
(minCapacity > MAX_ARRAY_SIZE) ? Integer.MAX_VALUE : MAX_ARRAY_SIZE; }

 

 

and in ByteArrayOutputStream.java per len=176, and count+ (totally file number 
times)up to oom
 public synchronized void write(byte b[], int off, int len) { if ((off < 0) || 
(off > b.length) || (len < 0) || ((off + len) - b.length > 0))

{ throw new IndexOutOfBoundsException(); }

ensureCapacity(count + len); System.arraycopy(b, off, buf, count, len); count 
+= len; }
  

catalog setting
{quote}-default_pool_mem_limit=280g \
 -mem_limit=330g \
 -mem_limit_includes_jvm=true \
 -buffer_pool_limit=180g \
 -max_hdfs_partitions_parallel_load=50 \
 -non_impala_java_vlog=0 \
 -inc_stats_size_limit_bytes=4400000000 \
 -pull_incremental_statistics=false \
 -num_threads_per_core=8 \
 -num_metadata_loading_threads=50 \
 -num_hdfs_worker_threads=50 \
 -num_acceptor_threads=20 \
 -accepted_cnxn_setup_thread_pool_size=10 \
 -max_row_batches=4000 \
 -max_queued_row_batch_bytes=100000000 \
 -compact_catalog_topic=true \
{quote}
statestore setting
{quote}-inc_stats_size_limit_bytes=4400000000 \
 -pull_incremental_statistics=false \
 -compact_catalog_topic=true \
{quote}
impalad setting
{quote}-mem_limit=250g \
 -buffer_pool_limit=60g \
 -mem_limit_includes_jvm=true \
 -default_pool_mem_limit=180g \
 -num_metadata_loading_threads=50 \
 -num_threads_per_core=8 \
 -num_hdfs_worker_threads=50 \
 -max_hdfs_partitions_parallel_load=50 \
 -num_acceptor_threads=20 \
 -accepted_cnxn_setup_thread_pool_size=5 \
 -pull_incremental_statistics=false \
 -inc_stats_size_limit_bytes=4400000000 \
 -compact_catalog_topic=true \
{quote}
noticed https://issues.apache.org/jira/browse/IMPALA-8667 

set -pull_incremental_statistics=false but true is the same

and https://issues.apache.org/jira/browse/IMPALA-5990

[https://github.com/cloudera/Impala/commit/c2184e56ae91bc9c6f04ac0dae1b75261d6d20a2]

like catalog object over the limit and not compressed, is there any solutions 
can query 'describe default.test' 

  was:
a table default.test (5 column, 91 partitions, about 256k files per partitions, 
3.2k per file, totally 23340537 files)

when query 'describe default.test' catalog log throw oom like
{quote}I0507 21:03:10.629343 4618 HdfsTable.java:1235] Fetching partition 
metadata from the Metastore: default.test
 I0507 21:03:10.811630 4618 HdfsTable.java:1239] Fetched partition metadata 
from the Metastore: default.test
 I0507 21:03:11.136696 4618 HdfsTable.java:862] Loading file and block metadata 
for 91 paths for table default.test using a thread pool of size 50
 I0507 21:05:03.368173 4618 HdfsTable.java:903] Loaded file and block metadata 
for default.test
 I0507 21:05:03.391232 4618 TableLoader.java:97] Loaded metadata for: 
default.test
 E0507 21:05:16.206063 51284 catalog-server.cc:377] OutOfMemoryError: null
 E0507 21:05:28.912606 51284 catalog-server.cc:377] OutOfMemoryError: null
 E0507 21:05:43.598096 51284 catalog-server.cc:377] OutOfMemoryError: null
 E0507 21:05:59.850869 51284 catalog-server.cc:377] OutOfMemoryError: null
 E0507 21:06:15.830071 51284 catalog-server.cc:377] OutOfMemoryError: null
 E0507 21:06:31.616187 51284 catalog-server.cc:377] OutOfMemoryError: null
 E0507 21:06:43.998366 51284 catalog-server.cc:377] OutOfMemoryError: null
 E0507 21:06:54.981011 51284 catalog-server.cc:377] OutOfMemoryError: null
 E0507 21:07:08.267032 51284 catalog-server.cc:377] OutOfMemoryError: null
 E0507 21:07:26.103045 51284 catalog-server.cc:377] OutOfMemoryError: null
 E0507 21:07:43.798761 51284 catalog-server.cc:377] OutOfMemoryError: null
 E0507 21:07:59.169924 51284 catalog-server.cc:377] OutOfMemoryError: null
 E0507 21:08:12.431888 51284 catalog-server.cc:377] OutOfMemoryError: null
{quote}
i do debug catalog use idea trace the oom stack like
{quote}java.io.ByteArrayOutputStream.hugeCapacity(ByteArrayOutputStream.java:123)
 java.io.ByteArrayOutputStream.grow(ByteArrayOutputStream.java:117)
 java.io.ByteArrayOutputStream.ensureCapacity(ByteArrayOutputStream.java:93)
 java.io.ByteArrayOutputStream.write(ByteArrayOutputStream.java:153)
 
org.apache.thrift.transport.TIOStreamTransport.write(TIOStreamTransport.java:145)
 
org.apache.thrift.protocol.TBinaryProtocol.writeBinary(TBinaryProtocol.java:211)
 
org.apache.impala.thrift.THdfsFileDesc$THdfsFileDescStandardScheme.write(THdfsFileDesc.java:369)
 
org.apache.impala.thrift.THdfsFileDesc$THdfsFileDescStandardScheme.write(THdfsFileDesc.java:332)
 org.apache.impala.thrift.THdfsFileDesc.write(THdfsFileDesc.java:283)
 
org.apache.impala.thrift.THdfsPartition$THdfsPartitionStandardScheme.write(THdfsPartition.java:2137)
 
org.apache.impala.thrift.THdfsPartition$THdfsPartitionStandardScheme.write(THdfsPartition.java:1862)
 org.apache.impala.thrift.THdfsPartition.write(THdfsPartition.java:1677)
 
org.apache.impala.thrift.THdfsTable$THdfsTableStandardScheme.write(THdfsTable.java:1249)
 
org.apache.impala.thrift.THdfsTable$THdfsTableStandardScheme.write(THdfsTable.java:1076)
 org.apache.impala.thrift.THdfsTable.write(THdfsTable.java:939)
 org.apache.impala.thrift.TTable$TTableStandardScheme.write(TTable.java:1632)
 org.apache.impala.thrift.TTable$TTableStandardScheme.write(TTable.java:1403)
 org.apache.impala.thrift.TTable.write(TTable.java:1212)
 
org.apache.impala.thrift.TCatalogObject$TCatalogObjectStandardScheme.write(TCatalogObject.java:1245)
 
org.apache.impala.thrift.TCatalogObject$TCatalogObjectStandardScheme.write(TCatalogObject.java:1102)
 org.apache.impala.thrift.TCatalogObject.write(TCatalogObject.java:942)
 org.apache.thrift.TSerializer.serialize(TSerializer.java:79)
 
org.apache.impala.catalog.CatalogServiceCatalog$GetCatalogDeltaContext.addCatalogObject(CatalogServiceCatalog.java:591)
 
org.apache.impala.catalog.CatalogServiceCatalog.addTableToCatalogDeltaHelper(CatalogServiceCatalog.java:1028)
 
org.apache.impala.catalog.CatalogServiceCatalog.addTableToCatalogDelta(CatalogServiceCatalog.java:974)
 
org.apache.impala.catalog.CatalogServiceCatalog.addDatabaseToCatalogDelta(CatalogServiceCatalog.java:880)
 
org.apache.impala.catalog.CatalogServiceCatalog.getCatalogDelta(CatalogServiceCatalog.java:671)
 org.apache.impala.service.JniCatalog.getCatalogDelta(JniCatalog.java:135)
{quote}
and the code stack
{quote}grow:119, ByteArrayOutputStream (java.io)
 ensureCapacity:93, ByteArrayOutputStream (java.io)
 write:153, ByteArrayOutputStream (java.io)
 write:145, TIOStreamTransport (org.apache.thrift.transport)
 writeBinary:211, TBinaryProtocol (org.apache.thrift.protocol)
 write:369, THdfsFileDesc$THdfsFileDescStandardScheme (org.apache.impala.thrift)
 write:332, THdfsFileDesc$THdfsFileDescStandardScheme (org.apache.impala.thrift)
 write:283, THdfsFileDesc (org.apache.impala.thrift)
 write:2137, THdfsPartition$THdfsPartitionStandardScheme 
(org.apache.impala.thrift)
 write:1862, THdfsPartition$THdfsPartitionStandardScheme 
(org.apache.impala.thrift)
 write:1677, THdfsPartition (org.apache.impala.thrift)
 write:1249, THdfsTable$THdfsTableStandardScheme (org.apache.impala.thrift)
 write:1076, THdfsTable$THdfsTableStandardScheme (org.apache.impala.thrift)
 write:939, THdfsTable (org.apache.impala.thrift)
 write:1632, TTable$TTableStandardScheme (org.apache.impala.thrift)
 write:1403, TTable$TTableStandardScheme (org.apache.impala.thrift)
 write:1212, TTable (org.apache.impala.thrift)
 write:1245, TCatalogObject$TCatalogObjectStandardScheme 
(org.apache.impala.thrift)
 write:1102, TCatalogObject$TCatalogObjectStandardScheme 
(org.apache.impala.thrift)
 write:942, TCatalogObject (org.apache.impala.thrift)
 serialize:79, TSerializer (org.apache.thrift)
 addCatalogObject:591, CatalogServiceCatalog$GetCatalogDeltaContext 
(org.apache.impala.catalog)
 addTableToCatalogDeltaHelper:1028, CatalogServiceCatalog 
(org.apache.impala.catalog)
 addTableToCatalogDelta:974, CatalogServiceCatalog (org.apache.impala.catalog)
 addDatabaseToCatalogDelta:880, CatalogServiceCatalog 
(org.apache.impala.catalog)
 getCatalogDelta:671, CatalogServiceCatalog (org.apache.impala.catalog)
 getCatalogDelta:135, JniCatalog (org.apache.impala.service)
{quote}
{{and in file ByteArrayOutputStream.java (Integer.MAX_VALUE=2147483647) code 
like}}
    private static int hugeCapacity(int minCapacity) {        if (minCapacity < 
0) // overflow            throw new OutOfMemoryError();        return 
(minCapacity > MAX_ARRAY_SIZE) ?            Integer.MAX_VALUE :            
MAX_ARRAY_SIZE;    }
 

 

and in ByteArrayOutputStream.java per len=176, and count+ (totally file number 
times)up to oom
    public synchronized void write(byte b[], int off, int len) {        if 
((off < 0) || (off > b.length) || (len < 0) ||            ((off + len) - 
b.length > 0)) {            throw new IndexOutOfBoundsException();        }     
   ensureCapacity(count + len);        System.arraycopy(b, off, buf, count, 
len);        count += len;    }
 

catalog setting
{quote}-default_pool_mem_limit=280g \
-mem_limit=330g \
-mem_limit_includes_jvm=true \
-buffer_pool_limit=180g \
-max_hdfs_partitions_parallel_load=50 \
-non_impala_java_vlog=0 \
-inc_stats_size_limit_bytes=4400000000 \
-pull_incremental_statistics=false \
-num_threads_per_core=8 \
-num_metadata_loading_threads=50 \
-num_hdfs_worker_threads=50 \
-num_acceptor_threads=20 \
-accepted_cnxn_setup_thread_pool_size=10 \
-max_row_batches=4000 \
-max_queued_row_batch_bytes=100000000 \
-compact_catalog_topic=true \
{quote}
statestore setting
{quote}-inc_stats_size_limit_bytes=4400000000 \
-pull_incremental_statistics=false \
-compact_catalog_topic=true \
{quote}
impalad setting
{quote}-mem_limit=250g \
-buffer_pool_limit=60g \
-mem_limit_includes_jvm=true \
-default_pool_mem_limit=180g \
-num_metadata_loading_threads=50 \
-num_threads_per_core=8 \
-num_hdfs_worker_threads=50 \
-max_hdfs_partitions_parallel_load=50 \
-num_acceptor_threads=20 \
-accepted_cnxn_setup_thread_pool_size=5 \
-pull_incremental_statistics=false \
-inc_stats_size_limit_bytes=4400000000 \
-compact_catalog_topic=true \
{quote}
noticed https://issues.apache.org/jira/browse/IMPALA-8667 

set -pull_incremental_statistics=false but true is the same

and https://issues.apache.org/jira/browse/IMPALA-5990

[https://github.com/cloudera/Impala/commit/c2184e56ae91bc9c6f04ac0dae1b75261d6d20a2]

like catalog object over the limit and not compressed, is there any solutions 
can query 'describe default.test' 


> partition table x(91partitions,200k+files per partition) cannot't query like 
> 'describe default.x'
> -------------------------------------------------------------------------------------------------
>
>                 Key: IMPALA-9733
>                 URL: https://issues.apache.org/jira/browse/IMPALA-9733
>             Project: IMPALA
>          Issue Type: Question
>          Components: Catalog
>    Affects Versions: Impala 3.2.0
>         Environment: jdk version: openjdk1.8.0_172
> os: centos 7.6.1810
>            Reporter: h.s
>            Priority: Major
>
> a table default.test (5 column, 91 partitions, about 256k files per 
> partitions, 3.2k per file, totally 23340537 files)
> when query 'describe default.test' catalog log throw oom like
> {quote}I0507 21:03:10.629343 4618 HdfsTable.java:1235] Fetching partition 
> metadata from the Metastore: default.test
>  I0507 21:03:10.811630 4618 HdfsTable.java:1239] Fetched partition metadata 
> from the Metastore: default.test
>  I0507 21:03:11.136696 4618 HdfsTable.java:862] Loading file and block 
> metadata for 91 paths for table default.test using a thread pool of size 50
>  I0507 21:05:03.368173 4618 HdfsTable.java:903] Loaded file and block 
> metadata for default.test
>  I0507 21:05:03.391232 4618 TableLoader.java:97] Loaded metadata for: 
> default.test
>  E0507 21:05:16.206063 51284 catalog-server.cc:377] OutOfMemoryError: null
>  E0507 21:05:28.912606 51284 catalog-server.cc:377] OutOfMemoryError: null
> {quote}
> i do debug catalog use idea trace the oom stack like
> {quote}java.io.ByteArrayOutputStream.hugeCapacity(ByteArrayOutputStream.java:123)
>  java.io.ByteArrayOutputStream.grow(ByteArrayOutputStream.java:117)
>  java.io.ByteArrayOutputStream.ensureCapacity(ByteArrayOutputStream.java:93)
>  java.io.ByteArrayOutputStream.write(ByteArrayOutputStream.java:153)
>  
> org.apache.thrift.transport.TIOStreamTransport.write(TIOStreamTransport.java:145)
>  
> org.apache.thrift.protocol.TBinaryProtocol.writeBinary(TBinaryProtocol.java:211)
>  
> org.apache.impala.thrift.THdfsFileDesc$THdfsFileDescStandardScheme.write(THdfsFileDesc.java:369)
>  
> org.apache.impala.thrift.THdfsFileDesc$THdfsFileDescStandardScheme.write(THdfsFileDesc.java:332)
>  org.apache.impala.thrift.THdfsFileDesc.write(THdfsFileDesc.java:283)
>  
> org.apache.impala.thrift.THdfsPartition$THdfsPartitionStandardScheme.write(THdfsPartition.java:2137)
>  
> org.apache.impala.thrift.THdfsPartition$THdfsPartitionStandardScheme.write(THdfsPartition.java:1862)
>  org.apache.impala.thrift.THdfsPartition.write(THdfsPartition.java:1677)
>  
> org.apache.impala.thrift.THdfsTable$THdfsTableStandardScheme.write(THdfsTable.java:1249)
>  
> org.apache.impala.thrift.THdfsTable$THdfsTableStandardScheme.write(THdfsTable.java:1076)
>  org.apache.impala.thrift.THdfsTable.write(THdfsTable.java:939)
>  org.apache.impala.thrift.TTable$TTableStandardScheme.write(TTable.java:1632)
>  org.apache.impala.thrift.TTable$TTableStandardScheme.write(TTable.java:1403)
>  org.apache.impala.thrift.TTable.write(TTable.java:1212)
>  
> org.apache.impala.thrift.TCatalogObject$TCatalogObjectStandardScheme.write(TCatalogObject.java:1245)
>  
> org.apache.impala.thrift.TCatalogObject$TCatalogObjectStandardScheme.write(TCatalogObject.java:1102)
>  org.apache.impala.thrift.TCatalogObject.write(TCatalogObject.java:942)
>  org.apache.thrift.TSerializer.serialize(TSerializer.java:79)
>  
> org.apache.impala.catalog.CatalogServiceCatalog$GetCatalogDeltaContext.addCatalogObject(CatalogServiceCatalog.java:591)
>  
> org.apache.impala.catalog.CatalogServiceCatalog.addTableToCatalogDeltaHelper(CatalogServiceCatalog.java:1028)
>  
> org.apache.impala.catalog.CatalogServiceCatalog.addTableToCatalogDelta(CatalogServiceCatalog.java:974)
>  
> org.apache.impala.catalog.CatalogServiceCatalog.addDatabaseToCatalogDelta(CatalogServiceCatalog.java:880)
>  
> org.apache.impala.catalog.CatalogServiceCatalog.getCatalogDelta(CatalogServiceCatalog.java:671)
>  org.apache.impala.service.JniCatalog.getCatalogDelta(JniCatalog.java:135)
> {quote}
> and the code stack
> {quote}grow:119, ByteArrayOutputStream (java.io)
>  ensureCapacity:93, ByteArrayOutputStream (java.io)
>  write:153, ByteArrayOutputStream (java.io)
>  write:145, TIOStreamTransport (org.apache.thrift.transport)
>  writeBinary:211, TBinaryProtocol (org.apache.thrift.protocol)
>  write:369, THdfsFileDesc$THdfsFileDescStandardScheme 
> (org.apache.impala.thrift)
>  write:332, THdfsFileDesc$THdfsFileDescStandardScheme 
> (org.apache.impala.thrift)
>  write:283, THdfsFileDesc (org.apache.impala.thrift)
>  write:2137, THdfsPartition$THdfsPartitionStandardScheme 
> (org.apache.impala.thrift)
>  write:1862, THdfsPartition$THdfsPartitionStandardScheme 
> (org.apache.impala.thrift)
>  write:1677, THdfsPartition (org.apache.impala.thrift)
>  write:1249, THdfsTable$THdfsTableStandardScheme (org.apache.impala.thrift)
>  write:1076, THdfsTable$THdfsTableStandardScheme (org.apache.impala.thrift)
>  write:939, THdfsTable (org.apache.impala.thrift)
>  write:1632, TTable$TTableStandardScheme (org.apache.impala.thrift)
>  write:1403, TTable$TTableStandardScheme (org.apache.impala.thrift)
>  write:1212, TTable (org.apache.impala.thrift)
>  write:1245, TCatalogObject$TCatalogObjectStandardScheme 
> (org.apache.impala.thrift)
>  write:1102, TCatalogObject$TCatalogObjectStandardScheme 
> (org.apache.impala.thrift)
>  write:942, TCatalogObject (org.apache.impala.thrift)
>  serialize:79, TSerializer (org.apache.thrift)
>  addCatalogObject:591, CatalogServiceCatalog$GetCatalogDeltaContext 
> (org.apache.impala.catalog)
>  addTableToCatalogDeltaHelper:1028, CatalogServiceCatalog 
> (org.apache.impala.catalog)
>  addTableToCatalogDelta:974, CatalogServiceCatalog (org.apache.impala.catalog)
>  addDatabaseToCatalogDelta:880, CatalogServiceCatalog 
> (org.apache.impala.catalog)
>  getCatalogDelta:671, CatalogServiceCatalog (org.apache.impala.catalog)
>  getCatalogDelta:135, JniCatalog (org.apache.impala.service)
> {quote}
> {{and in file ByteArrayOutputStream.java (Integer.MAX_VALUE=2147483647) code 
> like}}
>  private static int hugeCapacity(int minCapacity)
> { if (minCapacity < 0) // overflow throw new OutOfMemoryError(); return 
> (minCapacity > MAX_ARRAY_SIZE) ? Integer.MAX_VALUE : MAX_ARRAY_SIZE; }
>  
>  
> and in ByteArrayOutputStream.java per len=176, and count+ (totally file 
> number times)up to oom
>  public synchronized void write(byte b[], int off, int len) { if ((off < 0) 
> || (off > b.length) || (len < 0) || ((off + len) - b.length > 0))
> { throw new IndexOutOfBoundsException(); }
> ensureCapacity(count + len); System.arraycopy(b, off, buf, count, len); count 
> += len; }
>   
> catalog setting
> {quote}-default_pool_mem_limit=280g \
>  -mem_limit=330g \
>  -mem_limit_includes_jvm=true \
>  -buffer_pool_limit=180g \
>  -max_hdfs_partitions_parallel_load=50 \
>  -non_impala_java_vlog=0 \
>  -inc_stats_size_limit_bytes=4400000000 \
>  -pull_incremental_statistics=false \
>  -num_threads_per_core=8 \
>  -num_metadata_loading_threads=50 \
>  -num_hdfs_worker_threads=50 \
>  -num_acceptor_threads=20 \
>  -accepted_cnxn_setup_thread_pool_size=10 \
>  -max_row_batches=4000 \
>  -max_queued_row_batch_bytes=100000000 \
>  -compact_catalog_topic=true \
> {quote}
> statestore setting
> {quote}-inc_stats_size_limit_bytes=4400000000 \
>  -pull_incremental_statistics=false \
>  -compact_catalog_topic=true \
> {quote}
> impalad setting
> {quote}-mem_limit=250g \
>  -buffer_pool_limit=60g \
>  -mem_limit_includes_jvm=true \
>  -default_pool_mem_limit=180g \
>  -num_metadata_loading_threads=50 \
>  -num_threads_per_core=8 \
>  -num_hdfs_worker_threads=50 \
>  -max_hdfs_partitions_parallel_load=50 \
>  -num_acceptor_threads=20 \
>  -accepted_cnxn_setup_thread_pool_size=5 \
>  -pull_incremental_statistics=false \
>  -inc_stats_size_limit_bytes=4400000000 \
>  -compact_catalog_topic=true \
> {quote}
> noticed https://issues.apache.org/jira/browse/IMPALA-8667 
> set -pull_incremental_statistics=false but true is the same
> and https://issues.apache.org/jira/browse/IMPALA-5990
> [https://github.com/cloudera/Impala/commit/c2184e56ae91bc9c6f04ac0dae1b75261d6d20a2]
> like catalog object over the limit and not compressed, is there any solutions 
> can query 'describe default.test' 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to