You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/07/19 09:42:17 UTC

[GitHub] [iceberg] nautilus28 opened a new issue #2838: Bucket partition transformation from timestamp throws error

nautilus28 opened a new issue #2838:
URL: https://github.com/apache/iceberg/issues/2838


   When bucketing timestamp into 16 buckets, Iceberg throws an exception below:
   
   ```
   import org.apache.iceberg.spark.IcebergSpark
   import org.apache.spark.sql.types.DataTypes
   
   IcebergSpark.registerBucketUDF(spark, "bucket", DataTypes.TimestampType, 16)
   ```
   
   
   ```
   [info]   org.apache.spark.SparkException: Job aborted due to stage failure: Task 10 in stage 11.0 failed 1 times, most recent failure: Lost task 10.0 in stage 11.0 (TID 428, 192.168.1.235, executor driver): org.apache.spark.SparkException: Failed to execute user defined function(UDFRegistration$$Lambda$4072/0x0000000801ba6588: (timestamp) => int)
   [info] 	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source)
   [info] 	at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$.$anonfun$prepareShuffleDependency$3(ShuffleExchangeExec.scala:248)
   [info] 	at scala.collection.Iterator$$anon$10.next(Iterator.scala:461)
   [info] 	at scala.collection.Iterator$$anon$10.next(Iterator.scala:461)
   [info] 	at org.apache.spark.util.random.SamplingUtils$.reservoirSampleAndCount(SamplingUtils.scala:42)
   [info] 	at org.apache.spark.RangePartitioner$.$anonfun$sketch$1(Partitioner.scala:306)
   [info] 	at org.apache.spark.RangePartitioner$.$anonfun$sketch$1$adapted(Partitioner.scala:304)
   [info] 	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndex$2(RDD.scala:889)
   [info] 	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndex$2$adapted(RDD.scala:889)
   [info] 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
   [info] 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
   [info] 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
   [info] 	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
   [info] 	at org.apache.spark.scheduler.Task.run(Task.scala:127)
   [info] 	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:446)
   [info] 	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
   [info] 	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:449)
   [info] 	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1130)
   [info] 	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:630)
   [info] 	at java.base/java.lang.Thread.run(Thread.java:832)
   [info] Caused by: java.lang.ClassCastException: class java.sql.Timestamp cannot be cast to class java.lang.Long (java.sql.Timestamp is in module java.sql of loader 'platform'; java.lang.Long is in module java.base of loader 'bootstrap')
   [info] 	at org.apache.iceberg.transforms.Bucket$BucketLong.apply(Bucket.java:177)
   [info] 	at org.apache.spark.sql.UDFRegistration.$anonfun$register$283(UDFRegistration.scala:747)
   [info] 	... 20 more
   [info]
   [info] Driver stacktrace:
   [info]   at org.apache.spark.scheduler.DAGScheduler.failJobAndIndependentStages(DAGScheduler.scala:2059)
   [info]   at org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2(DAGScheduler.scala:2008)
   [info]   at org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2$adapted(DAGScheduler.scala:2007)
   [info]   at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
   [info]   at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
   [info]   at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
   [info]   at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:2007)
   [info]   at org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1(DAGScheduler.scala:973)
   [info]   at org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1$adapted(DAGScheduler.scala:973)
   [info]   at scala.Option.foreach(Option.scala:407)
   [info]   ...
   [info]   Cause: org.apache.spark.SparkException: Failed to execute user defined function(UDFRegistration$$Lambda$4072/0x0000000801ba6588: (timestamp) => int)
   [info]   at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source)
   [info]   at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$.$anonfun$prepareShuffleDependency$3(ShuffleExchangeExec.scala:248)
   [info]   at scala.collection.Iterator$$anon$10.next(Iterator.scala:461)
   [info]   at scala.collection.Iterator$$anon$10.next(Iterator.scala:461)
   [info]   at org.apache.spark.util.random.SamplingUtils$.reservoirSampleAndCount(SamplingUtils.scala:42)
   [info]   at org.apache.spark.RangePartitioner$.$anonfun$sketch$1(Partitioner.scala:306)
   [info]   at org.apache.spark.RangePartitioner$.$anonfun$sketch$1$adapted(Partitioner.scala:304)
   [info]   at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndex$2(RDD.scala:889)
   [info]   at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndex$2$adapted(RDD.scala:889)
   [info]   at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
   [info]   ...
   [info]   Cause: java.lang.ClassCastException: class java.sql.Timestamp cannot be cast to class java.lang.Long (java.sql.Timestamp is in module java.sql of loader 'platform'; java.lang.Long is in module java.base of loader 'bootstrap')
   [info]   at org.apache.iceberg.transforms.Bucket$BucketLong.apply(Bucket.java:177)
   [info]   at org.apache.spark.sql.UDFRegistration.$anonfun$register$283(UDFRegistration.scala:747)
   [info]   at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source)
   [info]   at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$.$anonfun$prepareShuffleDependency$3(ShuffleExchangeExec.scala:248)
   [info]   at scala.collection.Iterator$$anon$10.next(Iterator.scala:461)
   [info]   at scala.collection.Iterator$$anon$10.next(Iterator.scala:461)
   [info]   at org.apache.spark.util.random.SamplingUtils$.reservoirSampleAndCount(SamplingUtils.scala:42)
   [info]   at org.apache.spark.RangePartitioner$.$anonfun$sketch$1(Partitioner.scala:306)
   [info]   at org.apache.spark.RangePartitioner$.$anonfun$sketch$1$adapted(Partitioner.scala:304)
   [info]   at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndex$2(RDD.scala:889)
   ```
   
   Spark 3.0.1
   Iceberg 0.11.1


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer edited a comment on issue #2838: Bucket partition transformation from timestamp throws error

