[GitHub] samza pull request #11: Fixed typo in run-in-multi-node-yarn.md

2016-09-27 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/samza/pull/11


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


Re: Review Request 51142: SAMZA-967: HDFS System Consumer

2016-09-27 Thread Navina Ramesh

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/51142/#review150648
---


Fix it, then Ship it!




lgtm +1 .. I think you were planning to add documentation with a separate 
JIRA/RB . Correct?


build.gradle (line 308)


why is this dependency needed here? It seems like this compile dependency 
is required for samza-hdfs and now samza-hdfs depends on samza-yarn. Is there a 
better way to do this?



samza-hdfs/src/main/java/org/apache/samza/system/hdfs/partitioner/DirectoryPartitioner.java
 (line 53)


It is important to document the assumption that we consider the HDFS file 
set to be immutable and how we handle inconsistencies. Looks like you validate 
and throw exception in validateAndGetOriginalFilteredFiles. 
Sorry about nagging regarding documentation. This feels like a complicated 
class, where we may easily forget our design assumptions. Better to clarify it 
in the doc.



samza-hdfs/src/test/scala/org/apache/samza/system/hdfs/TestHdfsSystemProducerTestSuite.scala
 (line 42)


Not related to you change. But can you clean up some unused imports before 
you commit this file? Thanks!



samza-shell/src/main/bash/bash-run-job.sh (line 27)


If we want to use this only for the azkaban runner, we should perhaps 
rename the file as run-job-for-azkaban.sh or something on those lines.


- Navina Ramesh


