Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat: integrate hive metastore for reading and write iceberg table #245

Merged
merged 2 commits into from
Aug 12, 2021
Merged
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
14 changes: 14 additions & 0 deletions java/openmldb-batch/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -259,6 +259,8 @@
<artifactId>junit</artifactId>
<version>4.13.1</version>
</dependency>

<!-- Spark -->
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
Expand All @@ -271,6 +273,12 @@
<version>${spark.version}</version>
<scope>${spark.dependencyScope}</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-hive_${scala.binary.version}</artifactId>
<version>${spark.version}</version>
</dependency>

<dependency>
<groupId>org.yaml</groupId>
<artifactId>snakeyaml</artifactId>
Expand Down Expand Up @@ -339,6 +347,12 @@
<version>0.11.1</version>
</dependency>

<dependency>
<groupId>org.apache.iceberg</groupId>
<artifactId>iceberg-hive</artifactId>
<version>0.9.1</version>
</dependency>

</dependencies>

<build>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -123,11 +123,17 @@ class OpenmldbBatchConfig extends Serializable {
//@ConfigOption(name="openmldb.hybridse.jsdk.path", doc="The path of HybridSE jsdk core file path")
var hybridseJsdkLibraryPath = ""

@ConfigOption("openmldb.enable.hive.metastore", "Need to set hive.metastore.uris")
var enableHiveMetaStore = false

@ConfigOption("openmldb.hive.default.database", "The default database from hive metastore")
var defaultHiveDatabase = "default"

@ConfigOption(name="openmldb.hadoop.warehouse.path", doc="The path of Hadoop warehouse")
var hadoopWarehousePath = ""

@ConfigOption(name="openmldb.iceberg.catalog.name", doc="The name of Iceberg catalog")
val icebergCatalogName = "iceberg_catalog"
val icebergHadoopCatalogName = "iceberg_catalog"

}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration
import org.apache.iceberg.PartitionSpec
import org.apache.iceberg.catalog.TableIdentifier
import org.apache.iceberg.hadoop.HadoopCatalog
import org.apache.iceberg.hive.HiveCatalog
import org.apache.iceberg.spark.SparkSchemaUtil
import org.apache.spark.SparkConf
import org.apache.spark.sql.catalyst.QueryPlanningTracker
Expand Down Expand Up @@ -58,6 +59,13 @@ class OpenmldbSession {
this.sparkSession = sparkSession
this.config = OpenmldbBatchConfig.fromSparkSession(sparkSession)
this.setDefaultSparkConfig()

// TODO: Register table if using other constructors
val catalogTables = this.sparkSession.catalog.listTables(config.defaultHiveDatabase).collect()
for (table <- catalogTables) {
logger.info(s"Register table ${table.name} for OpenMLDB engine")
registerTable(table.name, this.sparkSession.table(table.name))
}
}

/**
Expand All @@ -81,11 +89,15 @@ class OpenmldbSession {
// TODO: Need to set for official Spark 2.3.0 jars
//logger.debug("Set spark.hadoop.yarn.timeline-service.enabled as false")
//builder.config("spark.hadoop.yarn.timeline-service.enabled", value = false)

builder.config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")
builder.appName("App").master(sparkMaster)

if (config.enableHiveMetaStore) {
builder.enableHiveSupport()
}

this.sparkSession = builder.appName("App")
.master(sparkMaster)
.getOrCreate()
this.sparkSession = builder.getOrCreate()

this.setDefaultSparkConfig()
}
Expand All @@ -100,9 +112,18 @@ class OpenmldbSession {
sparkConf.set("spark.sql.session.timeZone", config.timeZone)

// Set Iceberg catalog
sparkConf.set("spark.sql.catalog.%s".format(config.icebergCatalogName), "org.apache.iceberg.spark.SparkCatalog")
sparkConf.set("spark.sql.catalog.%s.type".format(config.icebergCatalogName), "hadoop")
sparkConf.set("spark.sql.catalog.%s.warehouse".format(config.icebergCatalogName), this.config.hadoopWarehousePath)
if (!config.hadoopWarehousePath.isEmpty) {
sparkConf.set("spark.sql.catalog.%s".format(config.icebergHadoopCatalogName), "org.apache.iceberg.spark.SparkCatalog")
sparkConf.set("spark.sql.catalog.%s.type".format(config.icebergHadoopCatalogName), "hadoop")
sparkConf.set("spark.sql.catalog.%s.warehouse".format(config.icebergHadoopCatalogName), this.config.hadoopWarehousePath)
}

if (config.enableHiveMetaStore) {
sparkConf.set("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog")
sparkConf.set("spark.sql.catalog.spark_catalog.type", "hive")
// TODO: Check if "hive.metastore.uris" is set or not
}

}

/**
Expand Down Expand Up @@ -231,14 +252,35 @@ class OpenmldbSession {
* Create table and import data from DataFrame to offline storage.
*/
def importToOfflineStorage(databaseName: String, tableName: String, df: DataFrame): Unit = {
createOfflineTable(databaseName, tableName, df)
appendOfflineTable(databaseName, tableName, df)
createHiveTable(databaseName, tableName, df)
appendHiveTable(databaseName, tableName, df)
}

/**
* Create table in offline storage.
*/
def createOfflineTable(databaseName: String, tableName: String, df: DataFrame): Unit = {
def createHiveTable(databaseName: String, tableName: String, df: DataFrame): Unit = {
// TODO: Check if table exists

logger.info("Register the table %s to create table in offline storage".format(tableName))
df.createOrReplaceTempView(tableName)

val conf = getSparkSession.sessionState.newHadoopConf()
val catalog = new HiveCatalog(conf)
val icebergSchema = SparkSchemaUtil.schemaForTable(getSparkSession, tableName)
val partitionSpec = PartitionSpec.builderFor(icebergSchema).build()
val tableIdentifier = TableIdentifier.of(databaseName, tableName)

catalog.createTable(tableIdentifier, icebergSchema, partitionSpec)

// Register table in OpenMLDB engine
registerTable(tableName, df)
}

/**
* Create table in offline storage.
*/
def createHadoopCatalogTable(databaseName: String, tableName: String, df: DataFrame): Unit = {
// TODO: Check if table exists

logger.info("Register the table %s to create table in offline storage".format(tableName))
Expand All @@ -257,11 +299,25 @@ class OpenmldbSession {
/**
* Append data from DataFrame to offline storage.
*/
def appendOfflineTable(databaseName: String, tableName: String, df: DataFrame): Unit = {
def appendHiveTable(databaseName: String, tableName: String, df: DataFrame): Unit = {
logger.info("Register the table %s to append data in offline storage".format(tableName))
df.createOrReplaceTempView(tableName)

// TODO: Support other catalog name if possible
val icebergTableName = "%s.%s.%s".format("spark_catalog", databaseName, tableName)

// Insert parquet to Iceberg table
getSparkSession.table(tableName).writeTo(icebergTableName).append()
}

/**
* Append data from DataFrame to offline storage.
*/
def appendHadoopCatalogTable(databaseName: String, tableName: String, df: DataFrame): Unit = {
logger.info("Register the table %s to append data in offline storage".format(tableName))
df.createOrReplaceTempView(tableName)

val icebergTableName = "%s.%s.%s".format(config.icebergCatalogName, databaseName, tableName)
val icebergTableName = "%s.%s.%s".format(config.icebergHadoopCatalogName, databaseName, tableName)

// Insert parquet to Iceberg table
this.getSparkSession.table(tableName).writeTo(icebergTableName).append()
Expand Down