-
Notifications
You must be signed in to change notification settings - Fork 16
Adding support for CSV files #46
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
Open
uzadude
wants to merge
7
commits into
paypal:release/0.5
Choose a base branch
from
uzadude:csv
base: release/0.5
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from 5 commits
Commits
Show all changes
7 commits
Select commit
Hold shift + click to select a range
c1a5eed
Adding support for CSV files
uzadude c8ca5ad
supporting non psik
uzadude 7b1af93
add support in CSV split indexing and fix bug in delimiter
uzadude 1301780
restrict fetch to requested fields
uzadude 3034e30
add more types
uzadude b23a238
minor
uzadude 1ab37b5
Merge remote-tracking branch 'origin/release/0.5' into csv
uzadude File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
76 changes: 76 additions & 0 deletions
76
dione-hadoop/src/main/scala/com/paypal/dione/hdfs/index/csv/CsvIndexer.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,76 @@ | ||
| package com.paypal.dione.hdfs.index.csv | ||
|
|
||
| import com.paypal.dione.hdfs.index.{HdfsIndexer, HdfsIndexerMetadata} | ||
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.fs.Path | ||
| import org.apache.hadoop.mapreduce.lib.input.{FileSplit, LineRecordReader} | ||
| import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl | ||
| import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType} | ||
| import org.slf4j.LoggerFactory | ||
|
|
||
| /** | ||
| * based on org.apache.spark.sql.execution.datasources.HadoopFileLinesReader | ||
| */ | ||
|
|
||
| case class CsvIndexer(file: Path, start: Long, end: Long, conf: Configuration, | ||
| delimiter: Char) | ||
| extends HdfsIndexer[Seq[String]]() { | ||
|
|
||
| var reader: LineRecordReader = _ | ||
| private var curPosition = -1L | ||
| private var lastPosition = -1L | ||
|
|
||
| override def closeCurrentFile(): Unit = { | ||
| reader.close() | ||
| } | ||
|
|
||
| /** | ||
| * Regular seek. Called once per offset (block). | ||
| */ | ||
| override def seek(offset: Long): Unit = { | ||
| // @TODO - need to better optimize. not to re-init on every seek | ||
| initReader(offset, end) | ||
| } | ||
|
|
||
| /** | ||
| * Skip the next row - can avoid deserialization, etc. | ||
| */ | ||
| override def skip(): Unit = { | ||
| if (reader==null) { | ||
| initReader(start, end) | ||
| } | ||
| reader.nextKeyValue() | ||
| } | ||
|
|
||
| /** | ||
| * Read the next row | ||
| */ | ||
| override def readLine(): Seq[String] = { | ||
| if (reader==null) { | ||
| initReader(start, end) | ||
| } | ||
|
|
||
| lastPosition = curPosition | ||
| curPosition = Option(reader.getCurrentKey).map(_.get()).getOrElse(0) | ||
|
|
||
| val hasNext = reader.nextKeyValue() | ||
| if (hasNext) { | ||
| reader.getCurrentValue.toString.split(delimiter).toSeq | ||
| } else null | ||
| } | ||
|
|
||
| override def getCurMetadata(): HdfsIndexerMetadata = { | ||
| // some quirk of the first line read | ||
| val plusOne = if (lastPosition == curPosition) 1 else 0 | ||
| val pos = if (curPosition==0) start else curPosition | ||
| HdfsIndexerMetadata(file.toString, pos, plusOne) | ||
| } | ||
|
|
||
| val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) | ||
| val hadoopAttemptContext = new TaskAttemptContextImpl(conf, attemptId) | ||
|
|
||
| private def initReader(start: Long, end: Long) = { | ||
| reader = new LineRecordReader() | ||
| reader.initialize(new FileSplit(file, start, end - start, Array.empty), hadoopAttemptContext) | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,4 @@ | ||
| a1,b1,c1,d1,11,12,13 | ||
| a2,b2,c2,d2,21,22,23 | ||
| a3,b3,c3,d3,31,32,33 | ||
| a4,b4,c4,d4,41,42,43 |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
95 changes: 95 additions & 0 deletions
95
dione-hadoop/src/test/scala/com/paypal/dione/hdfs/index/csv/TestCsvIndexer.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,95 @@ | ||
| package com.paypal.dione.hdfs.index.csv | ||
|
|
||
| import com.paypal.dione.hdfs.index.HdfsIndexerMetadata | ||
| import com.paypal.dione.hdfs.index.csv.TestCsvIndexer.{entries, splitEntries} | ||
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.fs.Path | ||
| import org.junit.jupiter.api.MethodOrderer.OrderAnnotation | ||
| import org.junit.jupiter.api._ | ||
|
|
||
| object TestCsvIndexer { | ||
| var entries: Seq[(Seq[String], HdfsIndexerMetadata)] = _ | ||
| var splitEntries: Seq[(Seq[String], HdfsIndexerMetadata)] = _ | ||
| } | ||
|
|
||
| @TestMethodOrder(classOf[OrderAnnotation]) | ||
| class TestCsvIndexer { | ||
|
|
||
| val bizlogFile = new Path("src/test/resources/csv/file.csv") | ||
| val fileSystem = bizlogFile.getFileSystem(new Configuration()) | ||
|
|
||
| @Test | ||
| @Order(1) | ||
| def testSimpleCreateIndex(): Unit = { | ||
| entries = CsvIndexer(bizlogFile, 0, 1<<30, fileSystem.getConf, ',').iteratorWithMetadata.toList | ||
|
|
||
| Assertions.assertEquals(4, entries.size) | ||
|
|
||
| entries.take(10).foreach(println) | ||
|
|
||
| Assertions.assertEquals(entries.head._2, HdfsIndexerMetadata(bizlogFile.toString, 0, 0, -1)) | ||
| Assertions.assertEquals(entries.head._1.toList, List("a1", "b1", "c1", "d1", "11", "12", "13")) | ||
| Assertions.assertEquals(entries(2)._2, HdfsIndexerMetadata(bizlogFile.toString, 21, 0, -1)) | ||
| } | ||
|
|
||
| @Test | ||
| @Order(1) | ||
| def testCreateIndexFromSplit(): Unit = { | ||
| splitEntries = CsvIndexer(bizlogFile, 30, 1000, fileSystem.getConf, ',').iteratorWithMetadata.toList | ||
|
|
||
| Assertions.assertEquals(2, splitEntries.size) | ||
|
|
||
| splitEntries.take(10).foreach(println) | ||
|
|
||
| Assertions.assertEquals(splitEntries.head._2, HdfsIndexerMetadata(bizlogFile.toString, 30, 0, -1)) | ||
| Assertions.assertEquals(splitEntries.head._1.toList, List("a3", "b3", "c3", "d3", "31", "32", "33")) | ||
| Assertions.assertEquals(splitEntries.tail.head._2, HdfsIndexerMetadata(bizlogFile.toString, 42, 0, -1)) | ||
| } | ||
|
|
||
| @Order(2) | ||
| @Test | ||
| def testSimpleFetch(): Unit = { | ||
|
|
||
| val csvIndexer = CsvIndexer(bizlogFile, 0, 1 << 30, fileSystem.getConf, ',') | ||
|
|
||
| { | ||
| val sq = csvIndexer.fetch(HdfsIndexerMetadata(bizlogFile.toString, 0, 0)) | ||
| Assertions.assertEquals("c1", sq(2)) | ||
| Assertions.assertEquals("d1", sq(3)) | ||
| } | ||
|
|
||
| { | ||
| val sq = csvIndexer.fetch(HdfsIndexerMetadata(bizlogFile.toString, 21, 0)) | ||
| Assertions.assertEquals("c3", sq(2)) | ||
| Assertions.assertEquals("d3", sq(3)) | ||
| } | ||
|
|
||
| { | ||
| val sq = csvIndexer.fetch(HdfsIndexerMetadata(bizlogFile.toString, 42, 0)) | ||
| Assertions.assertEquals("b4", sq(1)) | ||
| Assertions.assertEquals("41", sq(4)) | ||
| } | ||
| } | ||
|
|
||
| @Order(2) | ||
| @Test | ||
| def testAllEntiresFetch(): Unit = { | ||
| testEntriesFetch(entries) | ||
| } | ||
|
|
||
| @Order(2) | ||
| @Test | ||
| def testSplitEntiresFetch(): Unit = { | ||
| testEntriesFetch(splitEntries) | ||
| } | ||
|
|
||
| def testEntriesFetch(entries: Seq[(Seq[String], HdfsIndexerMetadata)]): Unit = { | ||
| val csvIndexer = CsvIndexer(bizlogFile, 0, 1 << 30, fileSystem.getConf, ',') | ||
| entries.foreach(e => { | ||
| println("fetching: " + e._2) | ||
| val sq = csvIndexer.fetch(e._2) | ||
| Assertions.assertEquals(e._1, sq) | ||
| }) | ||
| } | ||
|
|
||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
65 changes: 65 additions & 0 deletions
65
dione-spark/src/main/scala/com/paypal/dione/spark/index/csv/CsvSparkIndexer.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,65 @@ | ||
| package com.paypal.dione.spark.index.csv | ||
|
|
||
| import com.paypal.dione.hdfs.index.HdfsIndexer | ||
| import com.paypal.dione.hdfs.index.csv.CsvIndexer | ||
| import com.paypal.dione.spark.index._ | ||
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.fs.Path | ||
| import org.apache.hadoop.hive.serde2.`lazy`.{LazySerDeParameters, LazyUtils} | ||
| import org.apache.spark.sql.SparkSession | ||
| import org.apache.spark.sql.types._ | ||
|
|
||
| object CsvSparkIndexer extends IndexManagerFactory { | ||
|
|
||
| override def canResolve(inputFormat: String, serde: String): Boolean = | ||
| inputFormat.contains("org.apache.hadoop.mapred.TextInputFormat") | ||
|
|
||
| override def createSparkIndexer(spark: SparkSession, indexSpec: IndexSpec): SparkIndexer = | ||
| CsvSparkIndexer(spark, indexSpec.dataTableName) | ||
| } | ||
|
|
||
| case class CsvSparkIndexer(@transient spark: SparkSession, dataTableName: String) | ||
| extends SparkIndexer { | ||
|
|
||
| override type T = Seq[String] | ||
|
|
||
| private val sparkCatalogTable = IndexManagerUtils.getSparkCatalogTable(spark, dataTableName) | ||
| private val schemaWithoutPartitionCols = spark.table(dataTableName).drop(sparkCatalogTable.partitionColumnNames: _*).schema | ||
| private var requestedFieldsSchema: Seq[(Int, StructField)] = _ | ||
|
|
||
| def initHdfsIndexer(file: Path, conf: Configuration, start: Long, end: Long, fieldsSchema: StructType): HdfsIndexer[Seq[String]] = { | ||
| if (requestedFieldsSchema == null) { | ||
| val fieldsMap = schemaWithoutPartitionCols.map(_.name).zipWithIndex.toMap | ||
| requestedFieldsSchema = fieldsSchema.map(f => fieldsMap(f.name) -> f) | ||
| } | ||
|
|
||
| // logic from org.apache.hadoop.hive.serde2.lazy.LazySerDeParameters.collectSeparators() | ||
| val tableProperties = sparkCatalogTable.storage.properties | ||
| val delimiter = LazyUtils.getByte(tableProperties.getOrElse("field.delim", | ||
| tableProperties.getOrElse("serialization.format", null)), LazySerDeParameters.DefaultSeparators(0)) | ||
| CsvIndexer(file, start, end, conf, delimiter.toChar) | ||
| } | ||
|
|
||
| def convert(t: Seq[String]): Seq[Any] = { | ||
| val l = t.toList | ||
| requestedFieldsSchema.map(rf => convertAny(rf._2.dataType, l(rf._1))) | ||
| } | ||
|
|
||
| def convertMap(t: Seq[String]): Map[String, Any] = { | ||
| val l = t.toList | ||
| requestedFieldsSchema.map(rf => rf._2.name -> convertAny(rf._2.dataType, l(rf._1))).toMap | ||
| } | ||
|
|
||
| private def convertAny(dataType: DataType, strVal: String): Any = | ||
| dataType match { | ||
| case _: StringType => strVal | ||
| case _: LongType => strVal.toLong | ||
| case _: IntegerType => strVal.toInt | ||
| case _: BooleanType => strVal.toBoolean | ||
| case _: FloatType => strVal.toFloat | ||
| case _: DoubleType => strVal.toDouble | ||
| case _: CharType => strVal.charAt(0) | ||
| case other => throw new RuntimeException("type " + other + " is currently not support for CsvSparkIndexer") | ||
| } | ||
|
|
||
| } |
20 changes: 20 additions & 0 deletions
20
dione-spark/src/test/scala/com/paypal/dione/spark/index/csv/TestCsvIndexManagerBase.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,20 @@ | ||
| package com.paypal.dione.spark.index.csv | ||
|
|
||
| import com.paypal.dione.spark.index.{IndexSpec, TestIndexManagerBase} | ||
| import org.junit.jupiter.api.MethodOrderer.OrderAnnotation | ||
| import org.junit.jupiter.api._ | ||
|
|
||
|
|
||
| @TestMethodOrder(classOf[OrderAnnotation]) | ||
| class TestCsvIndexManagerBase extends TestIndexManagerBase { | ||
|
|
||
| lazy val indexSpec: IndexSpec = IndexSpec("csv_data_tbl", "csv_data_tbl_idx", Seq("id_col")) | ||
|
|
||
| def initDataTable(fieldsSchema: String, partitionFieldSchema: String): Unit = { | ||
| spark.sql(s"create table ${indexSpec.dataTableName} ($fieldsSchema) partitioned by ($partitionFieldSchema)" + | ||
| s" row format delimited" + | ||
| s" fields terminated by '16'") | ||
| } | ||
|
|
||
| val testSamples = Seq(SampleTest("msg_100", Nil, "var_a_100", 607, 0, -1)) | ||
| } |
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
So I guess this is only for random fetches, right? The performance will be impractical for batch...
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
but than again, why not just save the reader open? am I missing anything?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
currently, the requested use-case is only fetches. but sure I would like to solve for both. started just with something that works.
In CSV we have only offset, sub-offset is almost only zero.
let's say we want to read every other row, then we need to understand when it will be better just to skip a row compared to when will we want to "re-init" from a new offset.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ok I got it... In the parquet implementation we have basically the same thing, right? I mean just one offset, when the row group is 128MB or more