xpleaf opened a new issue, #1748:
URL: https://github.com/apache/incubator-seatunnel/issues/1748

   ### Search before asking
   
   - [X] I had searched in the 
[issues](https://github.com/apache/incubator-seatunnel/issues?q=is%3Aissue+label%3A%22bug%22)
 and found no similar issues.
   
   
   ### What happened
   
   In ClickHouse defined data types for`"Int8" | "UInt8" | "Int16" | "UInt16" | 
"Int32"`, if read up data is resolved as a Byte, or Short (and this is possible 
is reasonable, I'll illustration below). Will trigger the ClassCastException.
   
   Create a table with tinyInt and SmallInt types in Hive:
   ```
   CREATE EXTERNAL TABLE `gac.tmp`(
       id int,
       key1 string,
       value1 tinyint,
       key2 string,
       value2 smallint,
       key3 bigint,
       value3 double
   )
   PARTITIONED BY (`dt` string);
   ```
   
   Define a ClickHouse table:
   
   ```sql
   CREATE TABLE gac.hive_tmp
   (
       id     Int32,
       key1   String,
       value1 Int8,
       key2   String,
       value2 Int16,
       key3   Int64,
       value3 Float64,
       dt String
   ) ENGINE = MergeTree;
   ```
   
   When Spark reads data from the Hive table, tinyint data will be interpreted 
as Byte and SmallINT data will be interpreted as Short. As a result, ClickHouse 
Sink may have a data conversion exception:
   
   `Caused by: java.lang.ClassCastException: java.lang.Byte cannot be cast to 
java.lang.Integer` or `java.lang.ClassCastException: java.lang.Short cannot be 
cast to java.lang.Integer`
   
   ### SeaTunnel Version
   
   2.1.0/dev
   
   ### SeaTunnel Config
   
   ```conf
   env {
     spark.app.name = "SeaTunnel"
     spark.sql.catalogImplementation = "hive"
     spark.executor.instances = 2
     spark.executor.cores = 1
     spark.executor.memory = "1g"
     spark.master = local
   }
   
   source {
     hive {
       pre_sql = "select id,key1,value1,key2,value2,key3,value3,dt from gac.tmp"
       result_table_name = "tmp"
     }
   }
   
   transform {}
   
   sink {
        clickhouse {
       host = "192.168.10.101:8123"
       clickhouse.socket_timeout = 50000
       database = "gac"
       table = "hive_tmp"
       fields = ["id", "key1", "value1", "key2", "value2", "key3", "value3", 
"dt"]
       username = "gac_se"
       password = "123456"
       bulk_size = 20000
        }
   }
   ```
   
   
   ### Running Command
   
   ```shell
   any
   ```
   
   
   ### Error Exception
   
   ```log
   Caused by: java.lang.ClassCastException: java.lang.Byte cannot be cast to 
java.lang.Integer
        at scala.runtime.BoxesRunTime.unboxToInt(BoxesRunTime.java:101)
        at 
org.apache.seatunnel.spark.clickhouse.sink.Clickhouse.org$apache$seatunnel$spark$clickhouse$sink$Clickhouse$$renderBaseTypeStatement(Clickhouse.scala:242)
        at 
org.apache.seatunnel.spark.clickhouse.sink.Clickhouse$$anonfun$org$apache$seatunnel$spark$clickhouse$sink$Clickhouse$$renderStatement$1.apply$mcVI$sp(Clickhouse.scala:291)
        at scala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)
        at 
org.apache.seatunnel.spark.clickhouse.sink.Clickhouse.org$apache$seatunnel$spark$clickhouse$sink$Clickhouse$$renderStatement(Clickhouse.scala:275)
        at 
org.apache.seatunnel.spark.clickhouse.sink.Clickhouse$$anonfun$output$1$$anonfun$apply$1.apply(Clickhouse.scala:96)
        at 
org.apache.seatunnel.spark.clickhouse.sink.Clickhouse$$anonfun$output$1$$anonfun$apply$1.apply(Clickhouse.scala:92)
        at scala.collection.Iterator$class.foreach(Iterator.scala:893)
        at scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
        at 
org.apache.seatunnel.spark.clickhouse.sink.Clickhouse$$anonfun$output$1.apply(Clickhouse.scala:92)
        at 
org.apache.seatunnel.spark.clickhouse.sink.Clickhouse$$anonfun$output$1.apply(Clickhouse.scala:80)
        at 
org.apache.spark.rdd.RDD$$anonfun$foreachPartition$1$$anonfun$apply$28.apply(RDD.scala:935)
        at 
org.apache.spark.rdd.RDD$$anonfun$foreachPartition$1$$anonfun$apply$28.apply(RDD.scala:935)
        at 
org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:2101)
        at 
org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:2101)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
        at org.apache.spark.scheduler.Task.run(Task.scala:121)
        at 
org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:402)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:408)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)
   
   
   or
   
   ```scala
   Caused by: java.lang.ClassCastException: java.lang.Short cannot be cast to 
java.lang.Integer
        at scala.runtime.BoxesRunTime.unboxToInt(BoxesRunTime.java:101)
        at 
org.apache.seatunnel.spark.clickhouse.sink.Clickhouse.org$apache$seatunnel$spark$clickhouse$sink$Clickhouse$$renderBaseTypeStatement(Clickhouse.scala:242)
        at 
org.apache.seatunnel.spark.clickhouse.sink.Clickhouse$$anonfun$org$apache$seatunnel$spark$clickhouse$sink$Clickhouse$$renderStatement$1.apply$mcVI$sp(Clickhouse.scala:291)
        at scala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)
        at 
org.apache.seatunnel.spark.clickhouse.sink.Clickhouse.org$apache$seatunnel$spark$clickhouse$sink$Clickhouse$$renderStatement(Clickhouse.scala:275)
        at 
org.apache.seatunnel.spark.clickhouse.sink.Clickhouse$$anonfun$output$1$$anonfun$apply$1.apply(Clickhouse.scala:96)
        at 
org.apache.seatunnel.spark.clickhouse.sink.Clickhouse$$anonfun$output$1$$anonfun$apply$1.apply(Clickhouse.scala:92)
        at scala.collection.Iterator$class.foreach(Iterator.scala:893)
        at scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
        at 
org.apache.seatunnel.spark.clickhouse.sink.Clickhouse$$anonfun$output$1.apply(Clickhouse.scala:92)
        at 
org.apache.seatunnel.spark.clickhouse.sink.Clickhouse$$anonfun$output$1.apply(Clickhouse.scala:80)
        at 
org.apache.spark.rdd.RDD$$anonfun$foreachPartition$1$$anonfun$apply$28.apply(RDD.scala:935)
        at 
org.apache.spark.rdd.RDD$$anonfun$foreachPartition$1$$anonfun$apply$28.apply(RDD.scala:935)
        at 
org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:2101)
        at 
org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:2101)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
        at org.apache.spark.scheduler.Task.run(Task.scala:121)
        at 
org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:402)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:408)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)
   ```
   ```
   
   
   ### Flink or Spark Version
   
   spark local
   
   ### Java or Scala Version
   
   jdk8
   
   ### Screenshots
   
   _No response_
   
   ### Are you willing to submit PR?
   
   - [X] Yes I am willing to submit a PR!
   
   ### Code of Conduct
   
   - [X] I agree to follow this project's [Code of 
Conduct](https://www.apache.org/foundation/policies/conduct)
   


-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to