Loading and Querying data via Batch Ingestion

320 views
Skip to first unread message

Trotter Cashion

unread,
Oct 28, 2012, 10:31:58 PM10/28/12
to druid-de...@googlegroups.com
Hey everyone, I've loaded a (very) small data set from s3 using the
HadoopDruidIndexer and am having trouble querying the data. After
running the indexer, I see a segment appear in my segment table in
mysql and the master log registers that it has polled the database and
found 1 segment. However, when I run a simple groupBy query against
the Broker service, I get an empty array as my response. Any ideas on
how I can figure out what's going on / going wrong? I'm not seeing any
output in the broker log, but I haven't tried with debug logging yet.

All information for the query I'm running is here:
https://gist.github.com/3e4547d73efc09a045f3

Thanks for the help!

- Trotter

Eric Tschetter

unread,
Oct 29, 2012, 10:05:53 AM10/29/12
to druid-de...@googlegroups.com
Trotter,

So, you loaded the segment and you got it in S3 and the DB so the
master notices it. Do you have any compute nodes (ServerMain) running
and does the Master have any logs about it asking a compute node to
load the segment? The expected behavior would be

1) Segment put in S3, segments table updated
2) Master notices segment
3) Master assigns segment to compute node (via ZK, under "loadQueuePath")
4) Compute node loads segment, announces segment available in ZK
("servedSegmentsPath")
5) Broker notices that compute node has segment (should be an INFO
line in the logs)
6) Query against broker should return data

It sounds like you've verified up through (2), can you verify 3, 4,
and 5 in the logs?

--Eric
> --
> You received this message because you are subscribed to the Google Groups "Druid Development" group.
> To post to this group, send email to druid-de...@googlegroups.com.
> To unsubscribe from this group, send email to druid-developm...@googlegroups.com.
> For more options, visit https://groups.google.com/groups/opt_out.
>
>

Trotter Cashion

unread,
Oct 29, 2012, 2:28:10 PM10/29/12
to druid-de...@googlegroups.com
Hey Eric,

I don't see 3, 4, or 5 in the logs. The master notices the segment,
but it doesn't log anything about assigning a segment. A portion of
the master log from startup is here:
https://gist.github.com/3e4547d73efc09a045f3#file_master.log. Maybe
the log message is there and I'm just not noticing it?

I've also got a zkCli watch on /druid/loadQueuePath/localhost and
/druid/servedSegmentsPath/localhost, and I don't see any data changes
there.

Also, while we're on the topic of logs, what is the proper way to turn
on debug logging? I just went into the code and set the logger,
because I was having a hard time getting a log4j.properties setting to
take.

Thanks for the help.

- Trotter

Eric Tschetter

unread,
Oct 29, 2012, 3:10:12 PM10/29/12
to druid-de...@googlegroups.com
Trotter,

Hrm, what is this property set to?

druid.master.startDelay

The demo configs have it set ti "PT600s" which means 10 minutes. This
delay is a delay that stops the master from doing anything for some
period of time post start up. It's in there as a hack to work around
the way that it currently loads stuff off ZK (basically, it's intended
to give the master enough time to get a full view of the segments
available on all of the nodes). Given that you have a small number of
segments right now push that down to like PT15s or something and then
wait at least 15 seconds after it says "I really am the master" and
see if you get any new log lines.

The proper way to turn on debug logging *should* be to have a
log4j.properties/log4j.xml in the base of the classpath. It's
possible that we've screwed up and included one of those in a
resources directory in one of the modules/dependencies such that it's
taking effect before what you are including in the classpath. That'd
be a bug, however :).

--Eric

Trotter Cashion

unread,
Oct 29, 2012, 3:31:06 PM10/29/12
to druid-de...@googlegroups.com
Ahh... it looks like it may not know about my compute nodes?


