Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

JSON serialization error #311

Closed
bobrik opened this issue Oct 26, 2014 · 15 comments
Closed

JSON serialization error #311

bobrik opened this issue Oct 26, 2014 · 15 comments

Comments

@bobrik
Copy link

bobrik commented Oct 26, 2014

I have some docs that looks like this:

{"@id":"2014-10-02-client_(select(0)from(select(sleep(4)))v)/*\u0027+(select(0)from(select(sleep(4)))v)+\u0027\"+(select(0)from(select(sleep(4)))v)+\"*/-d6c81da88d482fa636e2cc5b19abb3a6","@key":"client_(select(0)from(select(sleep(4)))v)/*\u0027+(select(0)from(select(sleep(4)))v)+\u0027\"+(select(0)from(select(sleep(4)))v)+\"*/","@timestamp":"2014-10-02","@value":1}
{"@id":"2014-10-02-client_(select(0)from(select(sleep(9)))v)/*\u0027+(select(0)from(select(sleep(9)))v)+\u0027\"+(select(0)from(select(sleep(9)))v)+\"*/-caba33866b9b6c36c0791fd8f622802a","@key":"client_(select(0)from(select(sleep(9)))v)/*\u0027+(select(0)from(select(sleep(9)))v)+\u0027\"+(select(0)from(select(sleep(9)))v)+\"*/","@timestamp":"2014-10-02","@value":1}
{"@id":"2014-10-02-client_(select(0)from(select(sleep(15)))v)/*\u0027+(select(0)from(select(sleep(15)))v)+\u0027\"+(select(0)from(select(sleep(15)))v)+\"*/-a1ae7275a600d2b9b77c1a35cf2a2203","@key":"client_(select(0)from(select(sleep(15)))v)/*\u0027+(select(0)from(select(sleep(15)))v)+\u0027\"+(select(0)from(select(sleep(15)))v)+\"*/","@timestamp":"2014-10-02","@value":1}

They are actually garbage that some malware generated, but that's not the point.

The point is that es-hadoop on top of spark+mesos breaks when I index these docs:

JavaEsSpark.saveJsonToEs(compactedRDD, output + "/" + type, ImmutableMap.of("es.mapping.id", "@id"));

When indexing with curl one by one does not produce any errors:

web245 ~ # curl -X PUT http://web245:9200/wtf/events/hey?pretty -d '{"@id":"2014-10-02-client_(select(0)from(select(sleep(4)))v)/*\u0027+(select(0)from(select(sleep(4)))v)+\u0027"+(select(0)from(select(sleep(4)))v)+\"*/-d6c81da88d482fa636e2cc5b19abb3a6","@key":"client_(select(0)from(select(sleep(4)))v)/*\u0027+(select(0)from(select(sleep(4)))v)+\u0027\"+(select(0)from(select(sleep(4)))v)+\"*/","@timestamp":"2014-10-02","@value":1}'
{
  "_index" : "wtf",
  "_type" : "events",
  "_id" : "hey",
  "_version" : 1,
  "created" : true
}
web245 ~ # curl -X PUT http://web245:9200/wtf/events/hey2?pretty -d '{"@id":"2014-10-02-client_(select(0)from(select(sleep(9)))v)/*\u0027+(select(0)from(select(sleep(9)))v)+\u0027\"+(select(0)from(select(sleep(9)))v)+\"*/-caba33866b9b6c36c0791fd8f622802a","@key":"client_(select(0)from(select(sleep(9)))v)/*\u0027+(select(0)from(select(sleep(9)))v)+\u0027\"+(select(0)from(select(sleep(9)))v)+\"*/","@timestamp":"2014-10-02","@value":1}'
{
  "_index" : "wtf",
  "_type" : "events",
  "_id" : "hey2",
  "_version" : 1,
  "created" : true
}
web245 ~ # curl -X PUT http://web245:9200/wtf/events/hey3?pretty -d '{"@id":"2014-10-02-client_(select(0)from(select(sleep(15)))v)/*\u0027+(select(0)from(select(sleep(15)))v)+\u0027\"+(select(0)from(select(sleep(15)))v)+\"*/-a1ae7275a600d2b9b77c1a35cf2a2203","@key":"client_(select(0)from(select(sleep(15)))v)/*\u0027+(select(0)from(select(sleep(15)))v)+\u0027\"+(select(0)from(select(sleep(15)))v)+\"*/","@timestamp":"2014-10-02","@value":1}'
{
  "_index" : "wtf",
  "_type" : "events",
  "_id" : "hey3",
  "_version" : 1,
  "created" : true
}

