> On July 15, 2016, 12:12 a.m., Yi Pan (Data Infrastructure) wrote: > > samza-core/src/main/java/org/apache/samza/configbuilder/ConfigBuilder.java, > > line 58 > > <https://reviews.apache.org/r/48356/diff/8/?file=1441886#file1441886line58> > > > > Here it seems like that you are mandating any config should have > > jobName and taskClass. However, I believe those two are not the only > > mandatory configuration variables. I would suggest that we remove these > > from the constructor of ConfigBuilder. Instead, make it more modular s.t. > > we can call: ConfigBuilder.getGenericConfigBuilder().jobConfig(jobFactory, > > jobName, jobId, coordinatorSystem).taskConfig(taskClass, taskInputs, ...) > > > > Then, finally, we can call build() to compile the complete > > configuration, in which we can call validate() for each modular config > > class (e.g. JobConfig.validate()) and also validate the inter-dependencies > > between the modular config classes (e.g. systems defined in JobConfig, > > StoreConfig, etc. must have a corresponding SystemConfig). > > > > P.S. just saw Chris' comment earlier. I think that if we can separate > > the ConfigBuilder into smaller modular sections, we can make the mandatory > > config variable in constructors. > > Navina Ramesh wrote: > Ok. I see the model you are proposing. > > 1. It is pretty ideal for how our configs are structured today. However, > it ends up being less intuitive user API. > > It seems more natural for the user to say: > ConfigBuilder.getGenericBuilder().addTaskInput(systemconfigs, serdeconfigs) , > instead of addTaskConfig, as it usually ends being more than just about the > input. > > 2. It is hard to tell which sub-configs are required or not, unless they > are mandated in the constructor. For example, in > ConfigBuilder.getGenericConfigBuilder().jobConfig(jobFactory, jobName, jobId, > coordinatorSystem), jobId is optional. That means, we should provide > overloaded constructors. If there are too many configurations within a > namespace, we may end-up creating "sub-builders". I didn't want over-do this > builder pattern without solid motivations. :) > > To your point on having a validate() method, there isn't a lot of > validation we can do in most of our configs (at modular level), except for > type and value range. Most validations seem inter-dependent. That's how I > ended up with a very monolithic validation in build() method of ConfigBuidler.
I agree w/ that the builder APIs have to be user-friendly. But it should be structed w/ sub-modules in implementation. Let's talk in person tomorrow. - Yi ----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/48356/#review142183 ----------------------------------------------------------- On Aug. 11, 2016, 1:23 a.m., Navina Ramesh wrote: > > ----------------------------------------------------------- > This is an automatically generated e-mail. To reply, visit: > https://reviews.apache.org/r/48356/ > ----------------------------------------------------------- > > (Updated Aug. 11, 2016, 1:23 a.m.) > > > Review request for samza, Chris Pettitt and Yi Pan (Data Infrastructure). > > > Repository: samza > > > Description > ------- > > Added ConfigBuilder and support classes > > Added JobCoordinator interfaces > > > Adding StreamProcessor, StandaloneJobCoordinator and updating SamzaContainer > interface > > > Added TestStreamProcessor and some unit tests for ConfigBuilders > > > Changing who defined processorId > > > Fixed checkstyle errors > > > Replaced SamzaException with ConfigException > > > Removing localityManager instantiation from Samza Container > > > Diffs > ----- > > build.gradle 1d4eb74b1294318db8454631ddd0901596121ab2 > checkstyle/import-control.xml 7e77702bcd5c32f7fdaf1558337505993c1abe06 > samza-core/src/main/java/org/apache/samza/config/TaskConfigJava.java > 021d42a70179f5d14f51ac87cb09dcc97218095e > > samza-core/src/main/java/org/apache/samza/configbuilder/CheckpointConfig.java > PRE-CREATION > samza-core/src/main/java/org/apache/samza/configbuilder/ConfigBuilder.java > PRE-CREATION > > samza-core/src/main/java/org/apache/samza/configbuilder/GenericConfigBuilder.java > PRE-CREATION > > samza-core/src/main/java/org/apache/samza/configbuilder/KafkaCheckpointConfig.java > PRE-CREATION > > samza-core/src/main/java/org/apache/samza/configbuilder/KafkaSystemConfig.java > PRE-CREATION > samza-core/src/main/java/org/apache/samza/configbuilder/SerdeConfig.java > PRE-CREATION > > samza-core/src/main/java/org/apache/samza/configbuilder/StandaloneConfigBuilder.java > PRE-CREATION > samza-core/src/main/java/org/apache/samza/configbuilder/SystemConfig.java > PRE-CREATION > > samza-core/src/main/java/org/apache/samza/container/grouper/stream/AllSspToSingleTaskGrouper.java > PRE-CREATION > > samza-core/src/main/java/org/apache/samza/container/grouper/stream/AllSspToSingleTaskGrouperFactory.java > PRE-CREATION > > samza-core/src/main/java/org/apache/samza/container/grouper/task/SingleContainerGrouper.java > PRE-CREATION > > samza-core/src/main/java/org/apache/samza/container/grouper/task/SingleContainerGrouperFactory.java > PRE-CREATION > samza-core/src/main/java/org/apache/samza/coordinator/JobCoordinator.java > PRE-CREATION > > samza-core/src/main/java/org/apache/samza/coordinator/JobCoordinatorFactory.java > PRE-CREATION > samza-core/src/main/java/org/apache/samza/processor/StreamProcessor.java > PRE-CREATION > > samza-core/src/main/java/org/apache/samza/standalone/StandaloneJobCoordinator.java > PRE-CREATION > > samza-core/src/main/java/org/apache/samza/standalone/StandaloneJobCoordinatorFactory.java > PRE-CREATION > samza-core/src/main/scala/org/apache/samza/config/TaskConfig.scala > 90c1904772f2814eaa6e36ebd35c273f2c6c9217 > samza-core/src/main/scala/org/apache/samza/container/RunLoop.scala > 538ebb8c34c351ef044d187857b688cc3c02a1db > samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala > f786fc08c8f7eced4f4084dc8326b288888b6422 > samza-core/src/main/scala/org/apache/samza/coordinator/JobCoordinator.scala > ba38b5cfa4e61b5513ce38dd2be32438b62cd7ce > samza-core/src/main/scala/org/apache/samza/job/local/ThreadJobFactory.scala > 56881d46be9f859999adabbbda20433b208e012e > > samza-core/src/test/java/org/apache/samza/configbuilder/TestStandaloneConfigBuilder.java > PRE-CREATION > > samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemFactory.scala > b574176107e8faf47a15fb1eb591dc79c3c9d896 > samza-test/src/test/java/org/apache/samza/processor/MyStreamTask.java > PRE-CREATION > > samza-test/src/test/java/org/apache/samza/processor/TestStreamProcessor.java > PRE-CREATION > samza-yarn/src/main/java/org/apache/samza/config/YarnConfig.java > 8f2dc4853a2b5dd712f25a2d2d16402bcba89d7a > > Diff: https://reviews.apache.org/r/48356/diff/ > > > Testing > ------- > > ./gradlew clean build > > Local integration test: > ./bin/grid start zookeeper > ./bin/grid start kafka > Then, run TestStreamProcessor.java > > > Thanks, > > Navina Ramesh > >