HDFS Deep Storage and Druid 0.6.0

326 views
Skip to first unread message

Christian Schulze

unread,
Oct 25, 2013, 8:10:43 AM10/25/13
to druid-de...@googlegroups.com
Hi folks,

I have another problem: I am trying try to use HDFS for Deep Storge for indexing and can't get it to work with Druid 0.6.0. Local storage works fine. I always get the following Exception during reduce:

java.lang.IllegalArgumentException: Wrong FS: file://foobar/dummydata_2_10_10/dummydata_2_10_10/20130102T000000.000+0100_20130103T000000.000+0100/2013-10-25T13_12_33.426+02_00/0, expected: file:///
	at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:390)
	at org.apache.hadoop.fs.RawLocalFileSystem.pathToFile(RawLocalFileSystem.java:55)
	at org.apache.hadoop.fs.RawLocalFileSystem.mkdirs(RawLocalFileSystem.java:340)
	at org.apache.hadoop.fs.ChecksumFileSystem.mkdirs(ChecksumFileSystem.java:492)
	at io.druid.indexer.IndexGeneratorJob$IndexGeneratorReducer.serializeOutIndex(IndexGeneratorJob.java:394)
	at io.druid.indexer.IndexGeneratorJob$IndexGeneratorReducer.reduce(IndexGeneratorJob.java:374)
	at io.druid.indexer.IndexGeneratorJob$IndexGeneratorReducer.reduce(IndexGeneratorJob.java:237)
	at org.apache.hadoop.mapreduce.Reducer.run(Reducer.java:177)
	at org.apache.hadoop.mapred.ReduceTask.runNewReducer(ReduceTask.java:649)
	at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:418)
	at org.apache.hadoop.mapred.Child$4.run(Child.java:255)
	at java.security.AccessController.doPrivileged(Native Method)
	at javax.security.auth.Subject.doAs(Subject.java:415)
	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1190)
	at org.apache.hadoop.mapred.Child.main(Child.java:249)

The path is obviously wrong and I think it tries to use the local storage. I set the path in config/overlord/runtime.properties (all other configs are default like in the tutorial):

druid.storage.type=hdfs
druid.storage.storageDirectory=foobar

Using something like "hdfs://hadoop-master:54310/foobar" for storageDirectory produces wrong paths too (something like file://hdfs:/hadoop-master:54310/foobar)

I start the indexing service with this command: java -Xmx2g -Duser.timezone=Europe/Berlin -Dfile.encoding=UTF-8 -classpath lib/*:`hadoop classpath`:config/overlord io.druid.cli.Main server overlord

My indexer configuration:

{
  "config": {
    "rollupSpec": {
      "rollupGranularity": "day",
      "aggs": [
        {
          "name": "event_1",
          "fieldName": "event_1",
          "type": "longSum"
        },
        {
          "name": "event_2",
          "fieldName": "event_2",
          "type": "longSum"
        },
        {
          "name": "event_3",
          "fieldName": "event_3",
          "type": "longSum"
        },
        {
          "name": "event_4",
          "fieldName": "event_4",
          "type": "longSum"
        },
        {
          "name": "event_5",
          "fieldName": "event_5",
          "type": "longSum"
        },
        {
          "name": "event6",
          "fieldName": "event_6",
          "type": "longSum"
        },
        {
          "name": "event7",
          "fieldName": "event_7",
          "type": "longSum"
        },
        {
          "name": "event8",
          "fieldName": "event_8",
          "type": "longSum"
        },
        {
          "name": "event9",
          "fieldName": "event_9",
          "type": "longSum"
        },
        {
          "name": "event10",
          "fieldName": "event_10",
          "type": "longSum"
        },
        {
          "name": "value_1",
          "fieldName": "value_1",
          "type": "doubleSum"
        },
        {
          "name": "value_2",
          "fieldName": "value_2",
          "type": "doubleSum"
        },
        {
          "name": "value_3",
          "fieldName": "value_3",
          "type": "doubleSum"
        },
        {
          "name": "value_4",
          "fieldName": "value_4",
          "type": "doubleSum"
        },
        {
          "name": "value_5",
          "fieldName": "value_5",
          "type": "doubleSum"
        },
        {
          "name": "value_6",
          "fieldName": "value_6",
          "type": "doubleSum"
        },
        {
          "name": "value_7",
          "fieldName": "value_7",
          "type": "doubleSum"
        },
        {
          "name": "value_8",
          "fieldName": "value_8",
          "type": "doubleSum"
        },
        {
          "name": "value_9",
          "fieldName": "value_9",
          "type": "doubleSum"
        },
        {
          "name": "value_10",
          "fieldName": "value_10",
          "type": "doubleSum"
        }
      ]
    },
    "pathSpec": {
      "paths": "csv/dummydata_2_10_10.csv",
      "type": "static"
    },
    "granularitySpec": {
      "gran": "DAY",
      "intervals": [
        "2013-01-01TZ/2013-01-08TZ"
      ],
      "type": "uniform"
    },
    "dataSpec": {
      "dimensions": [
        "dim_1",
        "dim_2"
      ],
      "columns": [
        "ts",
        "dim_1",
        "dim_2",
        "event_1",
        "event_2",
        "event_3",
        "event_4",
        "event_5",
        "event_6",
        "event_7",
        "event_8",
        "event_9",
        "event_10",
        "value_1",
        "value_2",
        "value_3",
        "value_4",
        "value_5",
        "value_6",
        "value_7",
        "value_8",
        "value_9",
        "value_10"
      ],
      "format": "csv"
    },
    "timestampFormat": "posix",
    "timestampColumn": "ts",
    "dataSource": "dummydata_2_10_10",
    "targetPartitionSize" : 5000000
  },
  "type": "index_hadoop"
}


Thank you!

Fangjin Yang

unread,
Oct 25, 2013, 1:59:27 PM10/25/13
to druid-de...@googlegroups.com
Hi Christian,

In Druid 0.6, s3, hdfs and cassandra deep storages are now all self contained modules. The only deep storage that is loaded and used by default is the local one. 

To use HDFS as your deep storage, include the following config:
-Ddruid.extensions.coordinates=["io.druid.extensions:druid-hdfs-storage:0.6.0"]

Let me know if you see problems with that.

Thanks,
FJ


--
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/f71c968e-527d-4c0c-9504-56f66e530e69%40googlegroups.com.
For more options, visit https://groups.google.com/groups/opt_out.

Christian Schulze

unread,
Oct 28, 2013, 12:19:29 PM10/28/13
to druid-de...@googlegroups.com
The module seems to load find but now I get: Can not construct instance of io.druid.storage.hdfs.HdfsDataSegmentPusherConfig ...

I attached the whole stacktrace to this post ...
stacktrace.log

Eric Tschetter

unread,
Oct 29, 2013, 12:29:53 PM10/29/13
to druid-de...@googlegroups.com
Christian,

That looks like an oversight on my part when I was refactoring stuff.  Remove the "abstract" from HdfsDataSegmentPusherConfig and try again.  (I.e. make HdfsDataSegmentPusherConfig a concrete class and it should work).

--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.

Fangjin Yang

unread,
Oct 29, 2013, 5:35:06 PM10/29/13
to druid-de...@googlegroups.com
I just pushed a fix for this into master so that should hopefully fix things.


Reply all
Reply to author
Forward
0 new messages