[ 
https://issues.apache.org/jira/browse/CARBONDATA-4279?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17418708#comment-17418708
 ] 

Bigicecream commented on CARBONDATA-4279:
-----------------------------------------

I think I confused you,

the bug I opened is happening when I am not using the Location keyword in the 
table creation and I am not getting any error when inserting the data

 

In your first comment, you suggested try adding Location keyword, after doing 
it, I started getting errors when inserting data to the table and when running 
_'show segments'_, I think it is a different bug but maybe it related

 

so I will answer each case:

Without Location keyword:
 # Yes
 # Without Location keyword
 # I try to insert 4 columns(timestamp and name are normal columns and dt and 
hr are partitions) it works fine
 # It works the same when the database is not created with specifying LOCATION

With Location keyword:
 # Yes
 # With Location keyword
 # I try to insert 4 columns(timestamp and name are normal columns and dt and 
hr are partitons) 
I getting an error when I do that,
 the stack trace of the error:
{code:java}
org.apache.spark.sql.AnalysisException: Cannot insert into target table because 
number of columns mismatch;
  at 
org.apache.spark.sql.util.CarbonException$.analysisException(CarbonException.scala:23)
  at 
org.apache.spark.sql.hive.CarbonPreInsertionCasts.castChildOutput(CarbonAnalysisRules.scala:330)
  at 
org.apache.spark.sql.hive.CarbonPreInsertionCasts$$anonfun$apply$3.applyOrElse(CarbonAnalysisRules.scala:261)
  at 
org.apache.spark.sql.hive.CarbonPreInsertionCasts$$anonfun$apply$3.applyOrElse(CarbonAnalysisRules.scala:253)
  at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:286)
  at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:286)
  at 
org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:71)
  at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:285)
  at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDown(LogicalPlan.scala:29)
  at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$class.transformDown(AnalysisHelper.scala:149)
  at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
  at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
  at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:275)
  at 
org.apache.spark.sql.hive.CarbonPreInsertionCasts.apply(CarbonAnalysisRules.scala:253)
  at 
org.apache.spark.sql.hive.CarbonPreInsertionCasts.apply(CarbonAnalysisRules.scala:251)
  at 
org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1$$anonfun$2.apply(RuleExecutor.scala:92)
  at 
org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1$$anonfun$2.apply(RuleExecutor.scala:92)
  at 
org.apache.spark.sql.execution.QueryExecutionMetrics$.withMetrics(QueryExecutionMetrics.scala:141)
  at 
org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1.apply(RuleExecutor.scala:91)
  at 
org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1.apply(RuleExecutor.scala:88)
  at 
scala.collection.LinearSeqOptimized$class.foldLeft(LinearSeqOptimized.scala:124)
  at scala.collection.immutable.List.foldLeft(List.scala:84)
  at 
org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1.apply(RuleExecutor.scala:88)
  at 
org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1.apply(RuleExecutor.scala:80)
  at scala.collection.immutable.List.foreach(List.scala:392)
  at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:80)
  at 
org.apache.spark.sql.catalyst.analysis.Analyzer.org$apache$spark$sql$catalyst$analysis$Analyzer$$executeSameContext(Analyzer.scala:164)
  at 
org.apache.spark.sql.catalyst.analysis.Analyzer$$anonfun$execute$1.apply(Analyzer.scala:156)
  at 
org.apache.spark.sql.catalyst.analysis.Analyzer$$anonfun$execute$1.apply(Analyzer.scala:156)
  at 
org.apache.spark.sql.catalyst.analysis.AnalysisContext$.withLocalMetrics(Analyzer.scala:104)
  at org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:155)
  at 
org.apache.spark.sql.catalyst.analysis.Analyzer$$anonfun$executeAndCheck$1.apply(Analyzer.scala:126)
  at 
org.apache.spark.sql.catalyst.analysis.Analyzer$$anonfun$executeAndCheck$1.apply(Analyzer.scala:125)
  at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.markInAnalyzer(AnalysisHelper.scala:201)
  at 
org.apache.spark.sql.catalyst.analysis.Analyzer.executeAndCheck(Analyzer.scala:125)
  at 
org.apache.spark.sql.execution.QueryExecution.analyzed$lzycompute(QueryExecution.scala:76)
  at 
org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:74)
  at 
org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:66)
  at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:79)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:643)
  ... 52 elided
{code}

 # I cannot insert data to the table not also when the database is not created 
with specifying LOCATION
  

> Insert data to table with a partitions resulting in 'Marked for Delete' 
> segment in Spark in EMR
> -----------------------------------------------------------------------------------------------
>
>                 Key: CARBONDATA-4279
>                 URL: https://issues.apache.org/jira/browse/CARBONDATA-4279
>             Project: CarbonData
>          Issue Type: Bug
>    Affects Versions: 2.3.0
>         Environment: Release label:emr-5.24.1
> Hadoop distribution:Amazon 2.8.5
> Applications:
> Hue 4.4.0, Spark 2.4.5,JupyterHub 0.9.6
> Jar complied with:
> apache-carbondata:2.3.0-SNAPSHOT
> spark:2.4.5
> hadoop:2.8.3
>            Reporter: Bigicecream
>            Priority: Blocker
>
> as described [here|https://github.com/apache/carbondata/issues/4212]
> After the commit 
> [https://github.com/apache/carbondata/commit/42f69827e0a577b6128417104c0a49cd5bf21ad7]
> I have successfully created a table with partitions, but when I trying insert 
> data the job end with a success
>  but the segment is marked as "Marked for Delete"
> I am running:
> {code:sql}
> CREATE TABLE lior_carbon_tests.mark_for_del_bug(
> timestamp string,
> name string
> )
> STORED AS carbondata
> PARTITIONED BY (dt string, hr string)
> {code}
> {code:sql}
> INSERT INTO lior_carbon_tests.mark_for_del_bug select 
> '2021-07-07T13:23:56.012+00:00','spark','2021-07-07','13'
> {code}
> {code:sql}
> select * from lior_carbon_tests.mark_for_del_bug
> {code}
> gives:
> {code:java}
> +---------+----+---+---+
> |timestamp|name| dt| hr|
> +---------+----+---+---+
> +---------+----+---+---+
> {code}
> And
> {code:java}
> show segments for TABLE lior_carbon_tests.mark_for_del_bug
> {code}
> gives
>  
> {code:java}
> +---+-----------------+-----------------------+---------------+---------+---------+----------+-----------+
> |ID |Status           |Load Start Time        |Load Time Taken|Partition|Data 
> Size|Index Size|File Format|
> +---+-----------------+-----------------------+---------------+---------+---------+----------+-----------+
> |0  |Marked for Delete|2021-09-02 15:24:21.022|11.798S        |NA       |NA   
>     |NA        |columnar_v3|
> +---+-----------------+-----------------------+---------------+---------+---------+----------+-----------+
> {code}
>  
> I took a looking at the folder structure in S3 and it seems fine



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to