Read from kafka "History"

312 views
Skip to first unread message

Mikhail Cherepnev

unread,
Aug 20, 2013, 9:05:36 AM8/20/13
to druid-de...@googlegroups.com
Hi everybody

If we run kafka and send there lots of messages and only after that we start realtime node does it fetch all messages from kafka topic, that were sent while realtime node was down?


I have strange behaviour of realtime node: I send lots of messages to kafka (they appear in kafka logs) but realtime node seems not fetching them. I restart RT node and for some time it begins fetching data (but from current time of kafka messages, not from last stopped) and then stops fetching again.

Is it problem with RT node or something else? or there is some problems with kafka-0.8? I got realization of KafkaFirehose from this google group: https://github.com/meeshaeel/druid/blob/kafka-0.8/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java

Any help appreciated

Fangjin Yang

unread,
Aug 20, 2013, 9:05:37 PM8/20/13
to druid-de...@googlegroups.com
HI Mikhail,

The realtime node will only commit offsets back to Kafka during its intermediate persists. These occur on a periodic basis depending on your config. Have you tried querying the RT node to see if it has ingested recent events? There are also metrics you can emit/log to see the counts of events ingested and thrown away. Let me know if this doesn't make sense.

--FJ

Mikhail Cherepnev

unread,
Aug 21, 2013, 2:54:21 AM8/21/13
to druid-de...@googlegroups.com

Hi Fangjin,

Querying RT node and broker node sometimes differs, but there is no a lot of records for a solid period of time, that persists in Kafka
Colud you explain how can I find metrics? In RT node log there is no such

Eric Tschetter

unread,
Aug 21, 2013, 3:06:42 PM8/21/13
to druid-de...@googlegroups.com
Mikhail,

To see the metrics, you need to adjust some config:

com.metamx.emitter.logging=true
com.metamx.emitter.logging.level=info

With that set, the node will dump metrics into the logs every 60 seconds.  You can look for "events/processed", "events/thrownAway" and "events/unparseable".

While the realtime node shouldn't stop consuming from Kafka, if you have a lot of old messages, it's very possible that it would throw them away (it "throws away" messages that are not within windowPeriod of the current time point.)  These should show up as "events/thrownAway" in the metrics dumped to the logs.  If the node is, in fact, not consuming from kafka, then that's very likely something on the kafka consumer side of things.  I'm not sure how stable 0.8 is right now (we use 0.7.2), but it would probably be beneficial to take a thread dump of the jvm and see what the various threads are doing.

--Eric


--
You received this message because you are subscribed to the Google Groups "Druid Development" group.
To unsubscribe from this group and stop receiving emails from it, send an email to druid-developm...@googlegroups.com.
To post to this group, send email to druid-de...@googlegroups.com.
To view this discussion on the web visit https://groups.google.com/d/msgid/druid-development/dd17964c-0131-49a2-bbad-2231b4b2cf43%40googlegroups.com.

For more options, visit https://groups.google.com/groups/opt_out.

Mikhail Cherepnev

unread,
Aug 22, 2013, 2:50:05 AM8/22/13
to druid-de...@googlegroups.com
Thanks Eric

After restart RT node with these params in config I have:

64339 [Global--0] INFO com.metamx.emitter.core.LoggingEmitter - Event [{"feed":"metrics","timestamp":"2013-08-22T06:32:31.110Z","service":"realtime","host":"ip-10-248-15-118:8082","metric":"events/unparseable","value":0,"user2":"datainsight"}]
64339 [Global--0] INFO com.metamx.emitter.core.LoggingEmitter - Event [{"feed":"metrics","timestamp":"2013-08-22T06:32:31.110Z","service":"realtime","host":"ip-10-248-15-118:8082","metric":"events/thrownAway","value":97308,"user2":"datainsight"}]
64339 [Global--0] INFO com.metamx.emitter.core.LoggingEmitter - Event [{"feed":"metrics","timestamp":"2013-08-22T06:32:31.110Z","service":"realtime","host":"ip-10-248-15-118:8082","metric":"events/processed","value":19163,"user2":"datainsight"}]

In the next iteration of emmiter all these are 0

