Hi Tim - I understand this approach. However, in this case, different data flows through each partition of operator B which is not my intent. My intent is to have two independent pipelines operating on the exact same set of data.
Each pipeline may itself be partitioned (for scalability). The reason for this is that if an operation fails in one pipeline, the piece of data that was being processed is still being processed by the other pipeline and there is no delay in processing. Otherwise, the data is not lost, but we must wait for the STRAM to restart the failed operator and resume processing from the upstream BufferServer. Please let me know if that makes sense. On 9/23/15, 11:31 AM, "Timothy Farkas" <[email protected]> wrote: >Hi Ilya, > >If you want multiple parallel pipelines with the same operators and logic, >you can enable parallel partitioning in your properties file. > > > <property> > <name>dt.operator.B.port.input.attr.PARTITION_PARALLEL</name> > <value>true</value> > </property> > >This will enable parallel partitioning for operator B. What this means is >that if there are 8 partitions of A there will be 8 partitions of B, each >partition of B is connected to exactly 1 upstream A operator (so this is >not N x N partitioning). Furthermore if A gets dynamically repartitioned >to >4 operators, B will get repartitioned to 4 operators as well and all its >downstream operators will get repartitioned appropriately. > >Thanks, >Tim > >On Wed, Sep 23, 2015 at 7:06 AM, Ganelin, Ilya ><[email protected]> >wrote: > >> Hi all. Say that I have a DAG (A -> B -> C) and, given any DAG, I want >>to >> create a new DAG that is (A -> B -> C, A -> B¹ -> C¹) >> >> At the moment, Apex does not support the following operation: >> >> dag.addStream(³NAME², A.output, B.input) >> dag.addStream(³NAME2², A.output, B¹.input) >> >> It throws the following error: >> java.lang.IllegalArgumentException: Operator A already connected to NAME >> >> One has to do : >> dag.addStream(³name², A.output, B.input, B¹.input). >> >> Is there a concrete reason that the first approach is not supported? Is >> there any way to enable such an approach to allow iterative building of >> graphs? This would allow me to create a utility that automatically >> generates independent parallel pipelines given the same point of origin >> which greatly facilitates low-latency fault handling. >> >> Is there a reason an operator may not be connected to multiple operators >> after initialization? >> >> Appreciate any input, thanks! >> ________________________________________________________ >> >> The information contained in this e-mail is confidential and/or >> proprietary to Capital One and/or its affiliates and may only be used >> solely in performance of work or services for Capital One. The >>information >> transmitted herewith is intended only for use by the individual or >>entity >> to which it is addressed. If the reader of this message is not the >>intended >> recipient, you are hereby notified that any review, retransmission, >> dissemination, distribution, copying or other use of, or taking of any >> action in reliance upon this information is strictly prohibited. If you >> have received this communication in error, please contact the sender and >> delete the material from your computer. >> ________________________________________________________ The information contained in this e-mail is confidential and/or proprietary to Capital One and/or its affiliates and may only be used solely in performance of work or services for Capital One. The information transmitted herewith is intended only for use by the individual or entity to which it is addressed. If the reader of this message is not the intended recipient, you are hereby notified that any review, retransmission, dissemination, distribution, copying or other use of, or taking of any action in reliance upon this information is strictly prohibited. If you have received this communication in error, please contact the sender and delete the material from your computer. ________________________________________________________ The information contained in this e-mail is confidential and/or proprietary to Capital One and/or its affiliates and may only be used solely in performance of work or services for Capital One. The information transmitted herewith is intended only for use by the individual or entity to which it is addressed. If the reader of this message is not the intended recipient, you are hereby notified that any review, retransmission, dissemination, distribution, copying or other use of, or taking of any action in reliance upon this information is strictly prohibited. If you have received this communication in error, please contact the sender and delete the material from your computer.
