> On Sept. 13, 2016, 1:37 a.m., Yi Pan (Data Infrastructure) wrote: > > samza-hdfs/src/main/java/org/apache/samza/system/hdfs/partitioner/DirectoryPartitioner.java, > > line 58 > > <https://reviews.apache.org/r/51142/diff/5/?file=1493803#file1493803line58> > > > > nit: since the input whiteList/blakcList are also regex, shouldn't we > > just name them the same?
Separating white list and black list simplifies the regex a lot. I see this convention in databus, Kafka (http://kafka.apache.org/documentation.html) and many other systems. > On Sept. 13, 2016, 1:37 a.m., Yi Pan (Data Infrastructure) wrote: > > samza-hdfs/src/main/java/org/apache/samza/system/hdfs/partitioner/DirectoryPartitioner.java, > > line 77 > > <https://reviews.apache.org/r/51142/diff/5/?file=1493803#file1493803line77> > > > > Make sure that you check-in to open source after we disable JDK7 build. > > This won't work for JDK7 build in open source. Will do. > On Sept. 13, 2016, 1:37 a.m., Yi Pan (Data Infrastructure) wrote: > > samza-hdfs/src/main/java/org/apache/samza/system/hdfs/partitioner/DirectoryPartitioner.java, > > line 83 > > <https://reviews.apache.org/r/51142/diff/5/?file=1493803#file1493803line83> > > > > Question: this seems to be highly related to how the HDFS files are > > organized. It is hard to see how a common practice would look like, > > especially in open source. Can we make the groupIdentifier pluggable? Why is it HDFS specific? At the very least, it can apply to any file system like systems. The idea of grouping (or advanced partitioning) is to allow multiple highly related files or, say AWS S3Objects, to be processed by the same task. Anyway, this is sort of pluggable already. If you don't specify "group.pattern" then the group identifier will be the entire file name (i.e. each group will simply be each single file themselves). > On Sept. 13, 2016, 1:37 a.m., Yi Pan (Data Infrastructure) wrote: > > samza-hdfs/src/main/java/org/apache/samza/system/hdfs/partitioner/DirectoryPartitioner.java, > > line 162 > > <https://reviews.apache.org/r/51142/diff/5/?file=1493803#file1493803line162> > > > > We had an on-going issue that the partitionId to the ssp mapping does > > not seem to be consistent between the job restarts. I suspect that might be > > a problem here as well, if the groupedPartitions list is not sorted in a > > consistent order? Wait, that would be a huge issue of Samza... I don't understand how is the mapping between partition id and ssp not consistent? The ssp contains the partition id itself, right? > On Sept. 13, 2016, 1:37 a.m., Yi Pan (Data Infrastructure) wrote: > > samza-hdfs/src/main/java/org/apache/samza/system/hdfs/partitioner/DirectoryPartitioner.java, > > line 174 > > <https://reviews.apache.org/r/51142/diff/5/?file=1493803#file1493803line174> > > > > Now I see what this PartitionDescriptor really mean... Is it much more > > straightforward if renamed to partitionToFilePathsMap? I started to realize that the name of partition descriptor isn't informative enough. I have updated the design doc to list it in the glossary section. My problem with "partitionToFilePath(s)" is just that, well, it's not a noun. "Descriptor" is more concise. > On Sept. 13, 2016, 1:37 a.m., Yi Pan (Data Infrastructure) wrote: > > samza-hdfs/src/main/java/org/apache/samza/system/hdfs/reader/AvroFileHdfsReader.java, > > line 24 > > <https://reviews.apache.org/r/51142/diff/5/?file=1493806#file1493806line24> > > > > One concern I had w/ this HdfsAvroFileReader/Writer is the version > > conflict issue. LinkedIn's Kafka version still uses avro-1.4 in the serde, > > while hdfs already uses avro-1.7 in 2.6.1. I guess that we need to find a > > solution inside LinkedIn to resolve it. Let's sync up face-to-face tomorrow. I was well aware of the avro issue. I tried so many different APIs that I finally found the set of APIs that work for both 1.4 and 1.7 - Hai ----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/51142/#review148629 ----------------------------------------------------------- On Sept. 9, 2016, 1:34 a.m., Hai Lu wrote: > > ----------------------------------------------------------- > This is an automatically generated e-mail. To reply, visit: > https://reviews.apache.org/r/51142/ > ----------------------------------------------------------- > > (Updated Sept. 9, 2016, 1:34 a.m.) > > > Review request for samza, Chris Pettitt, Yi Pan (Data Infrastructure), and > Navina Ramesh. > > > Bugs: SAMZA-967 > https://issues.apache.org/jira/browse/SAMZA-967 > > > Repository: samza > > > Description > ------- > > Add HDFS System Consumer: > > 1. System admin, partitioner > 2. System consumer with metrics > > Design doc can be found here: > https://issues.apache.org/jira/secure/attachment/12824078/HDFSSystemConsumer.pdf > > An overview of the high level architecture: > > The system factory is used by Samza to instantiate SystemConsumer, > SystemProducer, and SystemAdmin for a specific system. The > FileDataSystemFactory can be reused for other file system like sources. > > HDFSSystemAdmin will start a “DirectoryPartitioner” to figure out the set of > HDFS files need to be consumed for this job. The DirectoryPartitioner also > uses “GroupingPattern” to group files into partitions if advanced > partitioning is required. HDFSSystemAdmin will then persist the > “PartitionDescriptor” to HDFS. > > The HDFSSystemConsumer will then pick up the “PartitionDescriptor” from HDFS. > Based on this information as well as the actual assignment of partitions, it > would then know which files to read from. > > The initial implementation of the HDFS system consumer supports only avro > data files. It’s very easy to extend it to a variety of file format by > implementing the FileReader interface. > > > > > +------------------------------------------------------------------------------+ > > | > | > +-----------------+ HDFS > | > | Obtain | > | > | Partition > +------+----------------------^------+---------------------------------^-------+ > > | Description | | | > | > | | | | > | > | +-------------v-------+ | | > Filtering/ | > | | | | +---+ > Grouping +-----+ > | | HDFSAvroFileReader | | | > | > | | | Persist | | > | > | +---------+-----------+ Partition | | > | > | | Description | > +------v--------------+ +----------+----------+ > | | | | > | | | > | +---------+-----------+ | |Directory > Partitioner| | HDFSAvroWriter | > | | IFileReader | | | > | | | > | | | | > +------+--------------+ +----------+----------+ > | +---------+-----------+ | | > | > | | | | > | > | | | | > | > | +---------+-----------+ > +-+----------+--------+ +----------+----------+ > | | | | > | | | > | | HDFSSystemConsumer | | HDFSSystemAdmin > | | HDFSSystemProducer | > +----------> | | > | | | > +---------+-----------+ > +-----------+---------+ +----------+----------+ > | | > | > > +------------------------------------+------------------------------------+ > > | > > > +---------------------------------------+--------------------------------------+ > > | > | > | HDFSSystemFactory > | > | > | > > +------------------------------------------------------------------------------+ > > > Diffs > ----- > > build.gradle 1d4eb74b1294318db8454631ddd0901596121ab2 > gradle/dependency-versions.gradle 47c71bfde027835682889407261d4798b629d214 > samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemAdmin.java > PRE-CREATION > > samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemConsumer.java > PRE-CREATION > > samza-hdfs/src/main/java/org/apache/samza/system/hdfs/PartitionDescriptionUtil.java > PRE-CREATION > > samza-hdfs/src/main/java/org/apache/samza/system/hdfs/partitioner/DirectoryPartitioner.java > PRE-CREATION > > samza-hdfs/src/main/java/org/apache/samza/system/hdfs/partitioner/FileSystemAdapter.java > PRE-CREATION > > samza-hdfs/src/main/java/org/apache/samza/system/hdfs/partitioner/HdfsFileSystemAdapter.java > PRE-CREATION > > samza-hdfs/src/main/java/org/apache/samza/system/hdfs/reader/AvroFileHdfsReader.java > PRE-CREATION > > samza-hdfs/src/main/java/org/apache/samza/system/hdfs/reader/HdfsReaderFactory.java > PRE-CREATION > > samza-hdfs/src/main/java/org/apache/samza/system/hdfs/reader/MultiFileHdfsReader.java > PRE-CREATION > > samza-hdfs/src/main/java/org/apache/samza/system/hdfs/reader/SingleFileHdfsReader.java > PRE-CREATION > samza-hdfs/src/main/scala/org/apache/samza/system/hdfs/HdfsConfig.scala > 61b7570afae3219b618c8830905035063941bdd7 > > samza-hdfs/src/main/scala/org/apache/samza/system/hdfs/HdfsSystemAdmin.scala > 92eb4472533db67dca01f075cb460581b4bdac0d > > samza-hdfs/src/main/scala/org/apache/samza/system/hdfs/HdfsSystemFactory.scala > ef3c20a097ddf2feecaf8b0ad4587ea4bf6570b7 > > samza-hdfs/src/test/java/org/apache/samza/system/hdfs/TestHdfsSystemConsumer.java > PRE-CREATION > > samza-hdfs/src/test/java/org/apache/samza/system/hdfs/TestPartitionDesctiptionUtil.java > PRE-CREATION > > samza-hdfs/src/test/java/org/apache/samza/system/hdfs/partitioner/TestDirectoryPartitioner.java > PRE-CREATION > > samza-hdfs/src/test/java/org/apache/samza/system/hdfs/partitioner/TestHdfsFileSystemAdapter.java > PRE-CREATION > > samza-hdfs/src/test/java/org/apache/samza/system/hdfs/reader/TestAvroFileHdfsReader.java > PRE-CREATION > > samza-hdfs/src/test/java/org/apache/samza/system/hdfs/reader/TestMultiFileHdfsReader.java > PRE-CREATION > samza-hdfs/src/test/resources/integTest/emptyTestFile PRE-CREATION > samza-hdfs/src/test/resources/partitioner/testfile01 PRE-CREATION > samza-hdfs/src/test/resources/partitioner/testfile02 PRE-CREATION > samza-hdfs/src/test/resources/reader/TestEvent.avsc PRE-CREATION > > samza-hdfs/src/test/scala/org/apache/samza/system/hdfs/TestHdfsSystemProducerTestSuite.scala > 261310d03de204718621f601117f016da14841df > samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnJobFactory.scala > 4e328a5f8c2b496a71e36c106339b7af263c96c7 > > Diff: https://reviews.apache.org/r/51142/diff/ > > > Testing > ------- > > unit tests pass. > > manually tested by writing a real hdfs samza job and deploying to a yarn > cluster. > > > Thanks, > > Hai Lu > >