2012-10-29 12:24:21,934 DEBUG [Master-Exec--0]
com.metamx.common.concurrent.ScheduledExecutors - Running
com.metamx.druid.master.DruidMaster$3@3529c445 (delay PT60S)
2012-10-29 12:24:21,949 DEBUG [Master-Exec--0]
com.metamx.druid.master.DruidMasterSegmentInfoLoader - Available
DataSegments
2012-10-29 12:24:21,949 DEBUG [Master-Exec--0]
com.metamx.druid.master.DruidMasterSegmentInfoLoader -
DataSegment{size=1717, shardSpec=NoneShardSpec, metrics=[event_count,
revenue, clicks], dimensions=[clicks, state],
version='2012-10-29T10:12:27.847-07:00', loadSpec={type=s3_zip,
bucket=trotter-test,
key=output/2012-10-20T00:00:00.000-07:00_2012-10-21T00:00:00.000-07:00/2012-10-29T10:12:27.847-07:00/0/index.zip},
interval=2012-10-20T00:00:00.000/2012-10-21T00:00:00.000,
dataSource='trotter-test'}
2012-10-29 12:24:21,949 DEBUG [Master-Exec--0]
com.metamx.druid.master.DruidMasterSegmentInfoLoader -
DataSegment{size=1717, shardSpec=NoneShardSpec, metrics=[event_count,
revenue, clicks], dimensions=[clicks, state],
version='2012-10-28T18:46:00.548-07:00', loadSpec={type=s3_zip,
bucket=trotter-test,
key=output/2012-10-20T00:00:00.000-07:00_2012-10-21T00:00:00.000-07:00/2012-10-28T18:46:00.548-07:00/0/index.zip},
interval=2012-10-20T00:00:00.000/2012-10-21T00:00:00.000,
dataSource='trotter-test'}
2012-10-29 12:24:21,949 DEBUG [Master-Exec--0]
com.metamx.druid.master.DruidMaster - Servers
2012-10-29 12:24:21,949 DEBUG [Master-Exec--0]
com.metamx.druid.master.DruidMaster - DruidServer{name='localhost',
host='localhost', maxSize=300000000000, type=}
2012-10-29 12:24:21,949 DEBUG [Master-Exec--0]
com.metamx.druid.master.DruidMaster - -- DataSources
2012-10-29 12:24:21,950 WARN [Master-Exec--0]
com.metamx.druid.master.DruidMasterAssigner - Uh... I have no servers.
Not assigning anything...
2012-10-29 12:24:21,950 WARN [Master-Exec--0]
com.metamx.druid.master.DruidMasterReplicator - Number of servers[0]:
nothing to replicate
2012-10-29 12:24:21,951 INFO [Master-Exec--0]
com.metamx.druid.master.DruidMasterBalancer - No unique values found
for highest and lowest percent used servers: nothing to balance
2012-10-29 12:24:21,951 INFO [Master-Exec--0]
com.metamx.druid.master.DruidMasterLogger - Dropped 0 segments from 0
servers
2012-10-29 12:24:21,951 INFO [Master-Exec--0]
com.metamx.druid.master.DruidMasterLogger - Deleted 0 segments
2012-10-29 12:24:21,951 INFO [Master-Exec--0]
com.metamx.druid.master.DruidMasterLogger - Created 0 replicants
2012-10-29 12:24:21,951 INFO [Master-Exec--0]
com.metamx.druid.master.DruidMasterLogger - Removed 0 replicants

- Trotter

On Mon, Oct 29, 2012 at 12:10 PM, Eric Tschetter

Eric Tschetter