Posted by GitBox <gi...@apache.org>.
RussellSpitzer edited a comment on issue #2838:
URL: https://github.com/apache/iceberg/issues/2838#issuecomment-882620963


   https://github.com/apache/iceberg/blob/0c50b2074cd5dad59bbcb4b4599ec3ae11a34b49/api/src/main/java/org/apache/iceberg/transforms/Bucket.java#L55-L58
   
   Issue is probably that we are using internally a BucketLong, but advertising to Spark that we require a Timestamp Type. Spark gives us a timestamp and we bail because we try to then do a raw cast to apply the function. Can you test with "LongType" instead? We probably will have to patch this though ... There are no tests in TestIcebergSpark for these datatypes.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on issue #2838: Bucket partition transformation from timestamp throws error

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on issue #2838:
URL: https://github.com/apache/iceberg/issues/2838#issuecomment-882620963


   https://github.com/apache/iceberg/blob/0c50b2074cd5dad59bbcb4b4599ec3ae11a34b49/api/src/main/java/org/apache/iceberg/transforms/Bucket.java#L55-L58
   
   Issue is probably that we are using internally a BucketLong, but advertising to Spark that we require a Timestamp Type. Spark gives us a timestamp and we bail because we try to then do a raw cast to apply the function. Can you test with "Long.Type" instead? We probably will have to patch this though ... There are no tests in TestIcebergSpark for these datatypes.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on issue #2838: Bucket partition transformation from timestamp throws error

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on issue #2838:
URL: https://github.com/apache/iceberg/issues/2838#issuecomment-882732878


   Yeah that's too bad, I was hoping Spark would no to make the conversion to be acceptable for the UDF. We can fix this pretty easily though


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nautilus28 commented on issue #2838: Bucket partition transformation from timestamp throws error

Posted by GitBox <gi...@apache.org>.
nautilus28 commented on issue #2838:
URL: https://github.com/apache/iceberg/issues/2838#issuecomment-882643235






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nautilus28 commented on issue #2838: Bucket partition transformation from timestamp throws error

Posted by GitBox <gi...@apache.org>.
nautilus28 commented on issue #2838:
URL: https://github.com/apache/iceberg/issues/2838#issuecomment-882653517


   Same error with UDF of "LongType"
   
   `IcebergSpark.registerBucketUDF(spark, "bucket", DataTypes.LongType, 16)`


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer edited a comment on issue #2838: Bucket partition transformation from timestamp throws error