I suppose it happens because of that (correct if I 'm wrong):
I loaded data to kafka with timestamps from 01.07.2013 to 05.08.2013. Some time later I started loading another data with timestamps from 01.07.2013 to 05.08.2013 but seems they were out of windowPeriod and thats why not loaded. Am I right? If that's true how to prevent such behaviour?

Use case is:
We have loader that reads log files and sends all rows to Kafka in JSON. Druid RT node reads this data from kafka. Idea is to load all old logs until realtime and then once an hour upload new logs. And it is possible to start uploading new log files of another type from long time ago.

Or to load older logs we must use Hadoop batch ingestion?

Hagen Rother

unread,
Aug 22, 2013, 2:58:15 AM8/22/13
to druid-de...@googlegroups.com
You can set

        "rejectionPolicy": {"type": "messageTime"},

in the in the plumber section of your realtime spec.

It defaults to "serverTime", which compares the timestamp to the current wall clock.

"messageTime" in contrast compares the timestamp to the last timestamp processed. That should give you your desired behavior.



For more options, visit https://groups.google.com/groups/opt_out.



--
Hagen Rother
Senior Software Architect | madvertise

madvertise Mobile Advertising GmbH

Managing Director | Christof Wittig
Jurisdiction | Local Court Berlin-Charlottenburg HRB 116329 B

Mikhail Cherepnev

unread,
Aug 22, 2013, 3:02:45 AM8/22/13
to druid-de...@googlegroups.com
I have "rejectionPolicy": {"type": "messageTime"} already:
here is my config:
[{
  "schema" : { "dataSource":"datainsight",
               "aggregators":[{"type":"longSum", "name":"imps", "fieldName":"imps"},
                               {"type":"longSum", "name":"clicks", "fieldName":"clicks"},
                               {"type":"longSum", "name":"bids", "fieldName":"bids"},
                               {"type":"longSum", "name":"pc_conversions", "fieldName":"pc_conversions"},
                               {"type":"longSum", "name":"pv_conversions", "fieldName":"pv_conversions"},
                               {"type": "doubleSum", "name": "adv_payout", "fieldName": "adv_payout"},
                               {"type": "doubleSum", "name": "pub_payout", "fieldName": "pub_payout"}],
               "indexGranularity":"hour",
               "shardSpec" : { "type": "none" } },
  "config" : { "maxRowsInMemory" : 500000,
               "intermediatePersistPeriod" : "PT10m" },
  "firehose" : { "type" : "kafka-0.8",
                 "consumerProps" : { "zookeeper.connect" : "localhost:2181",
                                     "zookeeper.connection.timeout.ms" : "15000",
                                     "zookeeper.session.timeout.ms" : "15000",
                                     "zookeeper.sync.time.ms" : "5000",
                                     "group.id" : "consumer-group",
                                     "fetch.size" : "1048586",
                                     "auto.offset.reset" : "largest",
                                     "auto.commit.enable" : "false" },
                 "feed" : "datainsight",
                 "parser" : { "timestampSpec" : { "column" : "date_time", "format" : "yyyy-MM-dd HH:mm:ss" },
                              "data" : { "format" : "json"},
                              "dimensionExclusions" : [] }},
  "plumber" : { "type" : "realtime",
                "windowPeriod" : "PT60m",
                "segmentGranularity":"hour",
                "basePersistDirectory" : "/mnt/datainsight/logs/druid/realtime/basePersist",
                "rejectionPolicy": {"type": "messageTime"} }
               
}]

Hagen Rother

unread,
Aug 22, 2013, 3:08:12 AM8/22/13
to druid-de...@googlegroups.com
yep, that really sounds like you are trying to turn realtime into your batch importer.

There is also policy "none" (haven't tried it myself)

Please be aware that druid will not "reopen" a segment, i.e. if you reload data for a certain period, you *must* provide all known data for that period (no delta loading).

Good luck!



For more options, visit https://groups.google.com/groups/opt_out.

Mikhail Cherepnev

unread,
Aug 22, 2013, 6:25:15 AM8/22/13
to druid-de...@googlegroups.com
What is the best way to load historic data?

We have logs in HDFS and S3 but their format is not JSON and there is no some fields that we need in Druid.
And we have loader to Kafka? that reads logs and performs some transformation of every log row to JSON with some calculated fields.
What is the best way to keep newest data in RT nodes and be able to load old data to Druid as in example above: we uploaded one type of logs and then we decided to upload another type of logs for the same dates in one druid datasource?

Need some advices on how to organize such process.

Thanks, Mikhail

Eric Tschetter

unread,
Aug 23, 2013, 10:46:20 AM8/23/13
to druid-de...@googlegroups.com
Mikhail,

For what you are doing, I believe you will be better served by the HadoopDruidIndexer.  It was created to handle old things, while the realtime node is intended to just handle what is happening "right now".  The rejectionPolicy let's you change the definition of what "right now" means, but it doesn't change the fact that it ignores things too far from "right now".

I would recommending setting up a Hadoop pipeline that does your transformations and then pushes that data into Druid using the HadoopDruidIndexer.

For what it's worth, our setup is to have events streaming into Kafka and have a realtime node consuming that.  We also have another consumer taking those events and storing them in S3.  Every evening, we take all of the events stored in S3 and re-process them with a Hadoop flow (to clean up artifacts introduced by Kafka or service failures during the day) that ends with the hadoop indexer.

--Eric


Mikhail Cherepnev

unread,
Aug 23, 2013, 11:03:22 AM8/23/13
to druid-de...@googlegroups.com
So every day you reload all day data ignoring what was loaded with realtime node?
How about consistency? What if we made some reports according to realtime data and tomorrow we will see another values int hose reports? Theoretically it can happen?


What if to load old data we will make files with json that equal to json from kafka stream and store them in HDFS. And then can we load such files using HadoopDruidIndexer? Is there some examples on how to load list of files from HDFS to Druid Historic nodes? And what if druid already has those segments in HDFS storage? Will they be overrode or supplemented?

Eric Tschetter

unread,
Aug 23, 2013, 11:12:00 AM8/23/13
to druid-de...@googlegroups.com
So every day you reload all day data ignoring what was loaded with realtime node?
How about consistency? What if we made some reports according to realtime data and tomorrow we will see another values int hose reports? Theoretically it can happen?

Yes, we replace the data that the realtime node ingested.  For our own pipelines, we trust the Hadoop-processed numbers more than we trust the events that came in over the realtime streams.  Kafka maintains "at least once" semantics and we have definitely seen duplicates introduced by Kafka as well as just intrinsically existing in the input data from our customers.

It is true that the reports would differ, but, in our case, the Hadoop-generated data is more correct so the change in numbers is fixing the old "bad" numbers and thus those numbers _should_ change anyway.
 

What if to load old data we will make files with json that equal to json from kafka stream and store them in HDFS. And then can we load such files using HadoopDruidIndexer? Is there some examples on how to load list of files from HDFS to Druid Historic nodes? And what if druid already has those segments in HDFS storage? Will they be overrode or supplemented?

Yes, that would work.  One of our tutorials covers the general process of loading data from Hadoop, scroll down to the "Loading with HadoopDruidIndexer" part (https://github.com/metamx/druid/wiki/Loading-Your-Data) and this page https://github.com/metamx/druid/wiki/Batch-ingestion just generally discusses the HadoopDruidIndexer.

When you process the data with the HadoopDruidIndexer, you should process _all_ of the data for that time period (as Hagen said, you are not loading deltas, but loading the whole world for that period).  The system will handle the new segments, you don't have to do anything (once the new segments appear, the Master will handle swapping them in to replace any old ones that currently exist).

--Eric

 

Mikhail Cherepnev

unread,
Aug 26, 2013, 10:57:43 AM8/26/13
to druid-de...@googlegroups.com
Eric,

Could you explain difference between granularitySpec.gran and rollupSpec.rollupGranularity parameters in HadoopDruidIndexer config?
If I want to reaload all day with granularity as in RT config (one hour) should I use such config params? :

...
"granularitySpec": {
    "type":"uniform",
    "intervals": ["2013-06-27T00:00:00Z/P1D"],
    "gran":"hour"
  },
 "pathSpec": { "type": "static",
                    "paths": "hdfs://my-server:8020/log1/2013-06-27,hdfs://my-server:8020/log2/2013-06-27,hdfs://my-server:8020/log3/2013-06-27,hdfs://my-server:8020/log4/2013-06-27"},
  "rollupSpec": { "aggs": [
                   ...
                  ],
                  "rollupGranularity": "hour"}
....


And why there is no such convenient option for logs in JSON format as we have in RT node config: we declare only dimensionExclusions and all dimensions are identified automatically?
It would be perfect to use same config files for RT node, compute node and batch loaders somehow... Or same config parts...
There is a little mess: we use .properties and .json file types to config nodes and everywhere have properties with same values (i.e druid.database.connectURI in properties and updaterJobSpec.connectURI in json)


Anyway thanks a lot for a Great job! You are doing amazing thing!

Eric Tschetter

unread,
Aug 26, 2013, 6:11:21 PM8/26/13
to druid-de...@googlegroups.com
Could you explain difference between granularitySpec.gran and rollupSpec.rollupGranularity parameters in HadoopDruidIndexer config?
If I want to reaload all day with granularity as in RT config (one hour) should I use such config params? :

...
"granularitySpec": {
    "type":"uniform",
    "intervals": ["2013-06-27T00:00:00Z/P1D"],
    "gran":"hour"
  },
 "pathSpec": { "type": "static",
                    "paths": "hdfs://my-server:8020/log1/2013-06-27,hdfs://my-server:8020/log2/2013-06-27,hdfs://my-server:8020/log3/2013-06-27,hdfs://my-server:8020/log4/2013-06-27"},
  "rollupSpec": { "aggs": [
                   ...
                  ],
                  "rollupGranularity": "hour"}
....


granularitySpec is specifying the "granularity" of the segments.  So, the specified value will create one segment for each hour of the day.  We generally run with that at day, fwiw, but the best settings depends on your data and your use case.  It's a tradeoff between the number of segments and the size of segments, for the Hadoop indexer, it will automatically partition the segments up into separate chunks if it believes the segments are too big, so it's generally safe to specify a higher granularity and a targetPartitionSize and then let the code adapt to those settings.

rollupGranularity is specifying the rollup you want to do on the data.  Specifying hour will actually store the data at hourly granularity.  It is completely possible to specify granularitySpec.gran of "day" and rollupSpec.rollupGranularity of "hour", you will then store 24 hours worth of data in each segment.

 

And why there is no such convenient option for logs in JSON format as we have in RT node config: we declare only dimensionExclusions and all dimensions are identified automatically? 
It would be perfect to use same config files for RT node, compute node and batch loaders somehow... Or same config parts...
There is a little mess: we use .properties and .json file types to config nodes and everywhere have properties with same values (i.e druid.database.connectURI in properties and updaterJobSpec.connectURI in json)

I completely agree.  The only answer is legacy reasons.  This is one of the prices you pay for building something for an immediate use case and iterating on it: you often have better ideas later on in the life of the software and don't have/take the time to go back and make everything consistent.

Fwiw, I generally prefer json configuration over properties and am slowly migrating things in that direction.  The RT node config came after the Hadoop config and we didn't go back and adjust Hadoop to work with just dimensionExclusions.  Fwiw, this is an area that I would be more than happy to accept a pull request for :). 

--Eric

 

Mikhail Cherepnev

unread,
Aug 27, 2013, 3:41:31 AM8/27/13
to druid-de...@googlegroups.com
Eric, thanks for explanations

About pull request - think you'll get it sometime :)

HadoopDruidIndexer job started but failed wwith errors like that:
2013-08-27 00:43:00,358 WARN org.apache.hadoop.mapred.Child: Error running child
com.metamx.common.RE: Failure on row[{"date_time":"2013-06-27 23:44:59","daily_table":"2013-06-27","hour":"23","exchange":"google","publisher_id":"0","size":"728x90","advertiser_id":"39","campaign_id":"139","line_item_id":"11477","pricing_type":"CPM","creative_id":"2894","browser":"Other","os":"Win_7","geo_country":"RU","geo_region":"88","country_region":"RU-88","city":"Rostov","bids":1}]
	at com.metamx.druid.indexer.HadoopDruidIndexerMapper.map(HadoopDruidIndexerMapper.java:60)
	at com.metamx.druid.indexer.HadoopDruidIndexerMapper.map(HadoopDruidIndexerMapper.java:13)
	at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:140)
	at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:672)
	at org.apache.hadoop.mapred.MapTask.run(MapTask.java:330)
	at org.apache.hadoop.mapred.Child$4.run(Child.java:268)
	at java.security.AccessController.doPrivileged(Native Method)
	at javax.security.auth.Subject.doAs(Subject.java:396)
	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1408)
	at org.apache.hadoop.mapred.Child.main(Child.java:262)
