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/09/08 18:12:52 UTC

[GitHub] [iceberg] alex-shchetkov opened a new issue #3088: Can't create partitioned tables in pyspark DataFrameWriterV2

alex-shchetkov opened a new issue #3088:
URL: https://github.com/apache/iceberg/issues/3088


   Expectation: Using `writeTo().partitionedBy("x")` creates a partitioned table
   Reality: `writeTo().partitionedBy("x")` creates a partitioned-less table
   
   Quick Repro, using iceberg version 0.12:
   ```
   df = spark.createDataFrame([{"col1": "two", "tenant_id": "123"}, 
                               {"col1": "four", "tenant_id": "456"}])
   writer = (df
            .sortWithinPartitions("tenant_id")
            .writeTo(f"iceberg.ice_db.local_partition_test")
            .partitionedBy(
                "tenant_id"
            )
            .create()
   )
   ```
   
   The above creates a partitionless table:
   ```
   "partition-spec" : [ ],
     "default-spec-id" : 0,
     "partition-specs" : [ {
       "spec-id" : 0,
       "fields" : [ ]
     } ],
   ```
   ------
   Works fine when using sql directly:
   
   ```
   df.createOrReplaceGlobalTempView("partition_test")
   spark.sql(f"""
          create table iceberg.ice_db.local_partition_test2 using iceberg
          PARTITIONED BY (tenant_id)
          as select * from global_temp.partition_test order by tenant_id
          """).show()
   ```
   Produces:
   ```
   "partition-spec" : [ {
       "name" : "tenant_id",
       "transform" : "identity",
       "source-id" : 2,
       "field-id" : 1000
     } ],
   ```


-- 
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] jeff303 commented on issue #3088: Can't create partitioned tables with pyspark DataFrameWriterV2

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


   @alex-shchetkov , I think you need `.using("iceberg")` before `.create()` in the first snippet.


-- 
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] jeff303 commented on issue #3088: Can't create partitioned tables with pyspark DataFrameWriterV2

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


   @alex-shchetkov , I think you need `.using("iceberg")` before `.create()` in the first snippet.


-- 
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] peay commented on issue #3088: Can't create partitioned tables with pyspark DataFrameWriterV2

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


   @alex-shchetkov I had the same issue on Spark 3.1.2, and noticed it worked in Scala but not with `pyspark`.
   
   I believe that the root cause is https://github.com/apache/spark/commit/33deeb35f1c994328b577970d4577e6d9288bfc2, fixed in `pyspark` 3.1.3 and other patches releases.


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