I have 2 more questions:

1. The capacity of the global dictionary is Integer.MAX_VALUE? If count 
distinct values of column data cross all segments, what will be happened? 
duplication or error ?

2. Where I can manually edit a cube desc json? Now I use JAVA API to create or 
update cube.

Thanks!



------------------ ???????? ------------------
??????: "Yiming Liu";<[email protected]>;
????????: 2016??8??25??(??????) ????9:41
??????: "dev"<[email protected]>; "sunyerui"<[email protected]>; 

????: Re: Precisely Count Distinct on 100 million string values column



Good found.

The code AppendTrieDictionary line 604:

// nValueBytes
if (n.part.length > 255)
    throw new RuntimeException();

Hi Yerui,

Could you add more comments for the 255 limit, with more meaningful exception?


2016-08-24 20:44 GMT+08:00 lxw <[email protected]>:

> It caused by length(USER_ID) > 255.
> After exclude these dirty data, it works .
>
>
> Total 150 million records, execute this query:
>
> select city_code,
> sum(bid_request) as bid_request,
> count(distinct user_id) as uv
> from liuxiaowen.TEST_T_PBS_UV_FACT
> group by city_code
> order by uv desc limit 100
>
> Kylin cost  7 seconds, and Hive cost 180 seconds, the result is same.
>
>
>
> ------------------ Original ------------------
> From:  "lxw";<[email protected]>;
> Date:  Wed, Aug 24, 2016 05:27 PM
> To:  "dev"<[email protected]>;
>
> Subject:  Precisely Count Distinct on 100 million string values column
>
>
>
> Hi,
>
>     I am trying to use "Precisely Count Distinct" on 100 million string
> values column "USER_ID", I updated the cube json :
> "dictionaries": [     {       "column": "USER_ID",       "builder":
> "org.apache.kylin.dict.GlobalDictionaryBuilder"     }   ],
>
> "override_kylin_properties": {     
> "kylin.job.mr.config.override.mapred.map.child.java.opts":
> "-Xmx7g",     "kylin.job.mr.config.override.mapreduce.map.memory.mb":
> "7168"   }  when I build the cube, an error occurred on "#4 Step Name:
> Build Dimension Dictionary",
>   the error log in "kylin.log" :
>
> 2016-08-24 17:27:53,282 ERROR [pool-7-thread-10] dict.CachedTreeMap:239 :
> write value into /kylin_test1/kylin_metadata_test1/resources/GlobalDict/
> dict/LIUXIAOWEN.TEST_T_PBS_UV_FACT/USER_ID.tmp/cached_
> AQEByQXVzFd8r0YviP4x84YqUv-NcRiuCI2d exception: java.lang.RuntimeException
> java.lang.RuntimeException
>         at org.apache.kylin.dict.AppendTrieDictionary$DictNode.
> build_writeNode(AppendTrieDictionary.java:605)
>         at org.apache.kylin.dict.AppendTrieDictionary$DictNode.
> buildTrieBytes(AppendTrieDictionary.java:576)
>         at org.apache.kylin.dict.AppendTrieDictionary$DictNode.
> write(AppendTrieDictionary.java:523)
>         at org.apache.kylin.dict.CachedTreeMap.writeValue(
> CachedTreeMap.java:234)
>         at org.apache.kylin.dict.CachedTreeMap.write(
> CachedTreeMap.java:374)
>         at org.apache.kylin.dict.AppendTrieDictionary.flushIndex(
> AppendTrieDictionary.java:1043)
>         at org.apache.kylin.dict.AppendTrieDictionary$Builder.
> build(AppendTrieDictionary.java:954)
>         at org.apache.kylin.dict.GlobalDictionaryBuilder.build(
> GlobalDictionaryBuilder.java:82)
>         at org.apache.kylin.dict.DictionaryGenerator.buildDictionary(
> DictionaryGenerator.java:81)
>         at org.apache.kylin.dict.DictionaryManager.buildDictionary(
> DictionaryManager.java:323)
>         at org.apache.kylin.cube.CubeManager.buildDictionary(
> CubeManager.java:185)
>         at org.apache.kylin.cube.cli.DictionaryGeneratorCLI.
> processSegment(DictionaryGeneratorCLI.java:51)
>         at org.apache.kylin.cube.cli.DictionaryGeneratorCLI.
> processSegment(DictionaryGeneratorCLI.java:42)
>         at org.apache.kylin.engine.mr.steps.CreateDictionaryJob.run(
> CreateDictionaryJob.java:56)
>         at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:70)
>         at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:84)
>         at org.apache.kylin.engine.mr.common.HadoopShellExecutable.
> doWork(HadoopShellExecutable.java:63)
>         at org.apache.kylin.job.execution.AbstractExecutable.
> execute(AbstractExecutable.java:112)
>         at org.apache.kylin.job.execution.DefaultChainedExecutable.doWork(
> DefaultChainedExecutable.java:57)
>         at org.apache.kylin.job.execution.AbstractExecutable.
> execute(AbstractExecutable.java:112)
>         at org.apache.kylin.job.impl.threadpool.DefaultScheduler$
> JobRunner.run(DefaultScheduler.java:127)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(
> ThreadPoolExecutor.java:1145)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(
> ThreadPoolExecutor.java:615)
>         at java.lang.Thread.run(Thread.java:744)
> 2016-08-24 17:27:53,340 ERROR [pool-7-thread-10]
> common.HadoopShellExecutable:65 : error execute HadoopShellExecutable{id=
> 3a0f2751-dd2a-4a3b-a27a-58bfc0edbbfd-03, name=Build Dimension Dictionary,
> state=RUNNING}
> java.lang.RuntimeException
>         at org.apache.kylin.dict.CachedTreeMap.writeValue(
> CachedTreeMap.java:240)
>         at org.apache.kylin.dict.CachedTreeMap.write(
> CachedTreeMap.java:374)
>         at org.apache.kylin.dict.AppendTrieDictionary.flushIndex(
> AppendTrieDictionary.java:1043)
>         at org.apache.kylin.dict.AppendTrieDictionary$Builder.
> build(AppendTrieDictionary.java:954)
>         at org.apache.kylin.dict.GlobalDictionaryBuilder.build(
> GlobalDictionaryBuilder.java:82)
>         at org.apache.kylin.dict.DictionaryGenerator.buildDictionary(
> DictionaryGenerator.java:81)
>         at org.apache.kylin.dict.DictionaryManager.buildDictionary(
> DictionaryManager.java:323)
>         at org.apache.kylin.cube.CubeManager.buildDictionary(
> CubeManager.java:185)
>         at org.apache.kylin.cube.cli.DictionaryGeneratorCLI.
> processSegment(DictionaryGeneratorCLI.java:51)
>         at org.apache.kylin.cube.cli.DictionaryGeneratorCLI.
> processSegment(DictionaryGeneratorCLI.java:42)
>         at org.apache.kylin.engine.mr.steps.CreateDictionaryJob.run(
> CreateDictionaryJob.java:56)
>         at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:70)
>         at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:84)
>         at org.apache.kylin.engine.mr.common.HadoopShellExecutable.
> doWork(HadoopShellExecutable.java:63)
>         at org.apache.kylin.job.execution.AbstractExecutable.
> execute(AbstractExecutable.java:112)
>         at org.apache.kylin.job.execution.DefaultChainedExecutable.doWork(
> DefaultChainedExecutable.java:57)
>         at org.apache.kylin.job.execution.AbstractExecutable.
> execute(AbstractExecutable.java:112)
>         at org.apache.kylin.job.impl.threadpool.DefaultScheduler$
> JobRunner.run(DefaultScheduler.java:127)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(
> ThreadPoolExecutor.java:1145)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(
> ThreadPoolExecutor.java:615)
>         at java.lang.Thread.run(Thread.java:744)
>
>     and the error log in "kylin.out" :
>
>  Aug 24, 2016 5:25:32 PM com.google.common.cache.LocalCache
> processPendingNotifications
> WARNING: Exception thrown by removal listener
> java.lang.RuntimeException
>         at org.apache.kylin.dict.CachedTreeMap.writeValue(
> CachedTreeMap.java:240)
>         at org.apache.kylin.dict.CachedTreeMap.access$300(
> CachedTreeMap.java:52)
>         at org.apache.kylin.dict.CachedTreeMap$1.onRemoval(
> CachedTreeMap.java:149)
>         at com.google.common.cache.LocalCache.processPendingNotifications(
> LocalCache.java:2011)
>         at com.google.common.cache.LocalCache$Segment.
> runUnlockedCleanup(LocalCache.java:3501)
>         at com.google.common.cache.LocalCache$Segment.
> postWriteCleanup(LocalCache.java:3477)
>         at com.google.common.cache.LocalCache$Segment.put(
> LocalCache.java:2940)
>         at com.google.common.cache.LocalCache.put(LocalCache.java:4202)
>         at com.google.common.cache.LocalCache$LocalManualCache.
> put(LocalCache.java:4798)
>         at org.apache.kylin.dict.CachedTreeMap.put(CachedTreeMap.java:284)
>         at org.apache.kylin.dict.CachedTreeMap.put(CachedTreeMap.java:52)
>         at org.apache.kylin.dict.AppendTrieDictionary$Builder.
> addValue(AppendTrieDictionary.java:829)
>         at org.apache.kylin.dict.AppendTrieDictionary$Builder.
> addValue(AppendTrieDictionary.java:804)
>         at org.apache.kylin.dict.GlobalDictionaryBuilder.build(
> GlobalDictionaryBuilder.java:78)
>         at org.apache.kylin.dict.DictionaryGenerator.buildDictionary(
> DictionaryGenerator.java:81)
>         at org.apache.kylin.dict.DictionaryManager.buildDictionary(
> DictionaryManager.java:323)
>         at org.apache.kylin.cube.CubeManager.buildDictionary(
> CubeManager.java:185)
>         at org.apache.kylin.cube.cli.DictionaryGeneratorCLI.
> processSegment(DictionaryGeneratorCLI.java:51)
>         at org.apache.kylin.cube.cli.DictionaryGeneratorCLI.
> processSegment(DictionaryGeneratorCLI.java:42)
>         at org.apache.kylin.engine.mr.steps.CreateDictionaryJob.run(
> CreateDictionaryJob.java:56)
>         at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:70)
>         at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:84)
>         at org.apache.kylin.engine.mr.common.HadoopShellExecutable.
> doWork(HadoopShellExecutable.java:63)
>         at org.apache.kylin.job.execution.AbstractExecutable.
> execute(AbstractExecutable.java:112)
>         at org.apache.kylin.job.execution.DefaultChainedExecutable.doWork(
> DefaultChainedExecutable.java:57)
>         at org.apache.kylin.job.execution.AbstractExecutable.
> execute(AbstractExecutable.java:112)
>         at org.apache.kylin.job.impl.threadpool.DefaultScheduler$
> JobRunner.run(DefaultScheduler.java:127)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(
> ThreadPoolExecutor.java:1145)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(
> ThreadPoolExecutor.java:615)
>         at java.lang.Thread.run(Thread.java:744)
>
> usage: CreateDictionaryJob
>  -cubename <cubename>         Cube name. For exmaple, flat_item_cube
>  -input <input>               Input path
>  -segmentname <segmentname>   Cube segment name
>



-- 
With Warm regards

Yiming Liu (??????)

Reply via email to