Caused by: com.metamx.common.ISE: WTF?! No bucket found for row: MapBasedInputRow{timestamp=2013-06-27T23:44:59.000+04:00, event={date_time=2013-06-27 23:44:59, daily_table=2013-06-27, hour=23, exchange=google, publisher_id=0, size=728x90, advertiser_id=39, campaign_id=139, line_item_id=11477, pricing_type=CPM, creative_id=2894, browser=Other, os=Win_7, geo_country=RU, geo_region=88, country_region=RU-88, city=Rostov, bids=1}, dimensions=[daily_table, hour, exchange, publisher_id, size, advertiser_id, campaign_id, line_item_id, pricing_type, creative_id, browser, os, geo_country, geo_region, country_region, city]}
	at com.metamx.druid.indexer.IndexGeneratorJob$IndexGeneratorMapper.innerMap(IndexGeneratorJob.java:203)
	at com.metamx.druid.indexer.HadoopDruidIndexerMapper.map(HadoopDruidIndexerMapper.java:56)
	... 9 more

I want to reload all data for 2013-06-27 and have logs for that date in one subfolder in HDFS. But it seems that last hours of that day are out of that day because of timezones. When I start job I use parameter -Duser.timezone=UTC
Am I right with my explanation of that error?

Mikhail Cherepnev