There is probably some error in bulk indexing code.

The errors look like this:

14/10/26 14:18:15 WARN scheduler.TaskSetManager: Lost task 3.0 in stage 2.0 (TID 12, web365): org.elasticsearch.hadoop.rest.EsHadoopInvalidRequest: JsonParseException[Unexpected character ('+' (code 43)): was expecting comma to separate OBJECT entries
 at [Source: [B@5b7a8bc; line: 1, column: 112]]; fragment[d":"2014-10-02-clien]
        org.elasticsearch.hadoop.rest.RestClient.execute(RestClient.java:322)
        org.elasticsearch.hadoop.rest.RestClient.execute(RestClient.java:299)
        org.elasticsearch.hadoop.rest.RestClient.bulk(RestClient.java:149)
        org.elasticsearch.hadoop.rest.RestRepository.tryFlush(RestRepository.java:199)
        org.elasticsearch.hadoop.rest.RestRepository.flush(RestRepository.java:223)
        org.elasticsearch.hadoop.rest.RestRepository.close(RestRepository.java:236)
        org.elasticsearch.hadoop.rest.RestService$PartitionWriter.close(RestService.java:124)
        org.elasticsearch.spark.rdd.EsRDDWriter$$anonfun$write$1.apply$mcV$sp(EsRDDWriter.scala:33)
        org.apache.spark.TaskContext$$anon$2.onTaskCompletion(TaskContext.scala:99)
        org.apache.spark.TaskContext$$anonfun$markTaskCompleted$1.apply(TaskContext.scala:107)
        org.apache.spark.TaskContext$$anonfun$markTaskCompleted$1.apply(TaskContext.scala:107)
        scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
        scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
        org.apache.spark.TaskContext.markTaskCompleted(TaskContext.scala:107)
        org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:64)
        org.apache.spark.scheduler.Task.run(Task.scala:54)
        org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:177)
        java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        java.lang.Thread.run(Thread.java:745)
14/10/26 14:18:16 INFO scheduler.TaskSetManager: Finished task 4.0 in stage 2.0 (TID 13) in 680 ms on web345 (3/5)
14/10/26 14:18:21 INFO scheduler.TaskSetManager: Starting task 3.1 in stage 2.0 (TID 14, web562, ANY, 997 bytes)
14/10/26 14:18:21 INFO scheduler.TaskSetManager: Starting task 0.0 in stage 2.0 (TID 15, web169, ANY, 997 bytes)
14/10/26 14:18:29 INFO storage.BlockManagerMasterActor: Registering block manager web562:40494 with 1060.3 MB RAM
14/10/26 14:18:31 INFO storage.BlockManagerInfo: Added broadcast_2_piece0 in memory on web562:40494 (size: 2.4 KB, free: 1060.3 MB)
14/10/26 14:18:32 INFO storage.BlockManagerMasterActor: Registering block manager web169:38585 with 1060.3 MB RAM
14/10/26 14:18:32 WARN scheduler.TaskSetManager: Lost task 3.1 in stage 2.0 (TID 14, web562): org.elasticsearch.hadoop.rest.EsHadoopInvalidRequest: JsonParseException[Unexpected character ('+' (code 43)): was expecting comma to separate OBJECT entries
 at [Source: [B@ee220ae; line: 1, column: 112]]; fragment[d":"2014-10-02-clien]
        org.elasticsearch.hadoop.rest.RestClient.execute(RestClient.java:322)
        org.elasticsearch.hadoop.rest.RestClient.execute(RestClient.java:299)
        org.elasticsearch.hadoop.rest.RestClient.bulk(RestClient.java:149)
        org.elasticsearch.hadoop.rest.RestRepository.tryFlush(RestRepository.java:199)
        org.elasticsearch.hadoop.rest.RestRepository.flush(RestRepository.java:223)
        org.elasticsearch.hadoop.rest.RestRepository.close(RestRepository.java:236)
        org.elasticsearch.hadoop.rest.RestService$PartitionWriter.close(RestService.java:124)
        org.elasticsearch.spark.rdd.EsRDDWriter$$anonfun$write$1.apply$mcV$sp(EsRDDWriter.scala:33)
        org.apache.spark.TaskContext$$anon$2.onTaskCompletion(TaskContext.scala:99)
        org.apache.spark.TaskContext$$anonfun$markTaskCompleted$1.apply(TaskContext.scala:107)
        org.apache.spark.TaskContext$$anonfun$markTaskCompleted$1.apply(TaskContext.scala:107)
        scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
        scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
        org.apache.spark.TaskContext.markTaskCompleted(TaskContext.scala:107)
        org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:64)
        org.apache.spark.scheduler.Task.run(Task.scala:54)
        org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:177)
        java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        java.lang.Thread.run(Thread.java:745)