Posted by GitBox <gi...@apache.org>.
RussellSpitzer edited a comment on issue #2838:
URL: https://github.com/apache/iceberg/issues/2838#issuecomment-882620963


   https://github.com/apache/iceberg/blob/0c50b2074cd5dad59bbcb4b4599ec3ae11a34b49/api/src/main/java/org/apache/iceberg/transforms/Bucket.java#L55-L58
   
   Issue is probably that we are using internally a BucketLong, but advertising to Spark that we require a Timestamp Type. Spark gives us a timestamp and we bail because we try to then do a raw cast to apply the function. Can you test with "LongType" instead? We probably will have to patch this though ... There are no tests in TestIcebergSpark for these datatypes.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nautilus28 commented on issue #2838: Bucket partition transformation from timestamp throws error

Posted by GitBox <gi...@apache.org>.
nautilus28 commented on issue #2838:
URL: https://github.com/apache/iceberg/issues/2838#issuecomment-882643235






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer edited a comment on issue #2838: Bucket partition transformation from timestamp throws error

Posted by GitBox <gi...@apache.org>.
RussellSpitzer edited a comment on issue #2838:
URL: https://github.com/apache/iceberg/issues/2838#issuecomment-882620963


   https://github.com/apache/iceberg/blob/0c50b2074cd5dad59bbcb4b4599ec3ae11a34b49/api/src/main/java/org/apache/iceberg/transforms/Bucket.java#L55-L58
   
   Issue is probably that we are using internally a BucketLong, but advertising to Spark that we require a Timestamp Type. Spark gives us a timestamp and we bail because we try to then do a raw cast to apply the function. Can you test with "LongType" instead? We probably will have to patch this though ... There are no tests in TestIcebergSpark for these datatypes.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on issue #2838: Bucket partition transformation from timestamp throws error

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on issue #2838:
URL: https://github.com/apache/iceberg/issues/2838#issuecomment-882648002


   I mean create a UDF of type "LongType" but call it on a TimestampType column


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] izchen edited a comment on issue #2838: Bucket partition transformation from timestamp throws error

Posted by GitBox <gi...@apache.org>.
izchen edited a comment on issue #2838:
URL: https://github.com/apache/iceberg/issues/2838#issuecomment-950351465


   I think this error will be thrown when we use byte, short, timestamp, date, and binary types.
   
   https://github.com/apache/iceberg/blob/24896d84c586c6339752a69773409219c0c0ac02/spark/v3.0/spark/src/main/java/org/apache/iceberg/spark/IcebergSpark.java#L33-L38
   
   Maybe we should modify `bucket::apply` to `value -> bucket.apply(SparkValueConvertet.convert(sourceIcebergType, value))`
   
   I will submit a PR to modify the code and add more ut to cover all the atomic types of Spark.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] izchen commented on issue #2838: Bucket partition transformation from timestamp throws error

Posted by GitBox <gi...@apache.org>.
izchen commented on issue #2838:
URL: https://github.com/apache/iceberg/issues/2838#issuecomment-950351465


   I think this error will be thrown when we use timestamp, date, and binary types.
   
   https://github.com/apache/iceberg/blob/24896d84c586c6339752a69773409219c0c0ac02/spark/v3.0/spark/src/main/java/org/apache/iceberg/spark/IcebergSpark.java#L33-L38
   
   Maybe we should modify `bucket::apply` to `value -> bucket.apply(SparkValueConvertet.convert(sourceIcebergType, value))`
   
   I will submit a PR to modify the code and add more ut to cover all the atomic types of Spark.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on issue #2838: Bucket partition transformation from timestamp throws error

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on issue #2838:
URL: https://github.com/apache/iceberg/issues/2838#issuecomment-882620963






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nautilus28 commented on issue #2838: Bucket partition transformation from timestamp throws error

Posted by GitBox <gi...@apache.org>.
nautilus28 commented on issue #2838:
URL: https://github.com/apache/iceberg/issues/2838#issuecomment-882643235


   @RussellSpitzer `LongType` works but `DateType` and `TimestampType` don't work


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on issue #2838: Bucket partition transformation from timestamp throws error

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on issue #2838:
URL: https://github.com/apache/iceberg/issues/2838#issuecomment-882620963






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org