On Wed, Dec 21, 2016 at 10:58 AM, Shen Li <cs.she...@gmail.com> wrote: > Hi Kenn, > > Thanks a lot for the information. > > Sure, below are more details about the problem I encountered. > > I am developing a runner for IBM Streams, and am exploring possible ways to > conduct integration tests. As Streams is not an open-source project, we > cannot add the full set of libraries to Maven Central repo. Nor can we > guarantee to provide a server (with Streams installed) as a long-term > Jenkins slave. So, it seems more flexible to let the runner submit the > graph to a Streams cloud service, and provide the account info through > "-DrunnableOnServicePipelineOptions" (please correct me if it does not work > in this way). The problem is that the runner cannot convert the Pipeline > into a Streams graph format without a local Streams install. So, I am > thinking about sending the serialized Pipeline to the Cloud service for > execution. Maybe I should create some intermediate format between the > Pipeline and Streams graph format. Or, is there any other way to carry out > the integration test without a Streams install?
Choosing an intermediate representation that can be serialized and sent to a cloud service (where it is then translated into the actual implementation representation) is a fine solution. In fact that's what Dataflow itself does. Of course we'll want to move as close to (2) as possible once it exists. > On Wed, Dec 21, 2016 at 12:08 PM, Kenneth Knowles <k...@google.com.invalid> > wrote: > >> Hi Shen, >> >> I want to tell you (1) how things work today and (2) how we want them to be >> eventually. >> >> (1) So far, each runner translates the Pipeline to their own graph format >> before serialization, so we have not yet encountered this issue. >> >> (2) We intend to make a standard mostly-readable JSON format for a >> Pipeline. It is based the Avro schema sketched in the design doc at >> https://s.apache.org/beam-runner-api and there is also a draft JSON schema >> at https://github.com/apache/incubator-beam/pull/662. >> >> You may wish to follow https://issues.apache.org/jira/browse/BEAM-115, >> though that is a very general ticket. >> >> Can you share any more details? >> >> Kenn >> >> On Wed, Dec 21, 2016 at 8:47 AM, Shen Li <cs.she...@gmail.com> wrote: >> >> > Hi, >> > >> > What are the recommended ways to serialize/deserialize a Pipeline >> object? I >> > need to submit a pipeline object to cloud for execution and fetch the >> > result. >> > >> > Thanks, >> > >> > Shen >> > >>