phoenix-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Aaron Molitor <amoli...@splicemachine.com>
Subject Re: CsvBulkLoadTool with ~75GB file
Date Fri, 19 Aug 2016 17:15:39 GMT
Got to the point where data was loaded, but I could not get count of LINEITEM to complete.
 I realized that I should run compaction, kicking off compaction knocked over two regionservers.
 

I've attached the logs, could this be related to the hbase.mapreduce.bulkload.max.hfiles.perRegion.perFamily
to 48 change I made?



> On Aug 19, 2016, at 09:40, John Leach <jleach4@gmail.com> wrote:
> 
> Gabriel,
> 
> Thanks for the response I appreciate it.  
> 
> I struggle to understand how to use split points in the create statement. 
> 
> (1) Creating a table with Split Points:  
> 
> CREATE TABLE stats.prod_metrics ( host char(50) not null, created_date date not null,
>     txn_count bigint CONSTRAINT pk PRIMARY KEY (host, created_date)
>    DATA_BLOCK_ENCODING='NONE',VERSIONS=5,MAX_FILESIZE=2000000 split on (?, ?, ?) 
> )
> 
> Do you have to know the data distribution in the import files first?
> Do you pass in Hexadecimal bytes in the split on clause?  Is there a utility to generate
them for you based on the primary key?  I think I am missing something here.
> 
> As for the aggregation query, so large insert into queries would be faster in map reduce
vs. phoenix?  
> 
> Would they be stable in Phoenix at scale?
> 
> Regards,
> John
> 
>  
>> On Aug 19, 2016, at 9:52 AM, Gabriel Reid <gabriel.reid@gmail.com <mailto:gabriel.reid@gmail.com>>
wrote:
>> 
>> Hi John,
>> 
>> You can actually pre-split a table when creating it, either by
>> specifying split points in the CREATE TABLE statement[1] or by using
>> salt buckets[2]. In my current use cases I always use salting, but
>> take a look at the salting documentation[2] for the pros and cons of
>> this.
>> 
>> Your approaches as outlined is basically in line with how I would
>> upload a large amount of data into Phoenix, taking into account that
>> the tables would be split into multiple regions at creation time using
>> salt buckets.
>> 
>> About doing ETL to create an aggregate table: personally, if I were
>> trying to create an aggregate table, I would probably do the
>> aggregation outside of Phoenix (using MapReduce or Pig or something
>> similar). This is of course assuming that the aggregate table is the
>> only thing that is needed (i.e. that the un-aggregated table only
>> exists to feed the aggregate table). This isn't to say that Phoenix
>> can't be used to create the aggregate table, but I'm pretty sure that
>> using HBase as a backing store for large scale ETL/aggregation like
>> this just won't be as fast as MapReduce.
>> 
>> However, if your use-case does require using Phoenix for the
>> ETL/aggregation step, then your approach sounds correct.
>> 
>> - Gabriel
>> 
>> 1. http://phoenix.apache.org/language/index.html#create_table <http://phoenix.apache.org/language/index.html#create_table>
>> 2. http://phoenix.apache.org/salted.html <http://phoenix.apache.org/salted.html>
>> 
>> On Fri, Aug 19, 2016 at 3:32 PM, John Leach <jleach4@gmail.com <mailto:jleach4@gmail.com>>
wrote:
>>> Gabriel,
>>> 
>>> Do you guys provide pre-split mechanisms (sampling of import/query data, splitting
policies, etc.) or does the admin have to determine the split points?
>>> 
>>> I guess that begs the question of how you would do a basic ETL operation in Phoenix?
>>> 
>>> How would you do the following on a 100 gigs of data in the import table to 50
gigs in the aggregate table?
>>> 
>>> (1) Create import table.
>>> (2) Import data into that table.
>>> (3) create an aggregate table.
>>> (4) Insert data into aggregate table based on an aggregate of the imported table.
>>> 
>>> Here is what I am gathering from the conversation...
>>> 
>>> (1) Create Import Table.
>>> (2) Perform a pre-split based on some information about the data or via some
split mechanism (If not, import via MapReduce does not scale).
>>> (3) Run MapReduce job for importing data (Other mechanism does not scale?)
>>> (4) Compaction/Statistics Operation on Import Table (If not, query will not scale?)
>>> (5) Create Aggregate Table
>>> (6) Perform a pre-split based on some information about the Aggregate data or
via some split mechanism. (If not, insert will not scale?).
>>> (7) Run insert query?  If you had to run import on map reduce to scale, I suspect
the insert does MapReduce as well or is there some other mechanism that would scale?
>>> (8) Compaction/Statistics Operation on Aggregate Table
>>> 
>>> I really appreciate all the support.  We are trying to run a Phoenix TPCH benchmark
and are struggling a bit to understand the process.
>>> 
>>> Regards,
>>> John Leach
>>> 
>>>> On Aug 19, 2016, at 2:09 AM, Gabriel Reid <gabriel.reid@gmail.com <mailto:gabriel.reid@gmail.com>>
wrote:
>>>> 
>>>> Hi Aaron,
>>>> 
>>>> How many regions are there in the LINEITEM table? The fact that you
>>>> needed to bump the
>>>> hbase.mapreduce.bulkload.max.hfiles.perRegion.perFamily setting up to
>>>> 48 suggests that the amount of data going into a single region of that
>>>> table is probably pretty large.
>>>> 
>>>> Along the same line, I believe regions are the initial unit of
>>>> parallelism (in the absence of statistics[1]) the when running
>>>> aggregate queries in Phoenix. This means that if you have oversized
>>>> regions, you will have poor parallelism when running aggregate
>>>> queries, which could lead to RPC timeouts.
>>>> 
>>>> From what I see in the log info that you provided, your count query
>>>> started at 14:14:06, and errored out at 14:34:15, which appears to be
>>>> in line with the 20 minute HBase RPC timeout. This appears to indicate
>>>> that a scan over a single region is taking more than 20 minutes, which
>>>> again looks to me to be an indicator of an oversized region. If
>>>> possible, I would look into splitting up your LINEITEM table into more
>>>> (smaller) regions, which should improve both import and query
>>>> performance.
>>>> 
>>>> - Gabriel
>>>> 
>>>> 1. http://phoenix.apache.org/update_statistics.html <http://phoenix.apache.org/update_statistics.html>
>>>> 
>>>> On Thu, Aug 18, 2016 at 5:22 PM, Aaron Molitor
>>>> <amolitor@splicemachine.com <mailto:amolitor@splicemachine.com>>
wrote:
>>>>> Gabriel,
>>>>> 
>>>>> Thanks for the help, it's good to know that those params can be passed
from the command line and that the order is important.
>>>>> 
>>>>> I am trying to load the 100GB TPC-H data set and ultimately run the TPC-H
queries.  All of the tables loaded relatively easily except LINEITEM (the largest) required
me to increase the hbase.mapreduce.bulkload.max.hfiles.perRegion.perFamily to 48.  After that
the file loaded.
>>>>> 
>>>>> This brings me to my next question though.  What settings do I need to
change in order to count the [LINEITEM] table? At this point I have changed:
>>>>> - hbase.rpc.timeout set to 20 min
>>>>> - phoenix.query.timeoutMs set to 60 min
>>>>> 
>>>>> I am still getting an error, it appears to be an RPC timeout, as I have
mentioned I have already moved to an uncomfortably high setting.  Is there some other settings
I should be moving and not necessarily the rpc.timeout?
>>>>> 
>>>>> For reference, here's the full sqlline interaction, including the error:
>>>>> ################################################################################
>>>>> Latest phoenix error:
>>>>> splice@stl-colo-srv073 ~]$ /opt/phoenix/default/bin/sqlline.py $(hostname):2181:/hbase-unsecure
>>>>> Setting property: [incremental, false]
>>>>> Setting property: [isolation, TRANSACTION_READ_COMMITTED]
>>>>> issuing: !connect jdbc:phoenix:stl-colo-srv073.splicemachine.colo:2181:/hbase-unsecure
none none org.apache.phoenix.jdbc.PhoenixDriver
>>>>> Connecting to jdbc:phoenix:stl-colo-srv073.splicemachine.colo:2181:/hbase-unsecure
>>>>> SLF4J: Class path contains multiple SLF4J bindings.
>>>>> SLF4J: Found binding in [jar:file:/opt/phoenix/apache-phoenix-4.8.0-HBase-1.1-bin/phoenix-4.8.0-HBase-1.1-client.jar!/org/slf4j/impl/StaticLoggerBinder.class]
>>>>> SLF4J: Found binding in [jar:file:/usr/hdp/2.4.2.0-258/hadoop/lib/slf4j-log4j12-1.7.10.jar!/org/slf4j/impl/StaticLoggerBinder.class]
>>>>> SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings <http://www.slf4j.org/codes.html#multiple_bindings>
for an explanation.
>>>>> 16/08/18 14:14:06 WARN util.NativeCodeLoader: Unable to load native-hadoop
library for your platform... using builtin-java classes where applicable
>>>>> 16/08/18 14:14:08 WARN shortcircuit.DomainSocketFactory: The short-circuit
local reads feature cannot be used because libhadoop cannot be loaded.
>>>>> Connected to: Phoenix (version 4.8)
>>>>> Driver: PhoenixEmbeddedDriver (version 4.8)
>>>>> Autocommit status: true
>>>>> Transaction isolation: TRANSACTION_READ_COMMITTED
>>>>> Building list of tables and columns for tab-completion (set fastconnect
to true to skip)...
>>>>> 147/147 (100%) Done
>>>>> Done
>>>>> sqlline version 1.1.9
>>>>> 0: jdbc:phoenix:stl-colo-srv073.splicemachine> select count(*) from
TPCH.LINEITEM;
>>>>> Error: org.apache.phoenix.exception.PhoenixIOException: Failed after
attempts=36, exceptions:
>>>>> Thu Aug 18 14:34:15 UTC 2016, null, java.net.SocketTimeoutException:
callTimeout=60000, callDuration=1200310: row '' on table 'TPCH.LINEITEM' at region=TPCH.LINEITEM,,1471407572920.656deb38db6555b3eaea71944fdfdbc9.,
hostname=stl-colo-srv076.splicemachine.colo,16020,1471495858713, seqNum=17 (state=08000,code=101)
>>>>> org.apache.phoenix.exception.PhoenixIOException: org.apache.phoenix.exception.PhoenixIOException:
Failed after attempts=36, exceptions:
>>>>> Thu Aug 18 14:34:15 UTC 2016, null, java.net.SocketTimeoutException:
callTimeout=60000, callDuration=1200310: row '' on table 'TPCH.LINEITEM' at region=TPCH.LINEITEM,,1471407572920.656deb38db6555b3eaea71944fdfdbc9.,
hostname=stl-colo-srv076.splicemachine.colo,16020,1471495858713, seqNum=17
>>>>> 
>>>>>       at org.apache.phoenix.util.ServerUtil.parseServerException(ServerUtil.java:111)
>>>>>       at org.apache.phoenix.iterate.BaseResultIterators.getIterators(BaseResultIterators.java:774)
>>>>>       at org.apache.phoenix.iterate.BaseResultIterators.getIterators(BaseResultIterators.java:720)
>>>>>       at org.apache.phoenix.iterate.ConcatResultIterator.getIterators(ConcatResultIterator.java:50)
>>>>>       at org.apache.phoenix.iterate.ConcatResultIterator.currentIterator(ConcatResultIterator.java:97)
>>>>>       at org.apache.phoenix.iterate.ConcatResultIterator.next(ConcatResultIterator.java:117)
>>>>>       at org.apache.phoenix.iterate.BaseGroupedAggregatingResultIterator.next(BaseGroupedAggregatingResultIterator.java:64)
>>>>>       at org.apache.phoenix.iterate.UngroupedAggregatingResultIterator.next(UngroupedAggregatingResultIterator.java:39)
>>>>>       at org.apache.phoenix.jdbc.PhoenixResultSet.next(PhoenixResultSet.java:778)
>>>>>       at sqlline.BufferedRows.<init>(BufferedRows.java:37)
>>>>>       at sqlline.SqlLine.print(SqlLine.java:1649)
>>>>>       at sqlline.Commands.execute(Commands.java:833)
>>>>>       at sqlline.Commands.sql(Commands.java:732)
>>>>>       at sqlline.SqlLine.dispatch(SqlLine.java:807)
>>>>>       at sqlline.SqlLine.begin(SqlLine.java:681)
>>>>>       at sqlline.SqlLine.start(SqlLine.java:398)
>>>>>       at sqlline.SqlLine.main(SqlLine.java:292)
>>>>> Caused by: java.util.concurrent.ExecutionException: org.apache.phoenix.exception.PhoenixIOException:
Failed after attempts=36, exceptions:
>>>>> Thu Aug 18 14:34:15 UTC 2016, null, java.net.SocketTimeoutException:
callTimeout=60000, callDuration=1200310: row '' on table 'TPCH.LINEITEM' at region=TPCH.LINEITEM,,1471407572920.656deb38db6555b3eaea71944fdfdbc9.,
hostname=stl-colo-srv076.splicemachine.colo,16020,1471495858713, seqNum=17
>>>>> 
>>>>>       at java.util.concurrent.FutureTask.report(FutureTask.java:122)
>>>>>       at java.util.concurrent.FutureTask.get(FutureTask.java:202)
>>>>>       at org.apache.phoenix.iterate.BaseResultIterators.getIterators(BaseResultIterators.java:769)
>>>>>       ... 15 more
>>>>> Caused by: org.apache.phoenix.exception.PhoenixIOException: Failed after
attempts=36, exceptions:
>>>>> Thu Aug 18 14:34:15 UTC 2016, null, java.net.SocketTimeoutException:
callTimeout=60000, callDuration=1200310: row '' on table 'TPCH.LINEITEM' at region=TPCH.LINEITEM,,1471407572920.656deb38db6555b3eaea71944fdfdbc9.,
hostname=stl-colo-srv076.splicemachine.colo,16020,1471495858713, seqNum=17
>>>>> 
>>>>>       at org.apache.phoenix.util.ServerUtil.parseServerException(ServerUtil.java:111)
>>>>>       at org.apache.phoenix.iterate.TableResultIterator.initScanner(TableResultIterator.java:174)
>>>>>       at org.apache.phoenix.iterate.TableResultIterator.next(TableResultIterator.java:124)
>>>>>       at org.apache.phoenix.iterate.SpoolingResultIterator.<init>(SpoolingResultIterator.java:139)
>>>>>       at org.apache.phoenix.iterate.SpoolingResultIterator.<init>(SpoolingResultIterator.java:97)
>>>>>       at org.apache.phoenix.iterate.SpoolingResultIterator.<init>(SpoolingResultIterator.java:69)
>>>>>       at org.apache.phoenix.iterate.SpoolingResultIterator$SpoolingResultIteratorFactory.newIterator(SpoolingResultIterator.java:92)
>>>>>       at org.apache.phoenix.iterate.ParallelIterators$1.call(ParallelIterators.java:114)
>>>>>       at org.apache.phoenix.iterate.ParallelIterators$1.call(ParallelIterators.java:106)
>>>>>       at java.util.concurrent.FutureTask.run(FutureTask.java:262)
>>>>>       at org.apache.phoenix.job.JobManager$InstrumentedJobFutureTask.run(JobManager.java:183)
>>>>>       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:745)
>>>>> Caused by: org.apache.hadoop.hbase.client.RetriesExhaustedException:
Failed after attempts=36, exceptions:
>>>>> Thu Aug 18 14:34:15 UTC 2016, null, java.net.SocketTimeoutException:
callTimeout=60000, callDuration=1200310: row '' on table 'TPCH.LINEITEM' at region=TPCH.LINEITEM,,1471407572920.656deb38db6555b3eaea71944fdfdbc9.,
hostname=stl-colo-srv076.splicemachine.colo,16020,1471495858713, seqNum=17
>>>>> 
>>>>>       at org.apache.hadoop.hbase.client.RpcRetryingCallerWithReadReplicas.throwEnrichedException(RpcRetryingCallerWithReadReplicas.java:271)
>>>>>       at org.apache.hadoop.hbase.client.ScannerCallableWithReplicas.call(ScannerCallableWithReplicas.java:199)
>>>>>       at org.apache.hadoop.hbase.client.ScannerCallableWithReplicas.call(ScannerCallableWithReplicas.java:59)
>>>>>       at org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithoutRetries(RpcRetryingCaller.java:200)
>>>>>       at org.apache.hadoop.hbase.client.ClientScanner.call(ClientScanner.java:320)
>>>>>       at org.apache.hadoop.hbase.client.ClientScanner.nextScanner(ClientScanner.java:295)
>>>>>       at org.apache.hadoop.hbase.client.ClientScanner.initializeScannerInConstruction(ClientScanner.java:160)
>>>>>       at org.apache.hadoop.hbase.client.ClientScanner.<init>(ClientScanner.java:155)
>>>>>       at org.apache.hadoop.hbase.client.HTable.getScanner(HTable.java:821)
>>>>>       at org.apache.phoenix.iterate.TableResultIterator.initScanner(TableResultIterator.java:170)
>>>>>       ... 12 more
>>>>> Caused by: java.net.SocketTimeoutException: callTimeout=60000, callDuration=1200310:
row '' on table 'TPCH.LINEITEM' at region=TPCH.LINEITEM,,1471407572920.656deb38db6555b3eaea71944fdfdbc9.,
hostname=stl-colo-srv076.splicemachine.colo,16020,1471495858713, seqNum=17
>>>>>       at org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithRetries(RpcRetryingCaller.java:159)
>>>>>       at org.apache.hadoop.hbase.client.ResultBoundedCompletionService$QueueingFuture.run(ResultBoundedCompletionService.java:64)
>>>>>       ... 3 more
>>>>> Caused by: java.io.IOException: Call to stl-colo-srv076.splicemachine.colo/10.1.1.176:16020
failed on local exception: org.apache.hadoop.hbase.ipc.CallTimeoutException: Call id=27, waitTime=1200001,
operationTimeout=1200000 expired.
>>>>>       at org.apache.hadoop.hbase.ipc.AbstractRpcClient.wrapException(AbstractRpcClient.java:278)
>>>>>       at org.apache.hadoop.hbase.ipc.RpcClientImpl.call(RpcClientImpl.java:1239)
>>>>>       at org.apache.hadoop.hbase.ipc.AbstractRpcClient.callBlockingMethod(AbstractRpcClient.java:217)
>>>>>       at org.apache.hadoop.hbase.ipc.AbstractRpcClient$BlockingRpcChannelImplementation.callBlockingMethod(AbstractRpcClient.java:318)
>>>>>       at org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$BlockingStub.scan(ClientProtos.java:32831)
>>>>>       at org.apache.hadoop.hbase.client.ScannerCallable.openScanner(ScannerCallable.java:373)
>>>>>       at org.apache.hadoop.hbase.client.ScannerCallable.call(ScannerCallable.java:200)
>>>>>       at org.apache.hadoop.hbase.client.ScannerCallable.call(ScannerCallable.java:62)
>>>>>       at org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithoutRetries(RpcRetryingCaller.java:200)
>>>>>       at org.apache.hadoop.hbase.client.ScannerCallableWithReplicas$RetryingRPC.call(ScannerCallableWithReplicas.java:350)
>>>>>       at org.apache.hadoop.hbase.client.ScannerCallableWithReplicas$RetryingRPC.call(ScannerCallableWithReplicas.java:324)
>>>>>       at org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithRetries(RpcRetryingCaller.java:126)
>>>>>       ... 4 more
>>>>> Caused by: org.apache.hadoop.hbase.ipc.CallTimeoutException: Call id=27,
waitTime=1200001, operationTimeout=1200000 expired.
>>>>>       at org.apache.hadoop.hbase.ipc.Call.checkAndSetTimeout(Call.java:70)
>>>>>       at org.apache.hadoop.hbase.ipc.RpcClientImpl.call(RpcClientImpl.java:1213)
>>>>>       ... 14 more
>>>>> 0: jdbc:phoenix:stl-colo-srv073.splicemachine>
>>>>> ################################################################################
>>>>> 
>>>>>> On Aug 18, 2016, at 02:15, Gabriel Reid <gabriel.reid@gmail.com
<mailto:gabriel.reid@gmail.com>> wrote:
>>>>>> 
>>>>>> Hi Aaron,
>>>>>> 
>>>>>> I'll answered your questions directly first, but please see the bottom
>>>>>> part of this mail for important additional details.
>>>>>> 
>>>>>> You can specify the
>>>>>> "hbase.mapreduce.bulkload.max.hfiles.perRegion.perFamily" parameter
>>>>>> (referenced from your StackOverflow link) on the command line of
you
>>>>>> CsvBulkLoadTool command -- my understanding is that this is a purely
>>>>>> client-side parameter. You would provide it via -D as follows:
>>>>>> 
>>>>>>  hadoop jar phoenix-<version>-client.jar
>>>>>> org.apache.phoenix.mapreduce.CsvBulkLoadTool
>>>>>> -Dhbase.mapreduce.bulkload.max.hfiles.perRegion.perFamily=64 <other
>>>>>> command-line parameters>
>>>>>> 
>>>>>> The important point in the above example is that config-based
>>>>>> parameters specified with -D are given before the application-level
>>>>>> parameters, and after the class name to be run.
>>>>>> 
>>>>>> From my read of the HBase code, in this context you can also specify
>>>>>> the "hbase.hregion.max.filesize" parameter in the same way (in this
>>>>>> context it's a client-side parameter).
>>>>>> 
>>>>>> As far as speeding things up, the main points to consider are:
>>>>>> - ensure that compression is enabled for map-reduce jobs on your
>>>>>> cluster -- particularly map-output (intermediate) compression - see
>>>>>> https://datameer.zendesk.com/hc/en-us/articles/204258750-How-to-Use-Intermediate-and-Final-Output-Compression-MR1-YARN-
<https://datameer.zendesk.com/hc/en-us/articles/204258750-How-to-Use-Intermediate-and-Final-Output-Compression-MR1-YARN->
>>>>>> for a good overview
>>>>>> - check the ratio of map output records vs spilled records in the
>>>>>> counters on the import job. If the spilled records are higher than
map
>>>>>> output records (e.g. twice as high or three times as high), then
you
>>>>>> will probably benefit from raising the mapreduce.task.io.sort.mb
>>>>>> setting (see https://hadoop.apache.org/docs/r2.7.1/hadoop-mapreduce-client/hadoop-mapreduce-client-core/mapred-default.xml)
>>>>>> 
>>>>>> Now those are the answers to your questions, but I'm curious about
why
>>>>>> you're getting more than 32 HFiles in a single column family of a
>>>>>> single region. I assume that this means that you're loading large
>>>>>> amounts of data into a small number of regions. This is probably
not a
>>>>>> good thing -- it may impact performance of HBase in general (because
>>>>>> each region has such a large amount of data), and will also have
a
>>>>>> very negative impact on the running time of your import job (because
>>>>>> part of the parallelism of the import job is determined by the number
>>>>>> of regions being written to). I don't think you mentioned how many
>>>>>> regions you have on your table that you're importing to, but
>>>>>> increasing the number of regions will likely resolve several problems
>>>>>> for you. Another reason to do this is the fact that HBase will likely
>>>>>> start splitting your regions after this import due to their size.
>>>>>> 
>>>>>> - Gabriel
>>>>>> 
>>>>>> 
>>>>>> On Thu, Aug 18, 2016 at 3:47 AM, Aaron Molitor
>>>>>> <amolitor@splicemachine.com> wrote:
>>>>>>> Hi all I'm running the CsvBulkLoadTool trying to pull in some
data.  The MapReduce Job appears to complete, and gives some promising information:
>>>>>>> 
>>>>>>> 
>>>>>>> ################################################################################
>>>>>>>      Phoenix MapReduce Import
>>>>>>>              Upserts Done=600037902
>>>>>>>      Shuffle Errors
>>>>>>>              BAD_ID=0
>>>>>>>              CONNECTION=0
>>>>>>>              IO_ERROR=0
>>>>>>>              WRONG_LENGTH=0
>>>>>>>              WRONG_MAP=0
>>>>>>>              WRONG_REDUCE=0
>>>>>>>      File Input Format Counters
>>>>>>>              Bytes Read=79657289180
>>>>>>>      File Output Format Counters
>>>>>>>              Bytes Written=176007436620
>>>>>>> 16/08/17 20:37:04 INFO mapreduce.AbstractBulkLoadTool: Loading
HFiles from /tmp/66f905f4-3d62-45bf-85fe-c247f518355c
>>>>>>> 16/08/17 20:37:04 INFO zookeeper.RecoverableZooKeeper: Process
identifier=hconnection-0xa24982f connecting to ZooKeeper ensemble=stl-colo-srv073.splicemachine.colo:2181
>>>>>>> 16/08/17 20:37:04 INFO zookeeper.ZooKeeper: Initiating client
connection, connectString=stl-colo-srv073.splicemachine.colo:2181 sessionTimeout=1200000 watcher=hconnection-0xa24982f0x0,
quorum=stl-colo-srv073.splicemachine.colo:2181, baseZNode=/hbase-unsecure
>>>>>>> 16/08/17 20:37:04 INFO zookeeper.ClientCnxn: Opening socket connection
to server stl-colo-srv073.splicemachine.colo/10.1.1.173:2181. Will not attempt to authenticate
using SASL (unknown error)
>>>>>>> 16/08/17 20:37:04 INFO zookeeper.ClientCnxn: Socket connection
established to stl-colo-srv073.splicemachine.colo/10.1.1.173:2181, initiating session
>>>>>>> 16/08/17 20:37:04 INFO zookeeper.ClientCnxn: Session establishment
complete on server stl-colo-srv073.splicemachine.colo/10.1.1.173:2181, sessionid = 0x15696476bf90484,
negotiated timeout = 40000
>>>>>>> 16/08/17 20:37:04 INFO mapreduce.AbstractBulkLoadTool: Loading
HFiles for TPCH.LINEITEM from /tmp/66f905f4-3d62-45bf-85fe-c247f518355c/TPCH.LINEITEM
>>>>>>> 16/08/17 20:37:04 WARN mapreduce.LoadIncrementalHFiles: managed
connection cannot be used for bulkload. Creating unmanaged connection.
>>>>>>> 16/08/17 20:37:04 INFO zookeeper.RecoverableZooKeeper: Process
identifier=hconnection-0x456a0752 connecting to ZooKeeper ensemble=stl-colo-srv073.splicemachine.colo:2181
>>>>>>> 16/08/17 20:37:04 INFO zookeeper.ZooKeeper: Initiating client
connection, connectString=stl-colo-srv073.splicemachine.colo:2181 sessionTimeout=1200000 watcher=hconnection-0x456a07520x0,
quorum=stl-colo-srv073.splicemachine.colo:2181, baseZNode=/hbase-unsecure
>>>>>>> 16/08/17 20:37:04 INFO zookeeper.ClientCnxn: Opening socket connection
to server stl-colo-srv073.splicemachine.colo/10.1.1.173:2181. Will not attempt to authenticate
using SASL (unknown error)
>>>>>>> 16/08/17 20:37:04 INFO zookeeper.ClientCnxn: Socket connection
established to stl-colo-srv073.splicemachine.colo/10.1.1.173:2181, initiating session
>>>>>>> 16/08/17 20:37:04 INFO zookeeper.ClientCnxn: Session establishment
complete on server stl-colo-srv073.splicemachine.colo/10.1.1.173:2181, sessionid = 0x15696476bf90485,
negotiated timeout = 40000
>>>>>>> 16/08/17 20:37:06 INFO hfile.CacheConfig: CacheConfig:disabled
>>>>>>> ################################################################################
>>>>>>> 
>>>>>>> and eventually errors out with this exception.
>>>>>>> 
>>>>>>> ################################################################################
>>>>>>> 16/08/17 20:37:07 INFO mapreduce.LoadIncrementalHFiles: Trying
to load hfile=hdfs://stl-colo-srv073.splicemachine.colo:8020/tmp/66f905f4-3d62-45bf-85fe-c247f518355c/TPCH.LINEITEM/0/88b40cbbc4c841f99eae906af3b93cda
first=\x80\x00\x00\x00\x08\xB3\xE7\x84\x80\x00\x00\x04 last=\x80\x00\x00\x00\x09\x92\xAEg\x80\x00\x00\x03
>>>>>>> 16/08/17 20:37:07 INFO mapreduce.LoadIncrementalHFiles: Trying
to load hfile=hdfs://stl-colo-srv073.splicemachine.colo:8020/tmp/66f905f4-3d62-45bf-85fe-c247f518355c/TPCH.LINEITEM/0/de309e5c7b3841a6b4fd299ac8fa8728
first=\x80\x00\x00\x00\x15\xC1\x8Ee\x80\x00\x00\x01 last=\x80\x00\x00\x00\x16\xA0G\xA4\x80\x00\x00\x02
>>>>>>> 16/08/17 20:37:07 INFO mapreduce.LoadIncrementalHFiles: Trying
to load hfile=hdfs://stl-colo-srv073.splicemachine.colo:8020/tmp/66f905f4-3d62-45bf-85fe-c247f518355c/TPCH.LINEITEM/0/e7ed8bc150c9494b8c064a022b3609e0
first=\x80\x00\x00\x00\x09\x92\xAEg\x80\x00\x00\x04 last=\x80\x00\x00\x00\x0Aq\x85D\x80\x00\x00\x02
>>>>>>> 16/08/17 20:37:07 INFO mapreduce.LoadIncrementalHFiles: Trying
to load hfile=hdfs://stl-colo-srv073.splicemachine.colo:8020/tmp/66f905f4-3d62-45bf-85fe-c247f518355c/TPCH.LINEITEM/0/c35e01b66d85450c97da9bb21bfc650f
first=\x80\x00\x00\x00\x0F\xA9\xFED\x80\x00\x00\x04 last=\x80\x00\x00\x00\x10\x88\xD0$\x80\x00\x00\x03
>>>>>>> 16/08/17 20:37:07 INFO mapreduce.LoadIncrementalHFiles: Trying
to load hfile=hdfs://stl-colo-srv073.splicemachine.colo:8020/tmp/66f905f4-3d62-45bf-85fe-c247f518355c/TPCH.LINEITEM/0/b5904451d27d42f0bcb4c98a5b14f3e9
first=\x80\x00\x00\x00\x13%/\x83\x80\x00\x00\x01 last=\x80\x00\x00\x00\x14\x04\x08$\x80\x00\x00\x01
>>>>>>> 16/08/17 20:37:07 INFO mapreduce.LoadIncrementalHFiles: Trying
to load hfile=hdfs://stl-colo-srv073.splicemachine.colo:8020/tmp/66f905f4-3d62-45bf-85fe-c247f518355c/TPCH.LINEITEM/0/9d26e9a00e5149cabcb415c6bb429a34
first=\x80\x00\x00\x00\x06\xF6_\xE3\x80\x00\x00\x04 last=\x80\x00\x00\x00\x07\xD5 f\x80\x00\x00\x05
>>>>>>> 16/08/17 20:37:07 ERROR mapreduce.LoadIncrementalHFiles: Trying
to load more than 32 hfiles to family 0 of region with start key
>>>>>>> 16/08/17 20:37:07 INFO client.ConnectionManager$HConnectionImplementation:
Closing master protocol: MasterService
>>>>>>> 16/08/17 20:37:07 INFO client.ConnectionManager$HConnectionImplementation:
Closing zookeeper sessionid=0x15696476bf90485
>>>>>>> 16/08/17 20:37:07 INFO zookeeper.ZooKeeper: Session: 0x15696476bf90485
closed
>>>>>>> 16/08/17 20:37:07 INFO zookeeper.ClientCnxn: EventThread shut
down
>>>>>>> Exception in thread "main" java.io.IOException: Trying to load
more than 32 hfiles to one family of one region
>>>>>>>      at org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles.doBulkLoad(LoadIncrementalHFiles.java:420)
>>>>>>>      at org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles.doBulkLoad(LoadIncrementalHFiles.java:314)
>>>>>>>      at org.apache.phoenix.mapreduce.AbstractBulkLoadTool.completebulkload(AbstractBulkLoadTool.java:355)
>>>>>>>      at org.apache.phoenix.mapreduce.AbstractBulkLoadTool.submitJob(AbstractBulkLoadTool.java:332)
>>>>>>>      at org.apache.phoenix.mapreduce.AbstractBulkLoadTool.loadData(AbstractBulkLoadTool.java:270)
>>>>>>>      at org.apache.phoenix.mapreduce.AbstractBulkLoadTool.run(AbstractBulkLoadTool.java:183)
>>>>>>>      at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76)
>>>>>>>      at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:90)
>>>>>>>      at org.apache.phoenix.mapreduce.CsvBulkLoadTool.main(CsvBulkLoadTool.java:101)
>>>>>>>      at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>>>>>>>      at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
>>>>>>>      at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>>>>>>>      at java.lang.reflect.Method.invoke(Method.java:606)
>>>>>>>      at org.apache.hadoop.util.RunJar.run(RunJar.java:221)
>>>>>>>      at org.apache.hadoop.util.RunJar.main(RunJar.java:136)
>>>>>>> ################################################################################
>>>>>>> 
>>>>>>> a count of the table showa 0 rows:
>>>>>>> 0: jdbc:phoenix:srv073> select count(*) from TPCH.LINEITEM;
>>>>>>> +-----------+
>>>>>>> | COUNT(1)  |
>>>>>>> +-----------+
>>>>>>> | 0         |
>>>>>>> +-----------+
>>>>>>> 
>>>>>>> Some quick googling gives an hbase param that could be tweaked
(http://stackoverflow.com/questions/24950393/trying-to-load-more-than-32-hfiles-to-one-family-of-one-region).
>>>>>>> 
>>>>>>> Main Questions:
>>>>>>> - Will the CsvBulkLoadTool pick up these params, or will I need
to put them in hbase-site.xml?
>>>>>>> - Is there anything else I can tune to make this run quicker?
It took 5 hours for it to fail with the error above.
>>>>>>> 
>>>>>>> This is a 9 node (8 RegionServer) cluster running HDP 2.4.2 and
Phoenix 4.8.0-HBase-1.1
>>>>>>> Ambari default settings except for:
>>>>>>> - HBase RS heap size is set to 24GB
>>>>>>> - hbase.rpc.timeout set to 20 min
>>>>>>> - phoenix.query.timeoutMs set to 60 min
>>>>>>> 
>>>>>>> all nodes are Dell R420 with 2xE5-2430 v2 CPUs (24vCPU), 64GB
RAM
>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>> 
>>> 
> 


Mime
View raw message