1
0

Adding ability for inserts to be written to log files

This commit is contained in:
Nishith Agarwal
2018-05-13 16:25:11 -07:00
committed by vinoth chandar
parent 34827d50e1
commit 3da063f83b
52 changed files with 1061 additions and 519 deletions

View File

@@ -21,6 +21,7 @@ package com.uber.hoodie
import com.uber.hoodie.common.model.{HoodieCommitMetadata, HoodieRecord, HoodieTableType}
import com.uber.hoodie.common.table.HoodieTableMetaClient
import com.uber.hoodie.common.util.ParquetUtils
import com.uber.hoodie.config.HoodieWriteConfig
import com.uber.hoodie.exception.HoodieException
import com.uber.hoodie.table.HoodieTable
import org.apache.hadoop.fs.Path
@@ -52,8 +53,10 @@ class IncrementalRelation(val sqlContext: SQLContext,
if (metaClient.getTableType.equals(HoodieTableType.MERGE_ON_READ)) {
throw new HoodieException("Incremental view not implemented yet, for merge-on-read datasets")
}
val hoodieTable = HoodieTable.getHoodieTable(metaClient, null)
val commitTimeline = hoodieTable.getCommitTimeline.filterCompletedInstants();
// TODO : Figure out a valid HoodieWriteConfig
val hoodieTable = HoodieTable.getHoodieTable(metaClient, HoodieWriteConfig.newBuilder().withPath(basePath).build(),
sqlContext.sparkContext)
val commitTimeline = hoodieTable.getMetaClient.getCommitTimeline.filterCompletedInstants();
if (commitTimeline.empty()) {
throw new HoodieException("No instants to incrementally pull")
}