unread,
Oct 29, 2012, 3:56:37 PM10/29/12
to druid-de...@googlegroups.com
Yup, that's what the log says. Make sure that your compute has the
same "announcementsPath", "loadQueuePath" and "servedSegmentsPath" as
the master. In this case, it looks like the "announcementsPath" might
be off (or the compute node isn't publishing to it).

--Eric

Eric Tschetter

unread,
Oct 29, 2012, 3:59:39 PM10/29/12
to druid-de...@googlegroups.com
Hrm, it also strikes me to make sure that your "host" is unique. If
you are running multiple processes from the same server, you need to
actually include the port in the host like

druid.host=localhost:my_port

--Eric

Trotter Cashion

unread,
Oct 29, 2012, 5:19:54 PM10/29/12
to druid-de...@googlegroups.com
Perfect, setting the host appropriately made the master know that I
had a compute and then I had to set the compute's type to "historical"
for the master to actually consider it suitable for use. I now see my
segments being processed by the compute and put into zookeeper under
/druid/servedSegmentsPath. The broker log also show's that it's aware
of the segments.

Unfortunately, my query still returns nothing :-(. See anything wrong
with this? https://gist.github.com/3e4547d73efc09a045f3#file_sample_query.body

- Trotter

On Mon, Oct 29, 2012 at 12:59 PM, Eric Tschetter

Eric Tschetter

unread,
Oct 29, 2012, 6:57:29 PM10/29/12
to druid-de...@googlegroups.com
Hrm,

So, the aggregation:

{ "type": "doubleSum", "fieldName": "column_4", "name": "revenue" }

Won't actually produce anything. From the looks of it, you've produced

"dimensions":"clicks,state",
"metrics":"event_count,revenue,clicks"

From running the Hadoop indexer, but the aggregator is going to look
for "column_4" and get nothing. This should just result in a value of
0 though, instead of no data. Your count aggregator should definitely
be producing something as well.

Just to test and verify that something is getting returned, try this query:

{
"type": "timeBoundary",
"dataSource": "trotter-test"
}

That should give you the min and max timestamps for your dataset. If
that still doesn't work from the broker node, try it against the
compute node directly (both broker and compute expose the exact same
query API, so you can just curl the same thing to both and you should
get the same results)

--Eric

Trotter Cashion

unread,
Oct 30, 2012, 12:54:04 AM10/30/12
to druid-de...@googlegroups.com
Ok, so the timeBoundary query worked perfectly. As for the actual
query I'm running, you're saying I should change the "fieldName" to
"revenue" instead of "column_4"? I tried removing that aggregation all
together, but the api was still returning only an empty array "[]" to
me.

- Trotter

Eric Tschetter

unread,
Oct 30, 2012, 11:41:43 AM10/30/12
to druid-de...@googlegroups.com
> Ok, so the timeBoundary query worked perfectly. As for the actual
> query I'm running, you're saying I should change the "fieldName" to
> "revenue" instead of "column_4"? I tried removing that aggregation all
> together, but the api was still returning only an empty array "[]" to
> me.

Yes on the column_4 thing, but that's orthogonal. You don't have any
exceptions in either the compute or the broker's logs when you issue
the query, correct?

Can I get you to send the same query to the compute node and see if
you get results? You should just have to point it to the port that
the compute node is listening on.

--Eric

Alvin Hom

unread,
Oct 31, 2012, 1:29:58 AM10/31/12
to druid-de...@googlegroups.com
Eric,

I am facing similar issue as the original poster.  The master node is not discovering the compute nodes.

I have the compute, master, and broker nodes all running on the same box.  I have changed the runtime.properties set to different ports:

#master
druid.host=127.0.0.1
druid.port=8080

#compute
druid.host=127.0.0.1
druid.port=8090

I am not sure what you mean by setting localhost:my_port.  Can you clarify?  Thanks.

-Alvin

Alvin Hom

unread,
Oct 31, 2012, 1:56:48 AM10/31/12
to druid-de...@googlegroups.com
I am also seeing the following exception on the log,  I have run zkSetup already.

2012-10-31 05:21:13,147 INFO [main] com.metamx.druid.coordination.ZkCoordinator - Starting zkCoordinator for server[DruidServer{name='127.0.0.1', host='127.0.0.1', maxSize=300000000000, type=historical}] with config[com.metamx.druid.coordination.ZkCoordinatorConfig$$EnhancerByCGLIB$$7ab7ddd5@4876db09]
2012-10-31 05:21:13,194 WARN [Coordinator-ZKYP--0] com.metamx.druid.client.ZKPhoneBook - Exception thrown, serviceName[/druid/announcementsPath].
java.io.EOFException: No content to map to Object due to end of input
        at org.codehaus.jackson.map.ObjectMapper._initForReading(ObjectMapper.java:2775)
        at org.codehaus.jackson.map.ObjectMapper._readMapAndClose(ObjectMapper.java:2718)
        at org.codehaus.jackson.map.ObjectMapper.readValue(ObjectMapper.java:1863)
        at com.metamx.druid.client.ZKPhoneBook$InternalPhoneBook$UpdatingRunnable.run(ZKPhoneBook.java:385)
        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
        at java.lang.Thread.run(Thread.java:680)
Exception in thread "Coordinator-ZKYP--0" java.lang.RuntimeException: java.io.EOFException: No content to map to Object due to end of input
        at com.metamx.druid.client.ZKPhoneBook$InternalPhoneBook$UpdatingRunnable.run(ZKPhoneBook.java:401)
        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
        at java.lang.Thread.run(Thread.java:680)
Caused by: java.io.EOFException: No content to map to Object due to end of input
        at org.codehaus.jackson.map.ObjectMapper._initForReading(ObjectMapper.java:2775)
        at org.codehaus.jackson.map.ObjectMapper._readMapAndClose(ObjectMapper.java:2718)
        at org.codehaus.jackson.map.ObjectMapper.readValue(ObjectMapper.java:1863)
        at com.metamx.druid.client.ZKPhoneBook$InternalPhoneBook$UpdatingRunnable.run(ZKPhoneBook.java:385)
        ... 3 more

-Alvin

Trotter Cashion

unread,
Oct 31, 2012, 11:02:52 AM10/31/12
to druid-de...@googlegroups.com
Hey Alvin, you need it to look like this:

#master
druid.host=127.0.0.1:8080
druid.port=8080

#compute
druid.host=127.0.0.1:8090
druid.port=8090

- Trotter
> To view this discussion on the web visit
> https://groups.google.com/d/msg/druid-development/-/HSbvebTI8hMJ.

Eric Tschetter

unread,
Oct 31, 2012, 11:10:46 AM10/31/12
to druid-de...@googlegroups.com
> #master
> druid.host=127.0.0.1
> druid.port=8080
>
> #compute
> druid.host=127.0.0.1
> druid.port=8090

Set it up like

#master
druid.host=127.0.0.1:8080
druid.port=8080

#compute
druid.host=127.0.0.1:8090
druid.port=8090

Basically, the "druid.host" property right now needs to include the port.

--Eric
> To view this discussion on the web visit
> https://groups.google.com/d/msg/druid-development/-/HSbvebTI8hMJ.
>
Reply all
Reply to author
Forward
0 new messages