unread,
Aug 27, 2013, 8:01:10 AM8/27/13
to druid-de...@googlegroups.com
I tried to load different files, all hadoop jobs finished successfully but in console I've got error.
Here is full log of running HadoopDruidIndexer:
2013-08-27 11:47:33,463 INFO [main] com.metamx.common.lifecycle.Lifecycle$AnnotationBasedHandler - Invoking start method[public void com.metamx.druid.indexer.HadoopDruidIndexerNode.start() throws java.lang.Exception] on object[com.metamx.druid.indexer.HadoopDruidIndexerNode@7a3f437c].
2013-08-27 11:47:34,431 INFO [main] com.metamx.druid.indexer.HadoopDruidIndexerConfig - Running with config:
{
  "dataSource" : "datainsight",
  "timestampColumn" : "date_time",
  "timestampFormat" : "yyyy-MM-dd HH:mm:ss",
  "dataSpec" : {
    "format" : "json",
    "dimensions" : [ "daily_table", "hour", "exchange", "publisher_id", "size", "advertiser_id", "campaign_id", "line_item_id", "pricing_type", "creative_id", "browser", "os", "geo_country", "geo_region", "country_region", "city" ],
    "spatialDimensions" : [ ]
  },
  "granularitySpec" : {
    "type" : "uniform",
    "gran" : "HOUR",
    "intervals" : [ "2013-07-01T00:00:00.000Z/2013-07-02T00:00:00.000Z" ]

  },
  "pathSpec" : {
    "type" : "static",
    "paths" : "hdfs://my-server:8020/user/user1/logs/druid/log_imp/2013-07-01"
  },
  "workingPath" : "hdfs://my-server:8020/tmp",
  "segmentOutputPath" : "hdfs://my-server:8020/druid",
  "version" : "2013-08-27T11:47:34.314Z",
  "partitionsSpec" : {
    "partitionDimension" : null,
    "targetPartitionSize" : 5000000,
    "maxPartitionSize" : 7500000,
    "assumeGrouped" : false
  },
  "leaveIntermediate" : false,
  "cleanupOnFailure" : true,
  "shardSpecs" : null,
  "overwriteFiles" : false,
  "rollupSpec" : {
    "aggs" : [ {
    "rollupGranularity" : {
      "type" : "duration",
      "duration" : 60000,
      "origin" : "1970-01-01T00:00:00.000Z"
    },
    "rowFlushBoundary" : 500000
  },
  "updaterJobSpec" : {
    "type" : "db",
    "connectURI" : "jdbc:mysql://my-server:3306/druid",
    "user" : "druid",
    "password" : "diurd",
    "segmentTable" : "segments",
    "useValidationQuery" : false,
    "validationQuery" : "SELECT 1"
  },
  "ignoreInvalidRows" : false,
  "registererers" : null
}
2013-08-27 11:47:34,855 WARN [main] org.apache.hadoop.util.NativeCodeLoader - Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
2013-08-27 11:47:34,869 INFO [main] com.metamx.druid.indexer.path.StaticPathSpec - Adding paths[hdfs://my-server:8020/user/user1/logs/druid/log_imp/2013-07-01]
2013-08-27 11:47:35,697 INFO [main] com.metamx.druid.indexer.path.StaticPathSpec - Adding paths[hdfs://my-server:8020/user/user1/logs/druid/log_imp/2013-07-01]
2013-08-27 11:47:35,927 WARN [main] org.apache.hadoop.mapred.JobClient - Use GenericOptionsParser for parsing the arguments. Applications should implement Tool for the same.
2013-08-27 11:47:37,115 INFO [main] org.apache.hadoop.mapreduce.lib.input.FileInputFormat - Total input paths to process : 3
2013-08-27 11:47:37,518 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Job datainsight-determine_partitions_groupby-[2013-07-01T00:00:00.000Z/2013-07-02T00:00:00.000Z] submitted, status available at: http://my-server:50030/jobdetails.jsp?jobid=job_201307151148_1459
2013-08-27 11:47:37,519 INFO [main] org.apache.hadoop.mapred.JobClient - Running job: job_201307151148_1459
2013-08-27 11:47:38,523 INFO [main] org.apache.hadoop.mapred.JobClient -  map 0% reduce 0%
2013-08-27 11:47:48,561 INFO [main] org.apache.hadoop.mapred.JobClient -  map 33% reduce 0%
2013-08-27 11:47:51,572 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 0%
2013-08-27 11:47:54,583 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 100%
2013-08-27 11:47:56,593 INFO [main] org.apache.hadoop.mapred.JobClient - Job complete: job_201307151148_1459
2013-08-27 11:47:56,606 INFO [main] org.apache.hadoop.mapred.JobClient - Counters: 32
2013-08-27 11:47:56,608 INFO [main] org.apache.hadoop.mapred.JobClient -   File System Counters
2013-08-27 11:47:56,610 INFO [main] org.apache.hadoop.mapred.JobClient -     FILE: Number of bytes read=6
2013-08-27 11:47:56,610 INFO [main] org.apache.hadoop.mapred.JobClient -     FILE: Number of bytes written=662007
2013-08-27 11:47:56,611 INFO [main] org.apache.hadoop.mapred.JobClient -     FILE: Number of read operations=0
2013-08-27 11:47:56,611 INFO [main] org.apache.hadoop.mapred.JobClient -     FILE: Number of large read operations=0
2013-08-27 11:47:56,611 INFO [main] org.apache.hadoop.mapred.JobClient -     FILE: Number of write operations=0
2013-08-27 11:47:56,611 INFO [main] org.apache.hadoop.mapred.JobClient -     HDFS: Number of bytes read=1664064
2013-08-27 11:47:56,611 INFO [main] org.apache.hadoop.mapred.JobClient -     HDFS: Number of bytes written=95
2013-08-27 11:47:56,611 INFO [main] org.apache.hadoop.mapred.JobClient -     HDFS: Number of read operations=6
2013-08-27 11:47:56,611 INFO [main] org.apache.hadoop.mapred.JobClient -     HDFS: Number of large read operations=0
2013-08-27 11:47:56,611 INFO [main] org.apache.hadoop.mapred.JobClient -     HDFS: Number of write operations=1
2013-08-27 11:47:56,613 INFO [main] org.apache.hadoop.mapred.JobClient -   Job Counters
2013-08-27 11:47:56,613 INFO [main] org.apache.hadoop.mapred.JobClient -     Launched map tasks=3
2013-08-27 11:47:56,614 INFO [main] org.apache.hadoop.mapred.JobClient -     Launched reduce tasks=1
2013-08-27 11:47:56,614 INFO [main] org.apache.hadoop.mapred.JobClient -     Data-local map tasks=3
2013-08-27 11:47:56,614 INFO [main] org.apache.hadoop.mapred.JobClient -     Total time spent by all maps in occupied slots (ms)=25685
2013-08-27 11:47:56,614 INFO [main] org.apache.hadoop.mapred.JobClient -     Total time spent by all reduces in occupied slots (ms)=5673
2013-08-27 11:47:56,614 INFO [main] org.apache.hadoop.mapred.JobClient -     Total time spent by all maps waiting after reserving slots (ms)=0
2013-08-27 11:47:56,614 INFO [main] org.apache.hadoop.mapred.JobClient -     Total time spent by all reduces waiting after reserving slots (ms)=0
2013-08-27 11:47:56,616 INFO [main] org.apache.hadoop.mapred.JobClient -   Map-Reduce Framework
2013-08-27 11:47:56,616 INFO [main] org.apache.hadoop.mapred.JobClient -     Map input records=4202
2013-08-27 11:47:56,616 INFO [main] org.apache.hadoop.mapred.JobClient -     Map output records=0
2013-08-27 11:47:56,616 INFO [main] org.apache.hadoop.mapred.JobClient -     Map output bytes=0
2013-08-27 11:47:56,616 INFO [main] org.apache.hadoop.mapred.JobClient -     Input split bytes=594
2013-08-27 11:47:56,616 INFO [main] org.apache.hadoop.mapred.JobClient -     Combine input records=0
2013-08-27 11:47:56,616 INFO [main] org.apache.hadoop.mapred.JobClient -     Combine output records=0
2013-08-27 11:47:56,616 INFO [main] org.apache.hadoop.mapred.JobClient -     Reduce input groups=0
2013-08-27 11:47:56,617 INFO [main] org.apache.hadoop.mapred.JobClient -     Reduce shuffle bytes=18
2013-08-27 11:47:56,617 INFO [main] org.apache.hadoop.mapred.JobClient -     Reduce input records=0
2013-08-27 11:47:56,617 INFO [main] org.apache.hadoop.mapred.JobClient -     Reduce output records=0
2013-08-27 11:47:56,617 INFO [main] org.apache.hadoop.mapred.JobClient -     Spilled Records=0
2013-08-27 11:47:56,617 INFO [main] org.apache.hadoop.mapred.JobClient -     CPU time spent (ms)=9810
2013-08-27 11:47:56,617 INFO [main] org.apache.hadoop.mapred.JobClient -     Physical memory (bytes) snapshot=826146816
2013-08-27 11:47:56,617 INFO [main] org.apache.hadoop.mapred.JobClient -     Virtual memory (bytes) snapshot=3951435776
2013-08-27 11:47:56,617 INFO [main] org.apache.hadoop.mapred.JobClient -     Total committed heap usage (bytes)=672727040
2013-08-27 11:47:56,661 WARN [main] org.apache.hadoop.mapred.JobClient - Use GenericOptionsParser for parsing the arguments. Applications should implement Tool for the same.
2013-08-27 11:47:57,465 INFO [main] org.apache.hadoop.mapreduce.lib.input.FileInputFormat - Total input paths to process : 1
2013-08-27 11:47:57,679 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Job datainsight-determine_partitions_dimselection-[2013-07-01T00:00:00.000Z/2013-07-02T00:00:00.000Z] submitted, status available at: http://my-server:50030/jobdetails.jsp?jobid=job_201307151148_1460
2013-08-27 11:47:57,679 INFO [main] org.apache.hadoop.mapred.JobClient - Running job: job_201307151148_1460
2013-08-27 11:47:58,681 INFO [main] org.apache.hadoop.mapred.JobClient -  map 0% reduce 0%
2013-08-27 11:48:07,713 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 0%
2013-08-27 11:48:13,737 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 8%
2013-08-27 11:48:17,751 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 13%
2013-08-27 11:48:18,755 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 17%
2013-08-27 11:48:21,765 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 21%
2013-08-27 11:48:23,771 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 25%
2013-08-27 11:48:25,778 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 29%
2013-08-27 11:48:28,788 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 33%
2013-08-27 11:48:29,791 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 37%
2013-08-27 11:48:32,800 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 42%
2013-08-27 11:48:34,807 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 46%
2013-08-27 11:48:37,816 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 50%
2013-08-27 11:48:38,819 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 54%
2013-08-27 11:48:41,830 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 58%
2013-08-27 11:48:42,833 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 63%
2013-08-27 11:48:46,849 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 67%
2013-08-27 11:48:47,853 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 71%
2013-08-27 11:48:51,874 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 79%
2013-08-27 11:48:55,887 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 88%
2013-08-27 11:49:00,903 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 96%
2013-08-27 11:49:04,920 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 100%
2013-08-27 11:49:06,930 INFO [main] org.apache.hadoop.mapred.JobClient - Job complete: job_201307151148_1460
2013-08-27 11:49:06,933 INFO [main] org.apache.hadoop.mapred.JobClient - Counters: 32
2013-08-27 11:49:06,933 INFO [main] org.apache.hadoop.mapred.JobClient -   File System Counters
2013-08-27 11:49:06,933 INFO [main] org.apache.hadoop.mapred.JobClient -     FILE: Number of bytes read=144
2013-08-27 11:49:06,933 INFO [main] org.apache.hadoop.mapred.JobClient -     FILE: Number of bytes written=4167174
2013-08-27 11:49:06,933 INFO [main] org.apache.hadoop.mapred.JobClient -     FILE: Number of read operations=0
2013-08-27 11:49:06,933 INFO [main] org.apache.hadoop.mapred.JobClient -     FILE: Number of large read operations=0
2013-08-27 11:49:06,933 INFO [main] org.apache.hadoop.mapred.JobClient -     FILE: Number of write operations=0
2013-08-27 11:49:06,933 INFO [main] org.apache.hadoop.mapred.JobClient -     HDFS: Number of bytes read=249
2013-08-27 11:49:06,934 INFO [main] org.apache.hadoop.mapred.JobClient -     HDFS: Number of bytes written=0
2013-08-27 11:49:06,934 INFO [main] org.apache.hadoop.mapred.JobClient -     HDFS: Number of read operations=3
2013-08-27 11:49:06,934 INFO [main] org.apache.hadoop.mapred.JobClient -     HDFS: Number of large read operations=0
2013-08-27 11:49:06,934 INFO [main] org.apache.hadoop.mapred.JobClient -     HDFS: Number of write operations=0
2013-08-27 11:49:06,934 INFO [main] org.apache.hadoop.mapred.JobClient -   Job Counters
2013-08-27 11:49:06,934 INFO [main] org.apache.hadoop.mapred.JobClient -     Launched map tasks=1
2013-08-27 11:49:06,934 INFO [main] org.apache.hadoop.mapred.JobClient -     Launched reduce tasks=25
2013-08-27 11:49:06,934 INFO [main] org.apache.hadoop.mapred.JobClient -     Data-local map tasks=1
2013-08-27 11:49:06,934 INFO [main] org.apache.hadoop.mapred.JobClient -     Total time spent by all maps in occupied slots (ms)=9414
2013-08-27 11:49:06,934 INFO [main] org.apache.hadoop.mapred.JobClient -     Total time spent by all reduces in occupied slots (ms)=106428
2013-08-27 11:49:06,934 INFO [main] org.apache.hadoop.mapred.JobClient -     Total time spent by all maps waiting after reserving slots (ms)=0
2013-08-27 11:49:06,935 INFO [main] org.apache.hadoop.mapred.JobClient -     Total time spent by all reduces waiting after reserving slots (ms)=0
2013-08-27 11:49:06,935 INFO [main] org.apache.hadoop.mapred.JobClient -   Map-Reduce Framework
2013-08-27 11:49:06,935 INFO [main] org.apache.hadoop.mapred.JobClient -     Map input records=0
2013-08-27 11:49:06,935 INFO [main] org.apache.hadoop.mapred.JobClient -     Map output records=0
2013-08-27 11:49:06,935 INFO [main] org.apache.hadoop.mapred.JobClient -     Map output bytes=0
2013-08-27 11:49:06,935 INFO [main] org.apache.hadoop.mapred.JobClient -     Input split bytes=154
2013-08-27 11:49:06,935 INFO [main] org.apache.hadoop.mapred.JobClient -     Combine input records=0
2013-08-27 11:49:06,935 INFO [main] org.apache.hadoop.mapred.JobClient -     Combine output records=0
2013-08-27 11:49:06,935 INFO [main] org.apache.hadoop.mapred.JobClient -     Reduce input groups=0
2013-08-27 11:49:06,935 INFO [main] org.apache.hadoop.mapred.JobClient -     Reduce shuffle bytes=144
2013-08-27 11:49:06,935 INFO [main] org.apache.hadoop.mapred.JobClient -     Reduce input records=0
2013-08-27 11:49:06,935 INFO [main] org.apache.hadoop.mapred.JobClient -     Reduce output records=0
2013-08-27 11:49:06,936 INFO [main] org.apache.hadoop.mapred.JobClient -     Spilled Records=0
2013-08-27 11:49:06,936 INFO [main] org.apache.hadoop.mapred.JobClient -     CPU time spent (ms)=66260
2013-08-27 11:49:06,936 INFO [main] org.apache.hadoop.mapred.JobClient -     Physical memory (bytes) snapshot=3491840000
2013-08-27 11:49:06,936 INFO [main] org.apache.hadoop.mapred.JobClient -     Virtual memory (bytes) snapshot=24944799744
2013-08-27 11:49:06,936 INFO [main] org.apache.hadoop.mapred.JobClient -     Total committed heap usage (bytes)=3010920448
2013-08-27 11:49:06,940 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Job completed, loading up partitions for intervals[[2013-07-01T00:00:00.000Z/2013-07-01T01:00:00.000Z, 2013-07-01T01:00:00.000Z/2013-07-01T02:00:00.000Z, 2013-07-01T02:00:00.000Z/2013-07-01T03:00:00.000Z, 2013-07-01T03:00:00.000Z/2013-07-01T04:00:00.000Z, 2013-07-01T04:00:00.000Z/2013-07-01T05:00:00.000Z, 2013-07-01T05:00:00.000Z/2013-07-01T06:00:00.000Z, 2013-07-01T06:00:00.000Z/2013-07-01T07:00:00.000Z, 2013-07-01T07:00:00.000Z/2013-07-01T08:00:00.000Z, 2013-07-01T08:00:00.000Z/2013-07-01T09:00:00.000Z, 2013-07-01T09:00:00.000Z/2013-07-01T10:00:00.000Z, 2013-07-01T10:00:00.000Z/2013-07-01T11:00:00.000Z, 2013-07-01T11:00:00.000Z/2013-07-01T12:00:00.000Z, 2013-07-01T12:00:00.000Z/2013-07-01T13:00:00.000Z, 2013-07-01T13:00:00.000Z/2013-07-01T14:00:00.000Z, 2013-07-01T14:00:00.000Z/2013-07-01T15:00:00.000Z, 2013-07-01T15:00:00.000Z/2013-07-01T16:00:00.000Z, 2013-07-01T16:00:00.000Z/2013-07-01T17:00:00.000Z, 2013-07-01T17:00:00.000Z/2013-07-01T18:00:00.000Z, 2013-07-01T18:00:00.000Z/2013-07-01T19:00:00.000Z, 2013-07-01T19:00:00.000Z/2013-07-01T20:00:00.000Z, 2013-07-01T20:00:00.000Z/2013-07-01T21:00:00.000Z, 2013-07-01T21:00:00.000Z/2013-07-01T22:00:00.000Z, 2013-07-01T22:00:00.000Z/2013-07-01T23:00:00.000Z, 2013-07-01T23:00:00.000Z/2013-07-02T00:00:00.000Z]].
2013-08-27 11:49:06,946 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T000000.000Z_20130701T010000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:06,948 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T010000.000Z_20130701T020000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:06,951 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T020000.000Z_20130701T030000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:06,956 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T030000.000Z_20130701T040000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:06,959 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T040000.000Z_20130701T050000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:06,963 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T050000.000Z_20130701T060000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:06,965 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T060000.000Z_20130701T070000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:06,967 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T070000.000Z_20130701T080000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:06,970 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T080000.000Z_20130701T090000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:06,972 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T090000.000Z_20130701T100000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:06,974 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T100000.000Z_20130701T110000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:06,976 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T110000.000Z_20130701T120000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:06,979 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T120000.000Z_20130701T130000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:06,981 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T130000.000Z_20130701T140000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:06,983 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T140000.000Z_20130701T150000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:06,985 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T150000.000Z_20130701T160000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:06,990 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T160000.000Z_20130701T170000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:06,992 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T170000.000Z_20130701T180000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:06,993 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T180000.000Z_20130701T190000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:06,995 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T190000.000Z_20130701T200000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:06,997 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T200000.000Z_20130701T210000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:06,999 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T210000.000Z_20130701T220000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:07,001 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T220000.000Z_20130701T230000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:07,003 INFO [main] com.metamx.druid.indexer.DeterminePartitionsJob - Path[hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/20130701T230000.000Z_20130702T000000.000Z/partitions.json] didn't exist!?
2013-08-27 11:49:07,029 INFO [main] com.metamx.druid.indexer.path.StaticPathSpec - Adding paths[hdfs://my-server:8020/user/user1/logs/druid/log_imp/2013-07-01]
2013-08-27 11:49:07,039 WARN [main] org.apache.hadoop.mapred.JobClient - Use GenericOptionsParser for parsing the arguments. Applications should implement Tool for the same.
2013-08-27 11:49:07,936 INFO [main] org.apache.hadoop.mapreduce.lib.input.FileInputFormat - Total input paths to process : 3
2013-08-27 11:49:08,262 INFO [main] com.metamx.druid.indexer.IndexGeneratorJob - Job datainsight-index-generator-[2013-07-01T00:00:00.000Z/2013-07-02T00:00:00.000Z] submitted, status available at http://my-server:50030/jobdetails.jsp?jobid=job_201307151148_1461
2013-08-27 11:49:08,262 INFO [main] org.apache.hadoop.mapred.JobClient - Running job: job_201307151148_1461
2013-08-27 11:49:09,264 INFO [main] org.apache.hadoop.mapred.JobClient -  map 0% reduce 0%
2013-08-27 11:49:21,304 INFO [main] org.apache.hadoop.mapred.JobClient -  map 33% reduce 0%
2013-08-27 11:49:25,317 INFO [main] org.apache.hadoop.mapred.JobClient -  map 100% reduce 0%
2013-08-27 11:49:27,326 INFO [main] org.apache.hadoop.mapred.JobClient - Job complete: job_201307151148_1461
2013-08-27 11:49:27,329 INFO [main] org.apache.hadoop.mapred.JobClient - Counters: 24
2013-08-27 11:49:27,329 INFO [main] org.apache.hadoop.mapred.JobClient -   File System Counters
2013-08-27 11:49:27,329 INFO [main] org.apache.hadoop.mapred.JobClient -     FILE: Number of bytes read=0
2013-08-27 11:49:27,329 INFO [main] org.apache.hadoop.mapred.JobClient -     FILE: Number of bytes written=496559
2013-08-27 11:49:27,329 INFO [main] org.apache.hadoop.mapred.JobClient -     FILE: Number of read operations=0
2013-08-27 11:49:27,330 INFO [main] org.apache.hadoop.mapred.JobClient -     FILE: Number of large read operations=0
2013-08-27 11:49:27,330 INFO [main] org.apache.hadoop.mapred.JobClient -     FILE: Number of write operations=0
2013-08-27 11:49:27,330 INFO [main] org.apache.hadoop.mapred.JobClient -     HDFS: Number of bytes read=1664064
2013-08-27 11:49:27,330 INFO [main] org.apache.hadoop.mapred.JobClient -     HDFS: Number of bytes written=0
2013-08-27 11:49:27,330 INFO [main] org.apache.hadoop.mapred.JobClient -     HDFS: Number of read operations=6
2013-08-27 11:49:27,331 INFO [main] org.apache.hadoop.mapred.JobClient -     HDFS: Number of large read operations=0
2013-08-27 11:49:27,331 INFO [main] org.apache.hadoop.mapred.JobClient -     HDFS: Number of write operations=3
2013-08-27 11:49:27,331 INFO [main] org.apache.hadoop.mapred.JobClient -   Job Counters
2013-08-27 11:49:27,331 INFO [main] org.apache.hadoop.mapred.JobClient -     Launched map tasks=3
2013-08-27 11:49:27,331 INFO [main] org.apache.hadoop.mapred.JobClient -     Data-local map tasks=3
2013-08-27 11:49:27,331 INFO [main] org.apache.hadoop.mapred.JobClient -     Total time spent by all maps in occupied slots (ms)=28038
2013-08-27 11:49:27,332 INFO [main] org.apache.hadoop.mapred.JobClient -     Total time spent by all reduces in occupied slots (ms)=0
2013-08-27 11:49:27,332 INFO [main] org.apache.hadoop.mapred.JobClient -     Total time spent by all maps waiting after reserving slots (ms)=0
2013-08-27 11:49:27,332 INFO [main] org.apache.hadoop.mapred.JobClient -     Total time spent by all reduces waiting after reserving slots (ms)=0
2013-08-27 11:49:27,332 INFO [main] org.apache.hadoop.mapred.JobClient -   Map-Reduce Framework
2013-08-27 11:49:27,332 INFO [main] org.apache.hadoop.mapred.JobClient -     Map input records=4202
2013-08-27 11:49:27,332 INFO [main] org.apache.hadoop.mapred.JobClient -     Map output records=0
2013-08-27 11:49:27,333 INFO [main] org.apache.hadoop.mapred.JobClient -     Input split bytes=594
2013-08-27 11:49:27,333 INFO [main] org.apache.hadoop.mapred.JobClient -     Spilled Records=0
2013-08-27 11:49:27,333 INFO [main] org.apache.hadoop.mapred.JobClient -     CPU time spent (ms)=7080
2013-08-27 11:49:27,333 INFO [main] org.apache.hadoop.mapred.JobClient -     Physical memory (bytes) snapshot=382013440
2013-08-27 11:49:27,333 INFO [main] org.apache.hadoop.mapred.JobClient -     Virtual memory (bytes) snapshot=2971459584
2013-08-27 11:49:27,333 INFO [main] org.apache.hadoop.mapred.JobClient -     Total committed heap usage (bytes)=351141888
2013-08-27 11:49:27,388 INFO [main] com.metamx.druid.indexer.HadoopDruidIndexerMain - Throwable caught at startup, committing seppuku
java.lang.reflect.InvocationTargetException
    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:601)
    at com.metamx.common.lifecycle.Lifecycle$AnnotationBasedHandler.start(Lifecycle.java:331)
    at com.metamx.common.lifecycle.Lifecycle.start(Lifecycle.java:250)
    at com.metamx.druid.indexer.HadoopDruidIndexerMain.main(HadoopDruidIndexerMain.java:50)
Caused by: java.lang.RuntimeException: java.io.FileNotFoundException: File hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/segmentDescriptorInfo does not exist.
    at com.google.common.base.Throwables.propagate(Throwables.java:160)
    at com.metamx.druid.indexer.IndexGeneratorJob.getPublishedSegments(IndexGeneratorJob.java:182)
    at com.metamx.druid.indexer.DbUpdaterJob.run(DbUpdaterJob.java:61)
    at com.metamx.druid.indexer.HadoopDruidIndexerJob.run(HadoopDruidIndexerJob.java:99)
    at com.metamx.druid.indexer.HadoopDruidIndexerNode.start(HadoopDruidIndexerNode.java:125)
    ... 7 more
Caused by: java.io.FileNotFoundException: File hdfs://my-server:8020/tmp/datainsight/2013-08-27T114734.314Z/segmentDescriptorInfo does not exist.
    at org.apache.hadoop.hdfs.DistributedFileSystem.listStatus(DistributedFileSystem.java:409)
    at com.metamx.druid.indexer.IndexGeneratorJob.getPublishedSegments(IndexGeneratorJob.java:175)
    ... 10 more


In HDFS I have empty part-m-* and groupedData/part-r-* files for this job, but loaded files are not empty. There is json documents separated with new lines

Eric Tschetter

unread,
Aug 27, 2013, 11:09:40 AM8/27/13
to druid-de...@googlegroups.com
Mikhail,

I think you are running into an issue with the timestamp.  Specifically, Hadoop confuses matters with setting the timezone if UTC is not the default already on all of your machines.  Hadoop basically runs a bunch of processes remotely that won't see your "-Duser.timezone=UTC" property set on the client process.  You have to set the property "mapred.child.java.opts" either in your hadoop configs or in a way that it makes it into the configuration.

For our Hadoop jobs, we push all properties that are prefixed with "hadoop." forward into the configuration, so you can set it by setting  "-Dhadoop.mapred.child.java.opts=-Duser.timezone=UTC".  If your config is already setting things for mapred.child.java.opts, then you will need to duplicate them here as well in order to keep them (this will overwrite whatever the property was set to).  Try that and it should work.

For the second email, notice the "Map Output Records" values on each of the jobs.  Both of them are 0 meaning that the input rows are getting filtered out in the mappers.  The only filtering operation we do is based on timestamp, so I'm fairly certain it is a timezone issue.

--Eric



Reply all
Reply to author
Forward
0 new messages