On Sept. 20, 2016, 11:22 p.m., Hai Lu wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/51142/
> ---
> 
> (Updated Sept. 20, 2016, 11:22 p.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   |  |   
> |   
>|  

Re: Review Request 51142: SAMZA-967: HDFS System Consumer

2016-09-27 Thread Navina Ramesh


> 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
> > 
> >
> > 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.
> 
> Hai Lu wrote:
> 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
> 
> Yi Pan (Data Infrastructure) wrote:
> Great! I am really curious what are the set of compatible APIs! So, I 
> guess that we just enforce avro-1.4 when compiling samza-hdfs module? I 
> remember that I tried last time and got a build failure in samza-hdfs w/ 
> AvroDataFileHdfsWriter in samza-li build. I am curious how you made it work.
> 
> Navina Ramesh wrote:
> Right now, we exclude samza-hdfs build in samza-li. 
>   "build": "ligradle -PscalaVersion=2.10 -Prelease=true 
> -PallArtifacts build -x:samza-hdfs_2.10:build",
>   
> We may want to fully understand the avro changes introduced by 
> HdfsProducer and/or HdfsConsumer in samza-li. This sounds like a blocker for 
> me right now. How are we going to overcome avro conflict introduced in 
> HdfsSystemProducer?
> 
> Hai Lu wrote:
> I know. I included it back in samza-li and it worked just fine. Just need 
> some extra dependency to make the tests pass. I have been only using li_trunk 
> to deploy to Hadoop's YARN at LinkedIn

Got it. Thanks!


- Navina


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/51142/#review148629
---


On Sept. 20, 2016, 11:22 p.m., Hai Lu wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/51142/
> ---
> 
> (Updated Sept. 20, 2016, 11:22 p.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/| 
>|  | |  |  +---+
> 

Re: Review Request 51142: SAMZA-967: HDFS System Consumer

2016-09-27 Thread Navina Ramesh


> On Sept. 13, 2016, 12:33 a.m., Yi Pan (Data Infrastructure) wrote:
> > samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemConsumer.java,
> >  line 101
> > 
> >
> > Better, to avoid the wasteful remote IO if there are multiple calls to 
> > getPartitionDescriptor from multiple threads, is to use bucketized locks on 
> > the ConcurrentHashMap entries to ensure synchronization in populating a 
> > certain hash map entry. Guava cache implemented the bucketized locking as a 
> > built-in method already: 
> > http://www.tutorialspoint.com/guava/guava_caching_utilities.htm
> 
> Navina Ramesh wrote:
> What was the resolution here? Was there any change to the IO pattern to 
> use caching?
> 
> Hai Lu wrote:
> I believe this is just to optimize the situation that multiple calls 
> happen at the same time and causing everyone making remote calls. After the 
> change here, only the first one will actually make the remote call while 
> everyone else be blocked.
> 
> It's a very very tiny improvement, to be honest.

Ah.Got it.. Actually I was looking for the change in the diff window and I 
couldn't figure out where you have used. I understood once I applied your patch 
in my IDE. Thanks!

Yes . It is tiny but important improvement.


- Navina


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/51142/#review148612
---


On Sept. 20, 2016, 11:22 p.m., Hai Lu wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/51142/
> ---
> 
> (Updated Sept. 20, 2016, 11:22 p.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  |  |   
> |   
>|   

Re: Review Request 51142: SAMZA-967: HDFS System Consumer

2016-09-27 Thread Navina Ramesh


> On Sept. 14, 2016, 6:19 a.m., Yi Pan (Data Infrastructure) wrote:
> > samza-hdfs/src/main/java/org/apache/samza/system/hdfs/reader/MultiFileHdfsReader.java,
> >  line 59
> > 
> >
> > Not sure what are we doing here? What's the ordering that we are 
> > enforcing in this multi-file partition? I saw that you are trying to make 
> > the offsets as an offset vector on top of all files in the same partition. 
> > Why? Can we simplify it by making it full-ordered in the same partition 
> > instead of partial-ordered via an offset vector?

I couldn't figure out the resolution for this issue. Perhaps you discussed 
offline. Can you please update the discussion here for everyone's benefit?


> On Sept. 14, 2016, 6:19 a.m., Yi Pan (Data Infrastructure) wrote:
> > samza-hdfs/src/main/scala/org/apache/samza/system/hdfs/HdfsConfig.scala, 
> > line 66
> > 
> >
> > It would be nicer to make it conforming to Offspring style of config 
> > variable scoping. i.e. if the scope of configuration is for hdfs consumer, 
> > use systems.%s.consumer.hdfs.buffer-capacity. I would suggest to consult 
> > Prateek since he has been working on the Offspring config refactoring. For 
> > new config variables, "." should strictly be used as deliminator between 
> > scopes, not as deliminator between words.

Going by the logic of using period to delimit scopes, shouldn't it be 
systems.%s.consumer.hdfs-buffer-capacity? Unless there is a hdfs scope that I 
am not seeing. It is kind of weird because we assume the indirection from 
systemname (%s) to its factory will act as a scope. I am not sure what the 
correct pattern should be.


- Navina


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/51142/#review148780
---


On Sept. 20, 2016, 11:22 p.m., Hai Lu wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/51142/
> ---
> 
> (Updated Sept. 20, 2016, 11:22 p.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/| 
>|  | |  |  +---+
> 

Re: Review Request 51142: SAMZA-967: HDFS System Consumer

2016-09-27 Thread Navina Ramesh

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/51142/#review150637
---




samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemConsumer.java 
(line 56)


Can you please add javadoc related to thread-safety of the class?



samza-hdfs/src/main/scala/org/apache/samza/system/hdfs/HdfsConfig.scala (line 
70)


what is the "default-partitioner"? Is it possible to have more than one 
partitioner?



samza-hdfs/src/main/scala/org/apache/samza/system/hdfs/HdfsSystemFactory.scala 
(line 37)


Doesn't this add a dependency between samza-hdfs and samza-kafka?

It seems to have been introduced by the HdfsSystemProducer. Can we please 
fix it forward? Or as Yi suggested, please create a JIRA and add a TODO comment 
here referring to the JIRA

Thanks!


- Navina Ramesh


On Sept. 20, 2016, 11:22 p.m., Hai Lu wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/51142/
> ---
> 
> (Updated Sept. 20, 2016, 11:22 p.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 |  |   |  
>| | |

Re: Review Request 51142: SAMZA-967: HDFS System Consumer

2016-09-27 Thread Hai Lu


> 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
> > 
> >
> > 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.
> 
> Hai Lu wrote:
> 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
> 
> Yi Pan (Data Infrastructure) wrote:
> Great! I am really curious what are the set of compatible APIs! So, I 
> guess that we just enforce avro-1.4 when compiling samza-hdfs module? I 
> remember that I tried last time and got a build failure in samza-hdfs w/ 
> AvroDataFileHdfsWriter in samza-li build. I am curious how you made it work.
> 
> Navina Ramesh wrote:
> Right now, we exclude samza-hdfs build in samza-li. 
>   "build": "ligradle -PscalaVersion=2.10 -Prelease=true 
> -PallArtifacts build -x:samza-hdfs_2.10:build",
>   
> We may want to fully understand the avro changes introduced by 
> HdfsProducer and/or HdfsConsumer in samza-li. This sounds like a blocker for 
> me right now. How are we going to overcome avro conflict introduced in 
> HdfsSystemProducer?

I know. I included it back in samza-li and it worked just fine. Just need some 
extra dependency to make the tests pass. I have been only using li_trunk to 
deploy to Hadoop's YARN at LinkedIn


- Hai


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/51142/#review148629
---


On Sept. 20, 2016, 11:22 p.m., Hai Lu wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/51142/
> ---
> 
> (Updated Sept. 20, 2016, 11:22 p.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 +-+   
>

Re: Review Request 51142: SAMZA-967: HDFS System Consumer

2016-09-27 Thread Hai Lu


> On Sept. 13, 2016, 12:33 a.m., Yi Pan (Data Infrastructure) wrote:
> > samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemConsumer.java,
> >  line 101
> > 
> >
> > Better, to avoid the wasteful remote IO if there are multiple calls to 
> > getPartitionDescriptor from multiple threads, is to use bucketized locks on 
> > the ConcurrentHashMap entries to ensure synchronization in populating a 
> > certain hash map entry. Guava cache implemented the bucketized locking as a 
> > built-in method already: 
> > http://www.tutorialspoint.com/guava/guava_caching_utilities.htm
> 
> Navina Ramesh wrote:
> What was the resolution here? Was there any change to the IO pattern to 
> use caching?

I believe this is just to optimize the situation that multiple calls happen at 
the same time and causing everyone making remote calls. After the change here, 
only the first one will actually make the remote call while everyone else be 
blocked.

It's a very very tiny improvement, to be honest.


- Hai


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/51142/#review148612
---


On Sept. 20, 2016, 11:22 p.m., Hai Lu wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/51142/
> ---
> 
> (Updated Sept. 20, 2016, 11:22 p.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--+ +--+--+
>||  |   |  
>| | |
>|  

Re: Review Request 51142: SAMZA-967: HDFS System Consumer

2016-09-27 Thread Navina Ramesh


> On Sept. 13, 2016, 12:33 a.m., Yi Pan (Data Infrastructure) wrote:
> > samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemConsumer.java,
> >  line 101
> > 
> >
> > Better, to avoid the wasteful remote IO if there are multiple calls to 
> > getPartitionDescriptor from multiple threads, is to use bucketized locks on 
> > the ConcurrentHashMap entries to ensure synchronization in populating a 
> > certain hash map entry. Guava cache implemented the bucketized locking as a 
> > built-in method already: 
> > http://www.tutorialspoint.com/guava/guava_caching_utilities.htm

What was the resolution here? Was there any change to the IO pattern to use 
caching?


- Navina


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/51142/#review148612
---


On Sept. 20, 2016, 11:22 p.m., Hai Lu wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/51142/
> ---
> 
> (Updated Sept. 20, 2016, 11:22 p.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 |  |   |  
>| | |
>|  | |  |   
> +--+--+ 

Re: Review Request 51142: SAMZA-967: HDFS System Consumer

2016-09-27 Thread Navina Ramesh


> 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
> > 
> >
> > 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.
> 
> Hai Lu wrote:
> 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
> 
> Yi Pan (Data Infrastructure) wrote:
> Great! I am really curious what are the set of compatible APIs! So, I 
> guess that we just enforce avro-1.4 when compiling samza-hdfs module? I 
> remember that I tried last time and got a build failure in samza-hdfs w/ 
> AvroDataFileHdfsWriter in samza-li build. I am curious how you made it work.

Right now, we exclude samza-hdfs build in samza-li. 
  "build": "ligradle -PscalaVersion=2.10 -Prelease=true -PallArtifacts 
build -x:samza-hdfs_2.10:build",
  
We may want to fully understand the avro changes introduced by HdfsProducer 
and/or HdfsConsumer in samza-li. This sounds like a blocker for me right now. 
How are we going to overcome avro conflict introduced in HdfsSystemProducer?


- Navina


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/51142/#review148629
---


On Sept. 20, 2016, 11:22 p.m., Hai Lu wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/51142/
> ---
> 
> (Updated Sept. 20, 2016, 11:22 p.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   |  |   
> |   
> 

Re: Review Request 51703: SAMZA-1024 : Enable passing of Configs and MetricsRegistry into Monitor objects.

2016-09-27 Thread Jake Maes

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/51703/#review150600
---


Ship it!




Ship It!

- Jake Maes


On Sept. 27, 2016, 6:57 p.m., Shanthoosh Venkataraman wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/51703/
> ---
> 
> (Updated Sept. 27, 2016, 6:57 p.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Jake Maes, Navina Ramesh, Jagadish 
> Venkatraman, Xinyu Liu, and Yi Pan (Data Infrastructure).
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> This patch aims at adding the following functionalities to Samza-Rest 
> monitors.
> 
> * Schedule different monitors at different intervals of time.
> * Define custom monitor configurations and pass config along to the monitor 
> objects.
> JIRA ticket : https://issues.apache.org/jira/browse/SAMZA-1024
> 
> 
> Diffs
> -
> 
>   docs/learn/documentation/versioned/rest/monitors.md 
> 9833068d9f542b80bb438db156a10c85d4d53097 
>   docs/learn/documentation/versioned/rest/overview.md 
> 5b958accf4e1a3f05b949e9dc6e2e19a453523ca 
>   samza-rest/src/main/java/org/apache/samza/monitor/Monitor.java 
> d69df5f73dbf2c494183f9dcc8061c20878742aa 
>   samza-rest/src/main/java/org/apache/samza/monitor/MonitorConfig.java 
> PRE-CREATION 
>   samza-rest/src/main/java/org/apache/samza/monitor/MonitorFactory.java 
> PRE-CREATION 
>   samza-rest/src/main/java/org/apache/samza/monitor/MonitorLoader.java 
> 502ecc49f32b4563e8ceb4ddfa6bc2542c60819e 
>   samza-rest/src/main/java/org/apache/samza/monitor/SamzaMonitorService.java 
> 2f4d9ddb76369c5e83d39152d492807dfb164981 
>   samza-rest/src/main/java/org/apache/samza/monitor/SchedulingProvider.java 
> aea1a9291e651660c798cabf59fcf0c0623bcbd0 
>   samza-rest/src/main/java/org/apache/samza/rest/SamzaRestConfig.java 
> 6f5c10ac89523626c7f7e05558422daad2ccd4e8 
>   samza-rest/src/main/java/org/apache/samza/rest/SamzaRestService.java 
> 5b34da814985fb09281f36c677a97372cacc7a75 
>   samza-rest/src/test/java/org/apache/samza/monitor/TestMonitorService.java 
> 1da343012b85f96f837e3cbf9a54ced3b29fede6 
>   samza-rest/src/test/java/org/apache/samza/monitor/mock/DummyMonitor.java 
> 8621db1b0e8ce3279cc8a5cb3a21bd137d442034 
>   
> samza-rest/src/test/java/org/apache/samza/monitor/mock/DummyMonitorFactory.java
>  PRE-CREATION 
>   
> samza-rest/src/test/java/org/apache/samza/monitor/mock/ExceptionThrowingMonitor.java
>  c4f3f735f78d56f8bb3ef203a05e2bec92489767 
>   
> samza-rest/src/test/java/org/apache/samza/monitor/mock/ExceptionThrowingMonitorFactory.java
>  PRE-CREATION 
> 
> Diff: https://reviews.apache.org/r/51703/diff/
> 
> 
> Testing
> ---
> 
> Unit tests are used to verify the intended functionality.
> 
> 
> Thanks,
> 
> Shanthoosh Venkataraman
> 
>



Re: Review Request 51703: SAMZA-1024 : Enable passing of Configs and MetricsRegistry into Monitor objects.

2016-09-27 Thread Shanthoosh Venkataraman

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/51703/
---

(Updated Sept. 27, 2016, 6:57 p.m.)


Review request for samza, Boris Shkolnik, Jake Maes, Navina Ramesh, Jagadish 
Venkatraman, Xinyu Liu, and Yi Pan (Data Infrastructure).


Repository: samza


Description
---

This patch aims at adding the following functionalities to Samza-Rest monitors.

* Schedule different monitors at different intervals of time.
* Define custom monitor configurations and pass config along to the monitor 
objects.
JIRA ticket : https://issues.apache.org/jira/browse/SAMZA-1024


Diffs (updated)
-

  docs/learn/documentation/versioned/rest/monitors.md 
9833068d9f542b80bb438db156a10c85d4d53097 
  docs/learn/documentation/versioned/rest/overview.md 
5b958accf4e1a3f05b949e9dc6e2e19a453523ca 
  samza-rest/src/main/java/org/apache/samza/monitor/Monitor.java 
d69df5f73dbf2c494183f9dcc8061c20878742aa 
  samza-rest/src/main/java/org/apache/samza/monitor/MonitorConfig.java 
PRE-CREATION 
  samza-rest/src/main/java/org/apache/samza/monitor/MonitorFactory.java 
PRE-CREATION 
  samza-rest/src/main/java/org/apache/samza/monitor/MonitorLoader.java 
502ecc49f32b4563e8ceb4ddfa6bc2542c60819e 
  samza-rest/src/main/java/org/apache/samza/monitor/SamzaMonitorService.java 
2f4d9ddb76369c5e83d39152d492807dfb164981 
  samza-rest/src/main/java/org/apache/samza/monitor/SchedulingProvider.java 
aea1a9291e651660c798cabf59fcf0c0623bcbd0 
  samza-rest/src/main/java/org/apache/samza/rest/SamzaRestConfig.java 
6f5c10ac89523626c7f7e05558422daad2ccd4e8 
  samza-rest/src/main/java/org/apache/samza/rest/SamzaRestService.java 
5b34da814985fb09281f36c677a97372cacc7a75 
  samza-rest/src/test/java/org/apache/samza/monitor/TestMonitorService.java 
1da343012b85f96f837e3cbf9a54ced3b29fede6 
  samza-rest/src/test/java/org/apache/samza/monitor/mock/DummyMonitor.java 
8621db1b0e8ce3279cc8a5cb3a21bd137d442034 
  
samza-rest/src/test/java/org/apache/samza/monitor/mock/DummyMonitorFactory.java 
PRE-CREATION 
  
samza-rest/src/test/java/org/apache/samza/monitor/mock/ExceptionThrowingMonitor.java
 c4f3f735f78d56f8bb3ef203a05e2bec92489767 
  
samza-rest/src/test/java/org/apache/samza/monitor/mock/ExceptionThrowingMonitorFactory.java
 PRE-CREATION 

Diff: https://reviews.apache.org/r/51703/diff/


Testing
---

Unit tests are used to verify the intended functionality.


Thanks,

Shanthoosh Venkataraman