14/10/26 14:18:32 INFO scheduler.TaskSetManager: Starting task 3.2 in stage 2.0 (TID 16, web305, ANY, 997 bytes)
14/10/26 14:18:32 INFO storage.BlockManagerInfo: Added broadcast_2_piece0 in memory on web305:46567 (size: 2.4 KB, free: 1060.3 MB)
14/10/26 14:18:32 WARN scheduler.TaskSetManager: Lost task 3.2 in stage 2.0 (TID 16, web305): org.elasticsearch.hadoop.rest.EsHadoopInvalidRequest: JsonParseException[Unexpected character ('+' (code 43)): was expecting comma to separate OBJECT entries
 at [Source: [B@33438f64; line: 1, column: 112]]; fragment[d":"2014-10-02-clien]
        org.elasticsearch.hadoop.rest.RestClient.execute(RestClient.java:322)
        org.elasticsearch.hadoop.rest.RestClient.execute(RestClient.java:299)
        org.elasticsearch.hadoop.rest.RestClient.bulk(RestClient.java:149)
        org.elasticsearch.hadoop.rest.RestRepository.tryFlush(RestRepository.java:199)
        org.elasticsearch.hadoop.rest.RestRepository.flush(RestRepository.java:223)
        org.elasticsearch.hadoop.rest.RestRepository.close(RestRepository.java:236)
        org.elasticsearch.hadoop.rest.RestService$PartitionWriter.close(RestService.java:124)
        org.elasticsearch.spark.rdd.EsRDDWriter$$anonfun$write$1.apply$mcV$sp(EsRDDWriter.scala:33)
        org.apache.spark.TaskContext$$anon$2.onTaskCompletion(TaskContext.scala:99)
        org.apache.spark.TaskContext$$anonfun$markTaskCompleted$1.apply(TaskContext.scala:107)
        org.apache.spark.TaskContext$$anonfun$markTaskCompleted$1.apply(TaskContext.scala:107)
        scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
        scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
        org.apache.spark.TaskContext.markTaskCompleted(TaskContext.scala:107)
        org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:64)
        org.apache.spark.scheduler.Task.run(Task.scala:54)
        org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:177)
        java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        java.lang.Thread.run(Thread.java:745)
