-
Notifications
You must be signed in to change notification settings - Fork 227
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
Futures timed out after [5 seconds] #84
Comments
@java8964 huh. ok I'll add a change so that timeout can be adjusted, and maybe make the default higher. That's weird, that's when just adjusting metadata and not writing the actual data, normally very quick. |
Is it hardcoded in the code? Is there any runtime configuration I can change? Or can you point to the source code where I can modify the default value? I want to test the read/query cases, so ingestion is not the real concern for us as now. Thanks |
I found out the source code of "5 seconds" and change it to "60 seconds", but now I got a new error which I don't understand, still in the same step as above. Here is the way I started my spark-shell: ~/spark/bin/spark-shell --jars /home/yzhang/FiloDB-0.2/spark/target/scala-2.10/filodb-spark-assembly-0.2.jar,/home/yzhang/spark-csv_2.10-1.4.0.jar,/home/yzhang/commons-csv-1.2.jar --driver-memory 3G --executor-memory 8g --total-executor-cores 18 But in "newDF.write.format("filodb.spark").option("dataset", "poc1").option("row_keys", "Email").option("segment_key","Domain").option("partition_keys","Partition").mode(SaveMode.Overwrite).save()" step, I got the follow errors on the console: 16/03/28 13:41:20 WARN ControlConnection: No rpc_address found for host /10.20.95.209 in 10.20.95.201:9042's peers system table. /10.20.95.209 will be ignored. What I don't understand is that the Class is in the jar file: [yzhang@p2-s1cassana201 ~]$ jar tvf /home/yzhang/FiloDB-0.2/spark/target/scala-2.10/filodb-spark-assembly-0.2.jar | grep "RowSource$IngestionErr" |
Here is the log I can find in the spark executor: 16/03/28 14:01:01 ERROR DatasetCoordinatorActor: Error in reprojection task (poc1/0) Let me explain that what kind of data I generated to test filoDB:
Any hint about why I got the above errors? Thanks |
Yeah, it's hardcoded in FiloRelation.scala, lines 35 and 40, see the "5 On Mon, Mar 28, 2016 at 10:24 AM, Yong Zhang notifications@github.com
|
@java8964 many thanks for detailed errors. Hmmm. I'd like to try to get the binary data from your Emails column. Basically the output of the Cassandra CQL query: SELECT partition, segmentid, chunkid, data from poc1_chunks WHERE columnname = 'Email'; as well as some of the emails that you are trying to ingest. This would help me debug. Thanks! I would also try deleting the keyspace and restarting just to be sure. Also: just a question. Is your Spark data already partitioned by node? What I mean is, currently you need to ensure that data for the same partition key is only on one node; if it is not you need to do a sort() by partition key in Spark before ingestion. thanks, |
Hi, Evan: Thanks for replying. It looks like the query you want me to run will get lots of rows, so I have to add "allow filtering", and only show the first 3 rows below: (not sure if you need more) cqlsh:filodb> SELECT partition, segmentid, chunkid, data from poc1_chunks WHERE columnname = 'Email' allow filtering; partition | segmentid | chunkid | data 0x00000130 | 0x676d61696c2e636f6d | 0 | 0x020100000c00000008000c0004000800080000002c010000040000000a000000280000003c0000005000000064000000780000008c000000a0000000b4000000c8000000dc000000110000003133363135313840676d61696c2e636f6d000000110000003133373038313340676d61696c2e636f6d000000110000003830353134373640676d61696c2e636f6d000000110000003830353235363540676d61696c2e636f6d000000110000003830353430333940676d61696c2e636f6d000000110000003830353435303140676d61696c2e636f6d000000110000003830353436323240676d61696c2e636f6d000000110000003830353531323840676d61696c2e636f6d000000110000003830363436363540676d61696c2e636f6d000000110000003830363738313140676d61696c2e636f6d00000008000800070000000800000000000000 I think I maybe missed the "sort" part. I remember you mentioned somewhere in the document, but can you help me understand why I have to put the same partition key data into one SPARK node? Won't all the data eventually stored in C*? This sounds like an expensive requirement for ingesting data. In your 100k test data, the partition key is round globaleventid of 10,000. I didn't recall there is any sort to make each partition on one spark node? Is that right? I will drop and reset the keyspace, and I will start with small size of data. I guess I am too excited to try 50M after I found out the 100k sample data work great in our environment :-) Before I restart everything, I like to discuss the data modeling I tried to do for this example data, maybe you can tell me if I totally misunderstand how FiloDB works. I could have these billions of emails, each with the some attributes, you can think them like some rates, scores, count etc. Periodically (maybe daily for now), some part of row (maybe small, like 10%) will be updated. That is one reason I don't want to use HDFS. More important, for the read path, I have to support:
Here is the top 10 rows I generated for testing FiloDB: As you can see, I could have more attributes, but just for testing right now, I append only 5 boolean attributes. I will drop and recreate all the keyspaces, and start with small data first. Just want some feedback about my questions above. Thanks Yong |
Hi Yong, thanks for the detailed email. I’ll respond inline later or tomorrow, but if you are coming to Strata San Jose at all, you should come by our FIloDB booth this week, would be easy to chat in person :) -Evan
|
Unfortunately, I am in the east coast. Would love to join with you guys in NY if possible next time. |
Yong, I’ll be in NYC for Scala Days the second week of May or so. Replies inlined:
Yes all the data ends up in C*, but the process of converting rows into efficient columnar chunks works best when the data for one partition ends up in the same node. Strictly speaking a sort is not what is needed, what is needed is a shuffle such that data for each partition ends up together; actually sorting data within each node is not needed. Cassandra does the same thing underneath the hood for regular row data - it will shuffle data across the network to the right coordinator according to the partition. Due to the columnar nature, Cassandra does not do that for this kind of data, so this has to be done for now for users. If we integrated directly into Cassandra, you would need a custom C* installation which would be much harder for folks to try out :-p
|
Evan, thanks for the detail information. It helps a lot for me understanding the internal work of FiloDB. Here are steps what I did again to test loading small data, and based on your feedback:
./filo-cli --command init
So instead of hashID, I generate the partition id directly using "hashCode % 1024", to make everything as simple as possible
val csvDF = sqlContext.read.format("com.databricks.spark.csv").option("header", "true").option("inferSchema", "true").load("/home/yzhang/data/emails_1k.csv") scala> csvDF.printSchema csvDF.sort($"PartitionId").show(1000) I can see all 1k data sorted by PartitionId
But in the spark executor log, I saw the following errors: [ERROR] [03/29/2016 10:44:24.002] [filo-spark-akka.actor.default-dispatcher-6] [akka://filo-spark/user/poc1_0_3] Internal error: release called on 67108864 bytes but task only has 0 [ERROR] [03/29/2016 10:44:24.002] [filo-spark-akka.actor.default-dispatcher-6] [akka://filo-spark/user/poc1_0_13] Internal error: release called on 67108864 bytes but task only has 0 Thanks |
Yong, can you please send me your 1k rows data file and I'll try to repro it locally? Thanks. -Evan
|
Sure. Attached here. |
Hi, Evan: Just FYI, I can reproduce this problem on my local laptop using "spark.filodb.store=in-memory". It is great for debugging with option of "in-memory". Yong |
Thanks, that’s really helpful. Will look at it when I can, most likely not until later this week.
|
Hmm. So I can reproduce this on laptop using in-memory, with the sorting. That’s interesting, will have to look into what this external row sorter thing does. If I remove the sort, then the ingest of all 1k rows works.
|
Yes, I noticed that too. I even test following, partition without sorting, but got the same error: scala> import org.apache.spark.sql.SaveMode scala> val csvDF = sqlContext.read.format("com.databricks.spark.csv").option("header", "true").option("inferSchema", "true").load("/home/yzhang/data/emails_1k.csv") scala> import org.apache.spark.sql.functions.{rowNumber} scala> import org.apache.spark.sql.expressions.Window scala> val w = Window.partitionBy($"PartitionId") scala> val newDF = csvDF.withColumn("rn", rowNumber.over(w)).drop("rn") scala> newDF.write.format("filodb.spark").option("dataset", "poc1").option("row_keys", "Email").option("segment_key", "Domain").option("partition_keys", "PartitionId").mode(SaveMode.Overwrite).save() |
It looks like this problem only exists with the new tungsten shuffle manager. I think 1k will have no problem to ingest, and tomorrow, I will test more data. Will update this ticket if I face other issue for my 50M goal. Thanks Yong |
Great thanks. Will keep investigating this though.
|
Hi, Evan: There are 2 issues I found out so far, but I understand for a version 0.2 release, it is normal:
I will try to understand more about FiloDB, but thanks for such an amazing open source idea. Let me know what I can help for maturing it. Yong |
Yong, Thanks so much for the kind words.
I wrote a blog post about doing fast joins recently. I will investigate the tungsten sort failure still. After Spark 1.6 you can’t turn off tungsten any more, so we do want to get the sort working.
|
Hi, Evan: Date: Thu, 31 Mar 2016 23:42:55 -0700 Yong, Thanks so much for the kind words.
I wrote a blog post about doing fast joins recently. I will investigate the tungsten sort failure still. After Spark 1.6 you can’t turn off tungsten any more, so we do want to get the sort working.
— |
https://groups.google.com/forum/#!forum/filodb-discuss https://groups.google.com/forum/#!forum/filodb-discuss I haven’t advertised it yet. I just added a link to the top of the README. There is also a Gitter channel.
|
BTW @java8964 I discovered, I think, why the error above Alas, changing the ingestion such that the actor can run on the same thread as the task is not easy. I'll investigate a bit. |
Hi, Evan: Thanks for the updating. Meantime, I played with FiloDB for different testing cases I have, and I got more questions :-)
You can see, in this case, I can get consistently about 22 seconds, even I randomly join by 1k, 10k, and 100k emails (I have to use "broadcast", I think it makes sense this case).
So instead of using domain as segment keys, I use the first 3 characters as the segment keys. scala> val poc3 = sqlContext.read.format("filodb.spark").option("dataset", "poc3").load()
|
Hi Yong, I’m very close to a fix for the Tungsten sort thing btw…. but a quick answer inline before i get my kids up….
This is the cassandra table layout (you can inspect the same using cqlsh): CREATE TABLE filodb.gdelt_chunks ( Thus as you can see, both the partition key and the segment ID (key) are part of the primary key. I think your design is correct - prefix 2 chars for email, then segment by domain. If you add domain to your filter/predicates (right now you have to do domain >= DOMAIN1 && domain <= DOMAIN1 unfortunately … for the pushdown to work) then you will be able to look up not only the exact partition but the exact segment within a partition as well.
Have a look at this blog post: TL/DR; the key is aiming for single partition queries (such as your email search above) for all tables, or at least achieving pushdowns. Broadcast doesn’t help too much if the table is big, because of the cost of broadcasting itself. I mean it avoids shuffles but there is also a high cost to broadcast it in the first place (unless you can keep broadcast tables in memory - we can discuss this more)
Hmm. Run ./filo-cli —command analyze —dataset poc3 and compare it to your other data model. You might see that the average size of segment is not that big. Feel free to post it here. That will tell you if it’s a good idea or not (for my NYC Taxi dataset, this was NOT a good idea - although a great stress test).
Thanks, this is very helpful to stabilize & productionize FiloDB. You run different tests than what my client is running.
|
I created a separate issue about Ingesting data (#86), we can talk about problem I faced about ingesting there. For this, I would like to discuss with you about the query optimization. Right now, I understand better about how the FiloDB storing the data. The question now is how pushdowns can be sent to FiloDB reliably in Spark. poc2.filter(poc2("Email") === "vgd999@yahoo.com").explain(true) This is good, as the filter is pushed down to FiloDB. So in cluster, I also get the sub second latency. scala> poc2.filter(poc2("Email") === "vgd999@yahoo.com" && poc2("Domain") >= "yahoo.com" && poc2("Domain") <= "yahoo.com").explain(true) So the above query plan is my goal. But if I have 1k data also in another spark df (not in filoDB), let's say in HDFS or other storage: scala> r_1k.count scala> r_1k.printSchema First, without broadcast: You can see Spark use "SortMergeJoin", which is a VERY expensive way for 100M + 1k join. That is the reason almost killing me when I test this on cluster. More important, you can see the Email (Partition key) is NOT part of the filter any more. In this case, all the C* node will be queried about this 1k data, even though it could be just small part of the nodes really contain the data of these 1k emails. With broadcast, the plan is following: So Spark goes with BroadcastHashJoin. Even it comes with cost of broadcasting, but it is much better than the SortMerge in this case. That is why I originally found out I have to use "broadcast". It is very difficult keeping filtering by partition keys in the join, even though that is the best way for filoDB. I have to come out sql like following: but in Spark, I CANNOT do the subquery with in Yong |
Keep the iterator reader in the original Spark worker thread, use a Queue
Yong, I actually cannot tell by the physical plan alone whether there is a pushdown, can you? Instead I rely on the logs. If you set log to INFO, at least for filodb.spark logger in your log4j.properties, you will see the pushdown like this: 16/04/06 14:02:24 INFO FiloRelation$: Incoming filters = List() (The above is for a query with no predicates. If there are working predicates the Incoming Filters will have filled out, and filters by position will show it being pushed down. So what are you trying to accomplish and can you show me the raw SQL instead of the Spark DF DSL? Ie you want to look up all the emails that contain some attribute in the r_1k table? I guess that would be expensive as there is no predicate pushdown that would really work. You would be better off to do a two step query, like val emails = r1kDF.select(……) The kind of sub-second JOIN would be of this form:
where a is a partition key.
|
* feat(coordinator): singleton ShardMapper recovery infrastructure; remove shardsPerCoord state from AssignmentStrategy * feat(coordinator): make ShardAssignmentStrategy truly stateless/idempotent; add specific unit test * fix(coordinator): fix bug when adding coordinator which already has shards assigned * Silence console logging for multi-JVM tests. Can be easily re-enabled. * feat(coordinator): Incorporate Guardian-based shard/subscription state recovery for ClusterSingleton Changes from Helena's previous implementation: - Instead of the NodeClusterActor telling its local Guardian to subscribe to everything, now each Guardian, on every node, subscribes to everything, thus guaranteeing every node will have the backup - The ShardCoordinatorActor sends an initial ShardMapper snapshot to every subscriber - API/mechanism to recover the current snapshots and subscriptions from the Guardian - unit test to test recovery mechanism * Update doc on sharding and state recovery * Do not start ingestion stream if it is already running (IngestionActor) * CR from Helena Turn on env var LOG_AKKA_TO_CONSOLE if you want detailed test output for multi-jvm and other coordinator tests.
I am trying to test FiloDB 0.2 with a 6 nodes C* (DSE 4.8.5) cluster, running Spark 1.5.2.
The 100k sample data coming with FiloDB works fine, but when I tried to load 50M data of our use case, with a dataset I come out of POC, I got the following error message when trying to do this step:
newDF.write.format("filodb.spark").option("dataset", "poc1").option("row_keys", "Email").option("segment_key","Domain").option("partition_keys","Partition").mode(SaveMode.Overwrite).save()
java.util.concurrent.TimeoutException: Futures timed out after [5 seconds]$iwC$ $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:25)$iwC$ $iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)$iwC$ $iwC$$iwC$$iwC$$iwC$$iwC.(:32)$iwC$ $iwC$$iwC$$iwC$$iwC.(:34)$iwC$ $iwC$$iwC$$iwC.(:36)$iwC$ $iwC$$iwC.(:38)$iwC$ $iwC.(:40)
at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:219)
at scala.concurrent.impl.Promise$DefaultPromise.result(Promise.scala:223)
at scala.concurrent.Await$$anonfun$result$1.apply(package.scala:107)
at scala.concurrent.BlockContext$DefaultBlockContext$.blockOn(BlockContext.scala:53)
at scala.concurrent.Await$.result(package.scala:107)
at filodb.spark.FiloRelation$.parse(FiloRelation.scala:36)
at filodb.spark.FiloRelation$.actorAsk(FiloRelation.scala:42)
at filodb.spark.package$FiloContext.createNewDataset(package.scala:165)
at filodb.spark.package$FiloContext.createOrUpdateDataset(package.scala:216)
at filodb.spark.package$FiloContext.saveAsFilo(package.scala:268)
at filodb.spark.DefaultSource.createRelation(DefaultSource.scala:53)
at org.apache.spark.sql.execution.datasources.ResolvedDataSource$.apply(ResolvedDataSource.scala:170)
at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:146)
at
at
at
at
at
at
at
at $iwC.(:42)
at (:44)
at .(:48)
at .()
at .(:7)
at .()
I am not sure this related to our C* settings, but it looks like a timeout from the FiloDB, but I don't see any configuration in the application.conf can adjust it.
What setting I should change for this?
Thanks
The text was updated successfully, but these errors were encountered: