Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -74,13 +74,30 @@ class SparkCatalog extends TableCatalog with SupportsFlussNamespaces with WithFl
} else if (e.getCause.isInstanceOf[TableAlreadyExistException]) {
throw new TableAlreadyExistsException(ident)
} else {
throw new RuntimeException(e)
throw e
}
}
}

override def alterTable(ident: Identifier, changes: TableChange*): Table = {
throw new UnsupportedOperationException("Altering table is not supported")
if (changes.size != 1 || !changes.head.isInstanceOf[TableChange.SetProperty]) {
throw new IllegalArgumentException("Altering table only supports setting properties for now")
}
try {
admin
.alterTable(toTablePath(ident), SparkConversions.toFlussTableChanges(changes).asJava, false)
.get()
loadTable(ident)
} catch {
case e: ExecutionException =>
if (e.getCause.isInstanceOf[TableNotExistException]) {
throw new NoSuchTableException(ident)
} else {
throw e
}
case e: UnsupportedOperationException =>
throw new IllegalArgumentException(e)
}
}

override def dropTable(ident: Identifier): Boolean = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import org.apache.fluss.types.RowType

import org.apache.spark.sql.FlussIdentityTransform
import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
import org.apache.spark.sql.connector.catalog.TableChange
import org.apache.spark.sql.connector.expressions.Transform
import org.apache.spark.sql.types.StructType

Expand Down Expand Up @@ -54,7 +55,7 @@ object SparkConversions {
tableDescriptorBuilder.partitionedBy(partitionKey: _*)

val primaryKeys = if (caseInsensitiveProps.contains(PRIMARY_KEY.key)) {
val pks = caseInsensitiveProps.get(PRIMARY_KEY.key).get.split(",")
val pks = caseInsensitiveProps.get(PRIMARY_KEY.key).get.split(",").map(_.trim)
schemaBuilder.primaryKey(pks: _*)
pks
} else {
Expand All @@ -64,7 +65,7 @@ object SparkConversions {
if (caseInsensitiveProps.contains(BUCKET_NUMBER.key)) {
val bucketNum = caseInsensitiveProps.get(BUCKET_NUMBER.key).get.toInt
val bucketKeys = if (caseInsensitiveProps.contains(BUCKET_KEY.key)) {
caseInsensitiveProps.get(BUCKET_KEY.key).get.split(",")
caseInsensitiveProps.get(BUCKET_KEY.key).get.split(",").map(_.trim)
} else {
primaryKeys.filterNot(partitionKey.contains)
}
Expand All @@ -76,7 +77,7 @@ object SparkConversions {
}

val (tableProps, customProps) =
caseInsensitiveProps.filterNot(SPARK_TABLE_OPTIONS.contains).partition {
caseInsensitiveProps.filterNot(e => SPARK_TABLE_OPTIONS.contains(e._1)).partition {
case (key, _) => key.startsWith(FlussConfigUtils.TABLE_PREFIX)
}

Expand All @@ -97,4 +98,13 @@ object SparkConversions {
}
partitionKeys.toArray
}

def toFlussTableChanges(changes: Seq[TableChange]): Seq[org.apache.fluss.metadata.TableChange] = {
changes.map {
case e: TableChange.SetProperty =>
org.apache.fluss.metadata.TableChange.set(e.property(), e.value())
// TODO Add full support for table changes
case _ => throw new UnsupportedOperationException("Unsupported table change")
}
}
}
18 changes: 18 additions & 0 deletions fluss-spark/fluss-spark-ut/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,10 @@
<artifactId>fluss-spark-ut_${scala.binary.version}</artifactId>
<name>Fluss : Engine Spark : UT</name>

<properties>
<paimon.version>1.3.1</paimon.version>
</properties>

<dependencies>
<dependency>
<groupId>org.apache.fluss</groupId>
Expand Down Expand Up @@ -68,6 +72,20 @@
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.fluss</groupId>
<artifactId>fluss-lake-paimon</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.paimon</groupId>
<artifactId>paimon-bundle</artifactId>
<version>${paimon.version}</version>
<scope>test</scope>
</dependency>

<!-- to avoid this issue: https://github.com/apache/arrow/issues/40896 -->
<dependency>
<groupId>org.eclipse.collections</groupId>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,19 @@
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#

org.apache.fluss.lake.paimon.PaimonLakeStoragePlugin
Loading