14/10/26 14:18:32 INFO scheduler.TaskSetManager: Starting task 3.3 in stage 2.0 (TID 17, web365, PROCESS_LOCAL, 997 bytes)
14/10/26 14:18:32 WARN scheduler.TaskSetManager: Lost task 3.3 in stage 2.0 (TID 17, web365): org.elasticsearch.hadoop.rest.EsHadoopInvalidRequest: JsonParseException[Unexpected character ('+' (code 43)): was expecting comma to separate OBJECT entries
 at [Source: [B@356ce39f; line: 1, column: 112]]; fragment[d":"2014-10-02-clien]
        org.elasticsearch.hadoop.rest.RestClient.execute(RestClient.java:322)
        org.elasticsearch.hadoop.rest.RestClient.execute(RestClient.java:299)
        org.elasticsearch.hadoop.rest.RestClient.bulk(RestClient.java:149)
        org.elasticsearch.hadoop.rest.RestRepository.tryFlush(RestRepository.java:199)
        org.elasticsearch.hadoop.rest.RestRepository.flush(RestRepository.java:223)
        org.elasticsearch.hadoop.rest.RestRepository.close(RestRepository.java:236)
        org.elasticsearch.hadoop.rest.RestService$PartitionWriter.close(RestService.java:124)
        org.elasticsearch.spark.rdd.EsRDDWriter$$anonfun$write$1.apply$mcV$sp(EsRDDWriter.scala:33)
        org.apache.spark.TaskContext$$anon$2.onTaskCompletion(TaskContext.scala:99)
        org.apache.spark.TaskContext$$anonfun$markTaskCompleted$1.apply(TaskContext.scala:107)
        org.apache.spark.TaskContext$$anonfun$markTaskCompleted$1.apply(TaskContext.scala:107)
        scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
        scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
        org.apache.spark.TaskContext.markTaskCompleted(TaskContext.scala:107)
        org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:64)
        org.apache.spark.scheduler.Task.run(Task.scala:54)
        org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:177)
        java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        java.lang.Thread.run(Thread.java:745)
14/10/26 14:18:32 ERROR scheduler.TaskSetManager: Task 3 in stage 2.0 failed 4 times; aborting job
14/10/26 14:18:32 INFO scheduler.TaskSchedulerImpl: Cancelling stage 2
14/10/26 14:18:32 INFO scheduler.DAGScheduler: Could not cancel tasks for stage 2
java.lang.UnsupportedOperationException
    at org.apache.spark.scheduler.SchedulerBackend$class.killTask(SchedulerBackend.scala:32)
    at org.apache.spark.scheduler.cluster.mesos.MesosSchedulerBackend.killTask(MesosSchedulerBackend.scala:41)
    at org.apache.spark.scheduler.TaskSchedulerImpl$$anonfun$cancelTasks$3$$anonfun$apply$1.apply$mcVJ$sp(TaskSchedulerImpl.scala:194)
    at org.apache.spark.scheduler.TaskSchedulerImpl$$anonfun$cancelTasks$3$$anonfun$apply$1.apply(TaskSchedulerImpl.scala:192)
    at org.apache.spark.scheduler.TaskSchedulerImpl$$anonfun$cancelTasks$3$$anonfun$apply$1.apply(TaskSchedulerImpl.scala:192)
    at scala.collection.mutable.HashSet.foreach(HashSet.scala:79)
    at org.apache.spark.scheduler.TaskSchedulerImpl$$anonfun$cancelTasks$3.apply(TaskSchedulerImpl.scala:192)
    at org.apache.spark.scheduler.TaskSchedulerImpl$$anonfun$cancelTasks$3.apply(TaskSchedulerImpl.scala:185)
    at scala.Option.foreach(Option.scala:236)
    at org.apache.spark.scheduler.TaskSchedulerImpl.cancelTasks(TaskSchedulerImpl.scala:185)
    at org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages$1.apply$mcVI$sp(DAGScheduler.scala:1211)
    at org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages$1.apply(DAGScheduler.scala:1197)
    at org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages$1.apply(DAGScheduler.scala:1197)
    at scala.collection.mutable.HashSet.foreach(HashSet.scala:79)
    at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1197)
    at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1174)
    at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1173)
    at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
    at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
    at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1173)
    at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:688)
    at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:688)
    at scala.Option.foreach(Option.scala:236)
    at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:688)
    at org.apache.spark.scheduler.DAGSchedulerEventProcessActor$$anonfun$receive$2.applyOrElse(DAGScheduler.scala:1391)
    at akka.actor.ActorCell.receiveMessage(ActorCell.scala:498)
    at akka.actor.ActorCell.invoke(ActorCell.scala:456)
    at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:237)
    at akka.dispatch.Mailbox.run(Mailbox.scala:219)
    at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:386)
    at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
    at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
    at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
    at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
