-
-
Notifications
You must be signed in to change notification settings - Fork 67
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(core): make path filter configurable #329
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -1,12 +1,17 @@ | ||
package com.github.mjakubowski84.parquet4s | ||
|
||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.fs.{FileAlreadyExistsException, FileStatus, FileSystem, RemoteIterator} | ||
import org.apache.hadoop.fs.FileAlreadyExistsException | ||
import org.apache.hadoop.fs.FileStatus | ||
import org.apache.hadoop.fs.FileSystem | ||
import org.apache.hadoop.fs.PathFilter | ||
import org.apache.hadoop.fs.RemoteIterator | ||
import org.apache.parquet.hadoop.ParquetFileWriter | ||
import org.apache.parquet.hadoop.util.HiddenFileFilter | ||
import org.slf4j.Logger | ||
|
||
import scala.concurrent.{ExecutionContext, Future} | ||
import scala.concurrent.ExecutionContext | ||
import scala.concurrent.Future | ||
import scala.util.matching.Regex | ||
|
||
private[parquet4s] object IOOps { | ||
|
@@ -67,12 +72,18 @@ trait IOOps { | |
} | ||
} | ||
|
||
/** @param path | ||
* a location in a file tree from which `findPartitionedPaths` collects descendant paths recursively | ||
* @param pathFilter | ||
* `findPartitionedPaths` traverses paths that match this predicate | ||
*/ | ||
protected def findPartitionedPaths( | ||
path: Path, | ||
configuration: Configuration | ||
configuration: Configuration, | ||
pathFilter: PathFilter = HiddenFileFilter.INSTANCE | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 👍 |
||
): Either[Exception, PartitionedDirectory] = { | ||
val fs = path.toHadoop.getFileSystem(configuration) | ||
findPartitionedPaths(fs, configuration, path, List.empty).fold( | ||
findPartitionedPaths(fs, configuration, path, pathFilter, List.empty).fold( | ||
PartitionedDirectory.failed, | ||
PartitionedDirectory.apply | ||
) | ||
|
@@ -82,10 +93,11 @@ trait IOOps { | |
fs: FileSystem, | ||
configuration: Configuration, | ||
path: Path, | ||
pathFilter: PathFilter, | ||
partitions: List[Partition] | ||
): Either[List[Path], List[PartitionedPath]] = { | ||
val (dirs, files) = fs | ||
.listStatus(path.toHadoop, HiddenFileFilter.INSTANCE) | ||
.listStatus(path.toHadoop, pathFilter) | ||
.toList | ||
.partition(_.isDirectory) | ||
if (dirs.nonEmpty && files.nonEmpty) | ||
|
@@ -96,11 +108,11 @@ trait IOOps { | |
Right(List.empty) // empty leaf dir | ||
else if (partitionedDirs.isEmpty) | ||
// leaf files | ||
Right(files.map(fileStatus => PartitionedPath(fileStatus, configuration, partitions))) | ||
Right(files.map(PartitionedPath(_, configuration, partitions))) | ||
else | ||
partitionedDirs | ||
.map { case (subPath, partition) => | ||
findPartitionedPaths(fs, configuration, subPath, partitions :+ partition) | ||
findPartitionedPaths(fs, configuration, subPath, pathFilter, partitions :+ partition) | ||
} | ||
.foldLeft[Either[List[Path], List[PartitionedPath]]](Right(List.empty)) { | ||
case (Left(invalidPaths), Left(moreInvalidPaths)) => | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -2,11 +2,15 @@ package com.github.mjakubowski84.parquet4s | |
|
||
import com.github.mjakubowski84.parquet4s.etl.CompoundParquetIterable | ||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.fs.PathFilter | ||
import org.apache.parquet.filter2.compat.FilterCompat | ||
import org.apache.parquet.hadoop | ||
import org.apache.parquet.hadoop.util.HadoopInputFile | ||
import org.apache.parquet.io.InputFile | ||
import org.apache.parquet.schema.{MessageType, Type} | ||
import org.slf4j.{Logger, LoggerFactory} | ||
import org.apache.parquet.schema.MessageType | ||
import org.apache.parquet.schema.Type | ||
import org.slf4j.Logger | ||
import org.slf4j.LoggerFactory | ||
|
||
import java.io.Closeable | ||
import java.util.TimeZone | ||
|
@@ -42,6 +46,12 @@ object ParquetReader extends IOOps { | |
*/ | ||
def filter(filter: Filter): Builder[T] | ||
|
||
/** @param pathFilter | ||
* optional path filter; ParquetReader traverses paths that match this predicate to resolve partitions. It uses | ||
* org.apache.parquet.hadoop.util.HiddenFileFilter by default. | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Mentioning Current HiddenFileFilter definition is as simple as There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Parquet4s relies so much on parquet-hadoop that I am not so concerned about leaking this detail. |
||
*/ | ||
def pathFilter(filter: PathFilter): Builder[T] | ||
|
||
/** Attempt to read data as partitioned. Partition names must follow Hive format. Partition values will be set in | ||
* read records to corresponding fields. | ||
*/ | ||
|
@@ -77,14 +87,17 @@ object ParquetReader extends IOOps { | |
options: ParquetReader.Options = ParquetReader.Options(), | ||
filter: Filter = Filter.noopFilter, | ||
projectedSchemaResolverOpt: Option[ParquetSchemaResolver[T]] = None, | ||
columnProjections: Seq[ColumnProjection] = Seq.empty | ||
columnProjections: Seq[ColumnProjection] = Seq.empty, | ||
pathFilter: PathFilter = hadoop.util.HiddenFileFilter.INSTANCE | ||
) extends Builder[T] { | ||
override def options(options: ParquetReader.Options): Builder[T] = | ||
this.copy(options = options) | ||
|
||
override def filter(filter: Filter): Builder[T] = | ||
this.copy(filter = filter) | ||
|
||
override def pathFilter(filter: PathFilter): Builder[T] = this.copy(pathFilter = pathFilter) | ||
|
||
override def partitioned: Builder[T] = this | ||
|
||
override def read(path: Path)(implicit decoder: ParquetRecordDecoder[T]): ParquetIterable[T] = | ||
|
@@ -96,7 +109,7 @@ object ParquetReader extends IOOps { | |
|
||
inputFile match { | ||
case hadoopInputFile: HadoopInputFile => | ||
partitionedIterable(Path(hadoopInputFile.getPath), valueCodecConfiguration, hadoopConf) | ||
partitionedIterable(Path(hadoopInputFile.getPath), valueCodecConfiguration, hadoopConf, pathFilter) | ||
case _ => | ||
singleIterable( | ||
inputFile = inputFile, | ||
|
@@ -112,9 +125,10 @@ object ParquetReader extends IOOps { | |
private def partitionedIterable( | ||
path: Path, | ||
valueCodecConfiguration: ValueCodecConfiguration, | ||
hadoopConf: Configuration | ||
hadoopConf: Configuration, | ||
pathFilter: PathFilter | ||
)(implicit decoder: ParquetRecordDecoder[T]): ParquetIterable[T] = | ||
findPartitionedPaths(path, hadoopConf) match { | ||
findPartitionedPaths(path, hadoopConf, pathFilter) match { | ||
case Left(exception) => | ||
throw exception | ||
case Right(partitionedDirectory) => | ||
|
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.
I just ran
organize imports
by scalafmt