14/10/26 14:18:34 INFO storage.BlockManagerInfo: Added broadcast_2_piece0 in memory on web169:38585 (size: 2.4 KB, free: 1060.3 MB)
14/10/26 14:18:34 WARN scheduler.TaskSetManager: Lost task 0.0 in stage 2.0 (TID 15, web169): org.elasticsearch.hadoop.rest.EsHadoopInvalidRequest: JsonParseException[Unexpected character ('+' (code 43)): was expecting comma to separate OBJECT entries
 at [Source: [B@350555c9; line: 1, column: 112]]; fragment[eep(4)))v)+'"+(selec]
        org.elasticsearch.hadoop.rest.RestClient.execute(RestClient.java:322)
        org.elasticsearch.hadoop.rest.RestClient.execute(RestClient.java:299)
        org.elasticsearch.hadoop.rest.RestClient.bulk(RestClient.java:149)
        org.elasticsearch.hadoop.rest.RestRepository.tryFlush(RestRepository.java:199)
        org.elasticsearch.hadoop.rest.RestRepository.flush(RestRepository.java:223)
        org.elasticsearch.hadoop.rest.RestRepository.close(RestRepository.java:236)
        org.elasticsearch.hadoop.rest.RestService$PartitionWriter.close(RestService.java:124)
        org.elasticsearch.spark.rdd.EsRDDWriter$$anonfun$write$1.apply$mcV$sp(EsRDDWriter.scala:33)
        org.apache.spark.TaskContext$$anon$2.onTaskCompletion(TaskContext.scala:99)
        org.apache.spark.TaskContext$$anonfun$markTaskCompleted$1.apply(TaskContext.scala:107)
        org.apache.spark.TaskContext$$anonfun$markTaskCompleted$1.apply(TaskContext.scala:107)
        scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
        scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
        org.apache.spark.TaskContext.markTaskCompleted(TaskContext.scala:107)
        org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:64)
        org.apache.spark.scheduler.Task.run(Task.scala:54)
        org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:177)
        java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        java.lang.Thread.run(Thread.java:745)
14/10/26 14:18:34 INFO scheduler.TaskSchedulerImpl: Removed TaskSet 2.0, whose tasks have all completed, from pool

and spark hangs after that (it should exit, I suppose).

This also reminds me about #217, because it is very hard to tell what is wrong looking at the error messages.

I'm on spark 1.1.0 and es-hadoop 2.1.Beta2.

@bobrik
Copy link
Author

bobrik commented Oct 26, 2014

Also this happens, although I cannot find records that cause that:

14/10/26 15:13:05 WARN scheduler.TaskSetManager: Lost task 2.3 in stage 0.0 (TID 12, web391): org.elasticsearch.hadoop.rest.EsHadoopInvalidRequest: JsonParseException[Illegal unquoted character ((CTRL-CHAR, code 31)): has to be escaped using backslash to be included in string value
 at [Source: [B@6700a147; line: 1, column: 57]]; fragment[step_finished-711bf1]
        org.elasticsearch.hadoop.rest.RestClient.execute(RestClient.java:322)
        org.elasticsearch.hadoop.rest.RestClient.execute(RestClient.java:299)
        org.elasticsearch.hadoop.rest.RestClient.bulk(RestClient.java:149)
        org.elasticsearch.hadoop.rest.RestRepository.tryFlush(RestRepository.java:199)
        org.elasticsearch.hadoop.rest.RestRepository.flush(RestRepository.java:223)
        org.elasticsearch.hadoop.rest.RestRepository.doWriteToIndex(RestRepository.java:175)
        org.elasticsearch.hadoop.rest.RestRepository.writeToIndex(RestRepository.java:138)
        org.elasticsearch.spark.rdd.EsRDDWriter.write(EsRDDWriter.scala:36)
        org.elasticsearch.spark.rdd.EsSpark$$anonfun$saveToEs$1.apply(EsSpark.scala:34)
        org.elasticsearch.spark.rdd.EsSpark$$anonfun$saveToEs$1.apply(EsSpark.scala:34)
        org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:62)
        org.apache.spark.scheduler.Task.run(Task.scala:54)
        org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:177)
        java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        java.lang.Thread.run(Thread.java:745)

Btw I use Gson to serialize records, if that matters.

@bobrik
Copy link
Author

bobrik commented Oct 26, 2014

After some investigation it appears that without ImmutableMap.of("es.mapping.id", "@id") problem disappears. Same happens with JavaEsSpark.saveToEs(): works without setting id, doesn't work with mapping.

Is it possible to add an ability to write JavaPairRDD where key is id for elasticsearch? My keys are pretty long, I'd like to avoid including them into docs itself.

@sdubey
Copy link

sdubey commented Nov 10, 2014

I have the similar problem while using pig and elasticsearch. Indexing works just fine till 45 m records. But somehow it breaks after that. I tried without using the 'es.mapping.id=key' as suggested and the execution went just fine.

Exception is -
java.io.IOException: org.elasticsearch.hadoop.rest.EsHadoopInvalidRequest: Unexpected end-of-input in VALUE_STRING at [Source: [B@13477747; line: 1, column: 1624] at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapReduce$Reduce.runPipeline(PigGenericMapReduce.java:470) at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapReduce$Reduce.processOnePackageOutput(PigGenericMapReduce.java:433) at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapReduce$Reduce.reduce(PigGenericMapReduce.java:413) at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapReduce$Reduce.reduce(PigGenericMapReduce.java:257) at org.apache.hadoop.mapreduce.Reducer.run(Reducer.java:176) at org.apache.hadoop.mapred.ReduceTask.runNewReducer(ReduceTask.java:649) at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:417) 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:1121) at org.apache.hadoop.mapred.Child.main(Child.java:249) Caused by: org.elasticsearch.hadoop.rest.EsHadoopInvalidRequest: Unexpected end-of-input in VALUE_STRING at [Source: [B@13477747; line: 1, column: 1624] at org.elasticsearch.hadoop.rest.RestClient.execute(RestClient.java:300) at org.elasticsearch.hadoop.rest.RestClient.execute(RestClient.java:278) at org.elasticsearch.hadoop.rest.RestClient.bulk(RestClient.java:139) at org.elasticsearch.hadoop.rest.RestRepository.sendBatch(RestRepository.java:170) at org.elasticsearch.hadoop.rest.RestRepository.doWriteToIndex(RestRepository.java:160) at org.elasticsearch.hadoop.rest.RestRepository.writeToIndex(RestRepository.java:130) at org.elasticsearch.hadoop.mr.EsOutputFormat$EsRecordWriter.write(EsOutputFormat.java:161) at org.elasticsearch.hadoop.pig.EsStorage.putNext(EsStorage.java:196) at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigOutputFormat$PigRecordWriter.write(PigOutputFormat.java:139) at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigOutputFormat$PigRecordWriter.write(PigOutputFormat.java:98) at org.apache.hadoop.mapred.ReduceTask$NewTrackingRecordWriter.write(ReduceTask.java:587) at org.apache.hadoop.mapreduce.TaskInputOutputContext.write(TaskInputOutputContext.java:80) at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapReduce$Reduce.runPipeline(PigGenericMapReduce.java:468) ... 11 more ,Task attempt_201410091750_0059_r_000140_0 failed to report status for 600 seconds. Killing!

@sdubey
Copy link

sdubey commented Nov 10, 2014

I need to have 'es.mapping.id=key' as part of my hadoop job, so it is still an open issue for me.

@sdubey
Copy link

sdubey commented Dec 10, 2014

Just to add more to this, For sometime I thought that my dataset (Which is pretty huge)might have some wrong characters or something to not qualify as a _id, but I bulk indexed the data from a standalone java application, it works fine without complaining.

costin added a commit that referenced this issue Dec 13, 2014
When constructing documents with fields from JSON data, properly
escape chars (according to the JSON spec) to avoid invalid documents

Relates #311
@costin
Copy link
Member

costin commented Dec 13, 2014

I've finally got around this issue which was caused by not escaping characters according to the JSON spec.
@bobrik @sdubey I've pushed a fix to master and published a 2.1.0.BUILD-SNAPSHOT that contains the fix.
Can you please test it out and let me know if it works for you?
Thanks!

@bobbych
Copy link

bobbych commented Dec 21, 2014

we have been having same issues with hive 0.13. Jobs fail with similar errors mentioned in this thread. Its either illegal control character (which are not present in our file) or unexpected control character. I compiled jar from master after fix.

Caused by: org.elasticsearch.hadoop.rest.EsHadoopInvalidRequest: JsonParseException[Illegal character ((CTRL-CHAR, code 1)): only regular white space (\r, \n, \t) is allowed between tokens
 at [Source: [B@43c8b261; line: 1, column: 2]]; fragment[{"index]
    at org.elasticsearch.hadoop.rest.RestClient.execute(RestClient.java:322)
    at org.elasticsearch.hadoop.rest.RestClient.execute(RestClient.java:299)
    at org.elasticsearch.hadoop.rest.RestClient.bulk(RestClient.java:149)
    at org.elasticsearch.hadoop.rest.RestRepository.tryFlush(RestRepository.java:199)
    at org.elasticsearch.hadoop.rest.RestRepository.flush(RestRepository.java:223)
    at org.elasticsearch.hadoop.rest.RestRepository.doWriteToIndex(RestRepository.java:175)
    at org.elasticsearch.hadoop.rest.RestRepository.writeProcessedToIndex(RestRepository.java:154)
    at org.elasticsearch.hadoop.hive.EsHiveOutputFormat$EsHiveRecordWriter.write(EsHiveOutputFormat.java:63)
    at org.apache.hadoop.hive.ql.exec.FileSinkOperator.processOp(FileSinkOperator.java:621)
    at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:793)
    at org.apache.hadoop.hive.ql.exec.SelectOperator.processOp(SelectOperator.java:87)
    at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:793)
    at org.apache.hadoop.hive.ql.exec.FilterOperator.processOp(FilterOperator.java:137)
    at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:793)
    at org.apache.hadoop.hive.ql.exec.TableScanOperator.processOp(TableScanOperator.java:92)
    at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:793)
    at org.apache.hadoop.hive.ql.exec.MapOperator.process(MapOperator.java:540)
    ... 9 more

@costin
Copy link
Member

costin commented Dec 22, 2014

@bobbych do you have a sample that I could use to reproduce the error? If you could post it on a gist or dropbox somewhere (potentially zipped to make sure the special chars are not replaced) that would help a lot.

Thanks!

@costin
Copy link
Member

costin commented Jan 12, 2015

@bobrik @sdubey Hi, did you manage to see if the changes in master fix your problems? Thanks!

@bobrik
Copy link
Author

bobrik commented Jan 12, 2015

@costin I managed to avoid the problem by hashing @id.

@costin
Copy link
Member

costin commented Jan 12, 2015

Sorry to hear that (ideally it shouldn't have been a problem in the first place).
If you ever do happen to try out the fix, let me know.

P.S. A hashing function is a good idea anyway to 'slim' the id field however pay attention to its collision

@bobbych
Copy link

bobbych commented Jan 12, 2015

@costin I think these strange CRTL characters are being added at network level. When use es client directly from hive job fails , but when i setup proxy (ngnix) and route traffic via proxy job ran successfully. not sure 100% what is going on but i think issue is related to network esp aws enhanced networking. Our Hadoop cluster has enhanced network while ES doesn't.

@sdubey
Copy link

sdubey commented Jan 14, 2015

@costin I tried the version 2.1.0.BUILD-SNAPSHOT you mentioned on Dec 13. Unfortunately it did not work for me. If there has been a change after that, I will try again this week.

@costin
Copy link
Member

costin commented Jan 14, 2015

@sdubey I'm not sure how you use the version but tools like Maven or/and Gradle automatically download the latest version so if you use those, just running an update should be enough (you should see in the console the new jar being downloaded and its suffix which typically is a timestamp. Additionally in the console when you use es-hadoop you get a git hash that is useful to identify the exact source compiled).
If you still experience issues, please provide some sample input that triggers the error plus the associated exception and some information about the environment. The more, the better.

Thanks!

@costin
Copy link
Member

costin commented Apr 28, 2015

Though it is not clear this issue has been fixed, as there hasn't been any update I'm closing it down. If the problem persists, please try the latest beta (currently 4) and if it is not fixed, create a new issue while potentially linking to this one.

Thanks!

@costin costin closed this as completed Apr 28, 2015
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

No branches or pull requests

4 participants