Beam Schemas: current status

2018-08-28 Thread Reuven Lax
I wanted to send a quick note to the community about the current status of
schema-aware PCollections in Beam. As some might remember we had a good
discussion last year about the design of these schemas, involving many
folks from different parts of the community. I sent a summary earlier this
year explaining how schemas has been integrated into the DoFn framework.
Much has happened since then, and here are some of the highlights.

First, I want to emphasize that all the schema-aware classes are currently
marked @Experimental. Nothing is set in stone yet, so if you have questions
about any decisions made, please start a discussion!

SQL

The first big milestone for schemas was porting all of BeamSQL to use the
framework, which was done in pr/5956. This was a lot of work, exposed many
bugs in the schema implementation, but now provides great evidence that
schemas work!

Schema inference

Beam can automatically infer schemas from Java POJOs (objects with public
fields) or JavaBean objects (objects with getter/setter methods). Often you
can do this by simply annotating the class. For example:

@DefaultSchema(JavaFieldSchema.class)

public class UserEvent {

 public String userId;

 public LatLong location;

 Public String countryCode;

 public long transactionCost;

 public double transactionDuration;

 public List traceMessages;

};

@DefaultSchema(JavaFieldSchema.class)

public class LatLong {

 public double latitude;

 public double longitude;

}

Beam will automatically infer schemas for these classes! So if you have a
PCollection, it will automatically get the following schema:

UserEvent:

 userId: STRING

 location: ROW(LatLong)

 countryCode: STRING

 transactionCost: INT64

 transactionDuration: DOUBLE

 traceMessages: ARRAY[STRING]]


LatLong:

 latitude: DOUBLE

 longitude: DOUBLE

Now it’s not always possible to annotate the class like this (you may not
own the class definition), so you can also explicitly register this using
Pipeline:getSchemaRegistry:registerPOJO, and the same for JavaBeans.

Coders

Beam has a built-in coder for any schema-aware PCollection, largely
removing the need for users to care about coders. We generate low-level
bytecode (using ByteBuddy) to implement the coder for each schema, so these
coders are quite performant. This provides a better default coder for Java
POJO objects as well. In the past users were recommended to use AvroCoder
for pojos, which many have found inefficient. Now there’s a more-efficient
solution.

Utility Transforms

Schemas are already useful for implementers of extensions such as SQL, but
the goal was to use them to make Beam itself easier to use. To this end,
I’ve been implementing a library of transforms that allow for easy
manipulation of schema PCollections. So far Filter and Select are merged,
Group is about to go out for review (it needs some more javadoc and unit
tests), and Join is being developed but doesn’t yet have a final interface.

Filter

Given a PCollection, I want to keep only those in an area of
southern manhattan. Well this is easy!

PCollection manhattanEvents = allEvents.apply(Filter

 .whereFieldName("latitude", lat -> lat < 40.720 && lat > 40.699)

 .whereFieldName("longitude", long -> long < -73.969 && long > -74.747));

Schemas along with lambdas allows us to write this transform declaratively.
The Filter transform also allows you to register filter functions that
operate on multiple fields at the same time.

Select

Let’s say that I don’t need all the fields in a row. For instance, I’m only
interested in the userId and traceMessages, and don’t care about the
location. In that case I can write the following:

PCollection selected = allEvents.apply(Select.fieldNames(“userId”, “
traceMessages”));


BTW, Beam also keeps track of which fields are accessed by a transform In
the future we can automatically insert Selects in front of subgraphs to
drop fields that are not referenced in that subgraph.

Group

Group is one of the more advanced transforms. In its most basic form, it
provides a convenient way to group by key:

PCollection> byUserAndCountry =

   allEvents.apply(Group.byFieldNames(“userId”, “countryCode”));

Notice how much more concise this is than using GroupByKey directly!

The Group transform really starts to shine however when you start
specifying aggregations. You can aggregate any field (or fields) and build
up an output schema based on these aggregations. For example:

PCollection> aggregated = allEvents.apply(

   Group.byFieldNames(“userId”, “countryCode”)

   .aggregateField("cost", Sum.ofLongs(), "total_cost")

   .aggregateField("cost", Top.largestFn(10), “top_purchases”)

   .aggregateField("transationDuration", ApproximateQuantilesCombineFn.
create(21),

 “durationHistogram”)));

This will individually aggregate the specified fields of the input items
(by user and country), and generate an output schema for these
aggregations. In this case, the output schema will be the following:


Re: Should we allow ValidatesRunner tests to have access to file systems?

2018-08-28 Thread Lukasz Cwik
I also agree about not having external dependencies in validates runner
tests.

One suggestion would have been to use attempted metrics but there is
currently no way to get access to runner metrics from within a DoFn easily
that is runner agnostic. This is likely a place for improvement since:
* cancelling a pipeline from within the pipeline is useful
* starting a new job against the existing runner from in a pipeline is
useful
* accessing attempted metrics to test DoFn's with side effects is useful
for error handling testing

On Mon, Aug 27, 2018 at 12:40 PM Alan Myrvold  wrote:

> I think this should be an integration test if it requires more access than
> the current ValidatesRunner tests.
>
> Although the ValidatesRunner and integration tests are similar, the intent
> is that the validates runner tests are smaller and more like component
> tests, and there have been discusions on fusing the validates runner tests
> into a smaller set of pipelines.
>
> On Mon, Aug 27, 2018 at 11:27 AM Robin Qiu  wrote:
>
>> Hello everyone,
>>
>> I am writing a test [1] for the support of @RequiresStableInput
>> annotation in Java SDK [2]. For the test to work, I need to have a ParDo
>> make some side effect (e.g. writing to a file system). However,
>> ValidatesRunner tests in Beam currently cannot depend on external states
>> (cannot write to file systems). So I am wondering if it is a good idea to
>> allow ValidatesRunner tests to have access to file systems. This way we can
>> create more flexible ValidatesRunner tests.
>>
>> I could make this test a integration test to get access to file systems
>> (e.g. like WordCountIT.java [3]). But functionally I think this test should
>> be a ValidatesRunner test, because it is testing the support of some SDK
>> features on runners.
>>
>> So what do you think? Any suggestions or concerns are appreciated.
>>
>> Best,
>> Robin
>>
>> [1] https://github.com/apache/beam/pull/6220
>> [2]
>> https://docs.google.com/document/d/117yRKbbcEdm3eIKB_26BHOJGmHSZl1YNoF0RqWGtqAM/edit#
>> [3]
>> https://github.com/apache/beam/blob/master/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java
>>
>>


Re: jira search in chrome omnibox

2018-08-28 Thread Valentyn Tymofieiev
Thanks for sharing.

I have also found useful following custom search query for PRs:
https://github.com/apache/beam/pulls?q=is%3Apr%20%s

Sample usage: type 'pr', space, type: 'author:tvalentyn'.

You could also incorporate 'author:' into the query:
https://github.com/apache/beam/pulls?q=is%3Apr%20author%3A

On Tue, Aug 28, 2018 at 4:26 PM Daniel Oliveira 
wrote:

> This seems pretty useful. Thanks Udi!
>
> On Mon, Aug 27, 2018 at 3:54 PM Udi Meiri  wrote:
>
>> In case you want to quickly look up JIRA tickets, e.g., typing 'j',
>> space, 'BEAM-4696'.
>> Search URL:
>> https://issues.apache.org/jira/QuickSearch.jspa?searchString=%s
>>
>>


Re: jira search in chrome omnibox

2018-08-28 Thread Daniel Oliveira
This seems pretty useful. Thanks Udi!

On Mon, Aug 27, 2018 at 3:54 PM Udi Meiri  wrote:

> In case you want to quickly look up JIRA tickets, e.g., typing 'j', space,
> 'BEAM-4696'.
> Search URL:
> https://issues.apache.org/jira/QuickSearch.jspa?searchString=%s
>
>


Re: [DISCUSS] Versioning, Hadoop related dependencies and enterprise users

2018-08-28 Thread Chamikara Jayalath
On Tue, Aug 28, 2018 at 12:05 PM Thomas Weise  wrote:

> I think there is an invalid assumption being made in this discussion,
> which is that most projects comply with semantic versioning. The reality in
> the open source big data space is unfortunately quite different. Ismaël has
> well characterized the situation and HBase isn't an exception. Another
> indicator for the scale of problem is extensive amount of shading used in
> Beam and other projects. It wouldn't be necessary if semver compliance was
> something we can rely on.
>
> Our recent Flink upgrade broke user(s). And we noticed a backward
> incompatible Flink change that affected the portable Flink runner even
> between patches.
>
> Many projects (including Beam) guarantee compatibility only for a subset
> of public API. Sometimes a REST API is not covered, sometimes not strictly
> internal protocols change and so on, all of which can break users, despite
> the public API remaining "compatible". As much as I would love to rely on
> the version number to tell me wether an upgrade is safe or not, that's not
> practically possible.
>
> Furthermore, we need to proceed with caution forcing upgrades on users
> that host the target systems. To stay with the Flink example, moving Beam
> from 1.4 to 1.5 is actually a major change to some, because they now have
> to upgrade their Flink clusters/deployments to be able to use the new
> version of Beam.
>
> Upgrades need to be done with caution and may require extensive
> verification beyond what our automation provides. I think the Spark change
> from 1.x to 2.x and also the JDK 1.8 change were good examples, they
> provided the community a window to provide feedback and influence the
> change.
>

Thanks for the clarification.

Current policy indeed requests caution and explicit checks when upgrading
all dependencies (including minor and patch versions) but language might
have to be updated to emphasize your concerns.

Here's the current text.

"Beam releases adhere to
 semantic
versioning. Hence, community members should take care when updating
dependencies. Minor version updates to dependencies should be backwards
compatible in most cases. Some updates to dependencies though may result in
backwards incompatible API or functionality changes to Beam. PR reviewers
and committers should take care to detect any dependency updates that could
potentially introduce backwards incompatible changes to Beam before merging
and PRs that update dependencies should include a statement regarding this
verification in the form of a PR comment. Dependency updates that result in
backwards incompatible changes to non-experimental features of Beam should
be held till next major version release of Beam. Any exceptions to this
policy should only occur in extreme cases (for example, due to a security
vulnerability of an existing dependency that is only fixed in a subsequent
major version) and should be discussed in the Beam dev list. Note that
backwards incompatible changes to experimental features may be introduced
in a minor version release."

Also, are there any other steps we can take to make sure that Beam
dependencies are not too old while offering a stable system ? Note that
having a lot of legacy dependencies that do not get upgraded regularly can
also result in user pain and Beam being unusable for certain users who run
into dependency conflicts when using Beam along with other systems (which
will increase the amount of shading/vendoring we have to do).

Please note that current tooling does not force upgrades or automatically
upgrade dependencies. It simply creates JIRAs that can be closed with a
reason if needed. For Python SDK though we have version ranges in place for
most dependencies [1] so these dependencies get updated automatically
according to the corresponding ranges.
https://github.com/apache/beam/blob/master/sdks/python/setup.py#L103

Thanks,
Cham


>
> Thanks,
> Thomas
>
>
>
> On Tue, Aug 28, 2018 at 11:29 AM Raghu Angadi  wrote:
>
>> Thanks for the IO versioning summary.
>> KafkaIO's policy of 'let the user decide exact version at runtime' has
>> been quite useful so far. How feasible is that for other connectors?
>>
>> Also, KafkaIO does not limit itself to minimum features available across
>> all the supported versions. Some of the features (e.g. server side
>> timestamps) are disabled based on runtime Kafka version.  The unit tests
>> currently run with single recent version. Integration tests could certainly
>> use multiple versions. With some more effort in writing tests, we could
>> make multiple versions of the unit tests.
>>
>> Raghu.
>>
>> IO versioning
>>> * Elasticsearch. We delayed the move to version 6 until we heard of
>>> more active users needing it (more deployments). We support 2.x and
>>> 5.x (but 2.x went recently EOL). Support for 6.x is in progress.
>>> * SolrIO, stable version is 7.x, LTS is 6.x. We support only 5.x
>>> because most big 

Re: An example of Integration test case

2018-08-28 Thread Rakesh Kumar
Thank you Robin for your quick response.


On Tue, Aug 28, 2018 at 1:25 PM Robin Qiu  wrote:

> Hi Rakesh,
>
> A python integration test example can be found here:
> https://github.com/apache/beam/blob/master/sdks/python/apache_beam/examples/wordcount_it_test.py
>
> Best,
> Robin
>
> On Tue, Aug 28, 2018 at 1:10 PM Rakesh Kumar  wrote:
>
>> Hi,
>>
>> I am writing my streaming application using Python SDK. I also want to
>> write an integration test cases. Do we have any good example of integration
>> test that I can refer?
>>
>> Thank you,
>> Rakesh
>> --
>> Rakesh Kumar
>> Software Engineer
>> 510-761-1364 <(510)%20761-1364> |
>>
>> 
>>
> --
Rakesh Kumar
Software Engineer
510-761-1364 |




Re: An example of Integration test case

2018-08-28 Thread Robin Qiu
Hi Rakesh,

A python integration test example can be found here:
https://github.com/apache/beam/blob/master/sdks/python/apache_beam/examples/wordcount_it_test.py

Best,
Robin

On Tue, Aug 28, 2018 at 1:10 PM Rakesh Kumar  wrote:

> Hi,
>
> I am writing my streaming application using Python SDK. I also want to
> write an integration test cases. Do we have any good example of integration
> test that I can refer?
>
> Thank you,
> Rakesh
> --
> Rakesh Kumar
> Software Engineer
> 510-761-1364 |
>
> 
>


An example of Integration test case

2018-08-28 Thread Rakesh Kumar
Hi,

I am writing my streaming application using Python SDK. I also want to
write an integration test cases. Do we have any good example of integration
test that I can refer?

Thank you,
Rakesh
-- 
Rakesh Kumar
Software Engineer
510-761-1364 |




Re: [DISCUSS] Versioning, Hadoop related dependencies and enterprise users

2018-08-28 Thread Thomas Weise
I think there is an invalid assumption being made in this discussion, which
is that most projects comply with semantic versioning. The reality in the
open source big data space is unfortunately quite different. Ismaël has
well characterized the situation and HBase isn't an exception. Another
indicator for the scale of problem is extensive amount of shading used in
Beam and other projects. It wouldn't be necessary if semver compliance was
something we can rely on.

Our recent Flink upgrade broke user(s). And we noticed a backward
incompatible Flink change that affected the portable Flink runner even
between patches.

Many projects (including Beam) guarantee compatibility only for a subset of
public API. Sometimes a REST API is not covered, sometimes not strictly
internal protocols change and so on, all of which can break users, despite
the public API remaining "compatible". As much as I would love to rely on
the version number to tell me wether an upgrade is safe or not, that's not
practically possible.

Furthermore, we need to proceed with caution forcing upgrades on users that
host the target systems. To stay with the Flink example, moving Beam from
1.4 to 1.5 is actually a major change to some, because they now have to
upgrade their Flink clusters/deployments to be able to use the new version
of Beam.

Upgrades need to be done with caution and may require extensive
verification beyond what our automation provides. I think the Spark change
from 1.x to 2.x and also the JDK 1.8 change were good examples, they
provided the community a window to provide feedback and influence the
change.

Thanks,
Thomas



On Tue, Aug 28, 2018 at 11:29 AM Raghu Angadi  wrote:

> Thanks for the IO versioning summary.
> KafkaIO's policy of 'let the user decide exact version at runtime' has
> been quite useful so far. How feasible is that for other connectors?
>
> Also, KafkaIO does not limit itself to minimum features available across
> all the supported versions. Some of the features (e.g. server side
> timestamps) are disabled based on runtime Kafka version.  The unit tests
> currently run with single recent version. Integration tests could certainly
> use multiple versions. With some more effort in writing tests, we could
> make multiple versions of the unit tests.
>
> Raghu.
>
> IO versioning
>> * Elasticsearch. We delayed the move to version 6 until we heard of
>> more active users needing it (more deployments). We support 2.x and
>> 5.x (but 2.x went recently EOL). Support for 6.x is in progress.
>> * SolrIO, stable version is 7.x, LTS is 6.x. We support only 5.x
>> because most big data distributions still use 5.x (however 5.x has
>> been EOL).
>> * KafkaIO uses version 1.x but Kafka recently moved to 2.x, however
>> most of the deployments of Kafka use earlier versions than 1.x. This
>> module uses a single version with the kafka client as a provided
>> dependency and so far it works (but we don’t have multi version
>> tests).
>>
>
>
> On Tue, Aug 28, 2018 at 8:38 AM Ismaël Mejía  wrote:
>
>> I think we should refine the strategy on dependencies discussed
>> recently. Sorry to come late with this (I did not follow closely the
>> previous discussion), but the current approach is clearly not in line
>> with the industry reality (at least not for IO connectors + Hadoop +
>> Spark/Flink use).
>>
>> A really proactive approach to dependency updates is a good practice
>> for the core dependencies we have e.g. Guava, Bytebuddy, Avro,
>> Protobuf, etc, and of course for the case of cloud based IOs e.g. GCS,
>> Bigquery, AWS S3, etc. However when we talk about self hosted data
>> sources or processing systems this gets more complicated and I think
>> we should be more flexible and do this case by case (and remove these
>> from the auto update email reminder).
>>
>> Some open source projects have at least three maintained versions:
>> - LTS – maps to what most of the people have installed (or the big
>> data distributions use) e.g. HBase 1.1.x, Hadoop 2.6.x
>> - Stable – current recommended version. HBase 1.4.x, Hadoop 2.8.x
>> - Next – latest release. HBase 2.1.x Hadoop 3.1.x
>>
>> Following the most recent versions can be good to be close to the
>> current development of other projects and some of the fixes, but these
>> versions are commonly not deployed for most users and adopting a LTS
>> or stable only approach won't satisfy all cases either. To understand
>> why this is complex let’s see some historical issues:
>>
>> IO versioning
>> * Elasticsearch. We delayed the move to version 6 until we heard of
>> more active users needing it (more deployments). We support 2.x and
>> 5.x (but 2.x went recently EOL). Support for 6.x is in progress.
>> * SolrIO, stable version is 7.x, LTS is 6.x. We support only 5.x
>> because most big data distributions still use 5.x (however 5.x has
>> been EOL).
>> * KafkaIO uses version 1.x but Kafka recently moved to 2.x, however
>> most of the deployments of Kafka use earlier versions than 1.x. 

Re: Design Proposal: Beam-Site Automation Reliability

2018-08-28 Thread Udi Meiri
FYI, we are about to add a new branch to apache/beam, named 'asf-site',
which will contain generated website sources.

On Thu, Jun 7, 2018 at 10:18 AM Jason Kuster  wrote:

> Sounds good; I'm really excited about these changes Scott. Thanks for
> taking this on!
>
> On Tue, Jun 5, 2018 at 4:00 PM Scott Wegner  wrote:
>
>> Thanks everyone; I've responded to feedback in the doc [1] and I believe
>> we've reached consensus. I've added implementation tasks in JIRA
>> under BEAM-4493 [2] and will start coding soon. As a recap, the high-level
>> plan is:
>>
>> * Migrate website source code to the main apache/beam repository
>> * Discontinue checking-in generated HTML during the PR workflow
>> * Align to the existing apache/beam PR process (code review policy,
>> precommits, generic Git merge)
>> * Filter pre-commit jobs to only run when necessary
>> * Add a post-commit Jenkins job to push generated HTML to a separate
>> publishing branch
>>
>> [1] https://s.apache.org/beam-site-automation
>> [2] https://issues.apache.org/jira/browse/BEAM-4493
>>
>> On Fri, Jun 1, 2018 at 10:33 AM Scott Wegner  wrote:
>>
>>> Pre-commit filtering has come up on previous discussions as well and is
>>> an obvious improvement. I've opened BEAM-4445 [1] for this and assigned it
>>> to myself.
>>>
>>> [1] https://issues.apache.org/jira/browse/BEAM-4445
>>>
>>> On Fri, Jun 1, 2018 at 10:01 AM Kenneth Knowles  wrote:
>>>
 +1

 Can we separate precommit filtering and get it set up independent from
 this? I think there's a lot of good directions to go once it is the norm.

 On Thu, May 31, 2018 at 9:25 PM Thomas Weise  wrote:

> Very nice, enthusiastic +1
>
> On Thu, May 31, 2018 at 3:24 PM, Scott Wegner 
> wrote:
>
>> Thanks to everyone who reviewed the doc. I put together a plan based
>> on the initial feedback to improve website automation reliability. At a
>> glance, I am proposing to:
>>
>> * Migrate website source code to the main apache/beam repository
>> * Discontinue checking-in generated HTML during the PR workflow
>> * Align to the existing apache/beam PR process (code review policy,
>> precommits, generic Git merge)
>> * Filter pre-commit jobs to only run when necessary
>> * Add a post-commit Jenkins job to push generated HTML to a separate
>> publishing branch
>>
>> Please take another look at the doc, specifically the new section
>> entitled "Proposed Solution":
>> https://s.apache.org/beam-site-automation
>> I'd like to gather feedback by Monday June 4, and if there is
>> consensus move forward with the implementation.
>>
>> Thanks,
>> Scott
>>
>>
>> Got feedback? tinyurl.com/swegner-feedback
>>
>> On Tue, May 29, 2018 at 4:32 PM Scott Wegner 
>> wrote:
>>
>>> I've been looking into the beam-site merge automation reliability,
>>> and I'd like to get some early feedback on ideas for improvement. Please
>>> take a look at https://s.apache.org/beam-site-automation:
>>>
>>> > Apache Beam's website is maintained via the beam-site Git
>>> repository, with a set of automation that manages the workflow from 
>>> merging
>>> a pull request to publishing. The automation is centralized in a tool
>>> called Mergebot, which was built for Beam and donated to the ASF. 
>>> However,
>>> the automation has been somewhat unreliable, and when there are issues,
>>> very few individuals have the necessary permissions and expertise to
>>> resolve them. Overall, the reliability of Beam-site automation is 
>>> impeding
>>> productivity for Beam-site development.
>>>
>>> At this point I'm seeking feedback on a few possible solutions:
>>>
>>> 1. Invest in improvements to Mergebot reliability. Make stability
>>> tweaks for various failure modes, distribute Mergebot expertise and
>>> operations permissions to more committers.
>>> 2. Deprecate Mergebot and revert to manual process. With the current
>>> unreliability, some committers choose to forego merge automation anyway.
>>> 3. Generate HTML only during publishing. This seems to be newly
>>> supported by the Apache GitPubSub workflow. This would eliminate most or
>>> all of the automation that Mergebot is responsible for.
>>>
>>> Feel free to add comments in the doc.
>>>
>>> Thanks,
>>> Scott
>>>
>>>
>>>
>>> Got feedback? tinyurl.com/swegner-feedback
>>>
>>
>
>
> --
> ---
> Jason Kuster
> Apache Beam / Google Cloud Dataflow
>
> See something? Say something. go/jasonkuster-feedback
> 
>


smime.p7s
Description: S/MIME Cryptographic Signature


Re: [DISCUSS] Versioning, Hadoop related dependencies and enterprise users

2018-08-28 Thread Raghu Angadi
Thanks for the IO versioning summary.
KafkaIO's policy of 'let the user decide exact version at runtime' has been
quite useful so far. How feasible is that for other connectors?

Also, KafkaIO does not limit itself to minimum features available across
all the supported versions. Some of the features (e.g. server side
timestamps) are disabled based on runtime Kafka version.  The unit tests
currently run with single recent version. Integration tests could certainly
use multiple versions. With some more effort in writing tests, we could
make multiple versions of the unit tests.

Raghu.

IO versioning
> * Elasticsearch. We delayed the move to version 6 until we heard of
> more active users needing it (more deployments). We support 2.x and
> 5.x (but 2.x went recently EOL). Support for 6.x is in progress.
> * SolrIO, stable version is 7.x, LTS is 6.x. We support only 5.x
> because most big data distributions still use 5.x (however 5.x has
> been EOL).
> * KafkaIO uses version 1.x but Kafka recently moved to 2.x, however
> most of the deployments of Kafka use earlier versions than 1.x. This
> module uses a single version with the kafka client as a provided
> dependency and so far it works (but we don’t have multi version
> tests).
>


On Tue, Aug 28, 2018 at 8:38 AM Ismaël Mejía  wrote:

> I think we should refine the strategy on dependencies discussed
> recently. Sorry to come late with this (I did not follow closely the
> previous discussion), but the current approach is clearly not in line
> with the industry reality (at least not for IO connectors + Hadoop +
> Spark/Flink use).
>
> A really proactive approach to dependency updates is a good practice
> for the core dependencies we have e.g. Guava, Bytebuddy, Avro,
> Protobuf, etc, and of course for the case of cloud based IOs e.g. GCS,
> Bigquery, AWS S3, etc. However when we talk about self hosted data
> sources or processing systems this gets more complicated and I think
> we should be more flexible and do this case by case (and remove these
> from the auto update email reminder).
>
> Some open source projects have at least three maintained versions:
> - LTS – maps to what most of the people have installed (or the big
> data distributions use) e.g. HBase 1.1.x, Hadoop 2.6.x
> - Stable – current recommended version. HBase 1.4.x, Hadoop 2.8.x
> - Next – latest release. HBase 2.1.x Hadoop 3.1.x
>
> Following the most recent versions can be good to be close to the
> current development of other projects and some of the fixes, but these
> versions are commonly not deployed for most users and adopting a LTS
> or stable only approach won't satisfy all cases either. To understand
> why this is complex let’s see some historical issues:
>
> IO versioning
> * Elasticsearch. We delayed the move to version 6 until we heard of
> more active users needing it (more deployments). We support 2.x and
> 5.x (but 2.x went recently EOL). Support for 6.x is in progress.
> * SolrIO, stable version is 7.x, LTS is 6.x. We support only 5.x
> because most big data distributions still use 5.x (however 5.x has
> been EOL).
> * KafkaIO uses version 1.x but Kafka recently moved to 2.x, however
> most of the deployments of Kafka use earlier versions than 1.x. This
> module uses a single version with the kafka client as a provided
> dependency and so far it works (but we don’t have multi version
> tests).
>
> Runners versioning
> * The move to Spark 1 to Spark 2 was decided after evaluating the
> tradeoffs between maintaining multiple version support and to have
> breaking changes with the issues of maintaining multiple versions.
> This is a rare case but also with consequences. This dependency is
> provided but we don't actively test issues on version migration.
> * Flink moved to version 1.5, introducing incompatibility in
> checkpointing (discussed recently and with not yet consensus on how to
> handle).
>
> As you can see, it seems really hard to have a solution that fits all
> cases. Probably the only rule that I see from this list is that we
> should upgrade versions for connectors that have been deprecated or
> arrived to the EOL (e.g. Solr 5.x, Elasticsearch 2.x).
>
> For the case of the provided dependencies I wonder if as part of the
> tests we should provide tests with multiple versions (note that this
> is currently blocked by BEAM-4087).
>
> Any other ideas or opinions to see how we can handle this? What other
> people in the community think ? (Notice that this can have relation
> with the ongoing LTS discussion.
>
>
> On Tue, Aug 28, 2018 at 10:44 AM Tim Robertson
>  wrote:
> >
> > Hi folks,
> >
> > I'd like to revisit the discussion around our versioning policy
> specifically for the Hadoop ecosystem and make sure we are aware of the
> implications.
> >
> > As an example our policy today would have us on HBase 2.1 and I have
> reminders to address this.
> >
> > However, currently the versions of HBase in the major hadoop distros are:
> >
> >  - Cloudera 5 on HBase 1.2 

Re: [DISCUSS] Versioning, Hadoop related dependencies and enterprise users

2018-08-28 Thread Chamikara Jayalath
Constrains to existing dependencies is a valid concern and we do not have a
good solution for this currently. One way to handle this is be simply to
close automatically created JIRAs with a comment and the tool will not try
to create further JIRAs for the same dependency after this. But we should
be able to come up with a better solution. In addition to this, integration
tests should also help to make sure that we do not upgrade these
dependencies breaking Beam components accidentally.

Thanks,
Cham

On Tue, Aug 28, 2018 at 10:45 AM Andrew Pilloud  wrote:

> The Beam SQL module faces similar problems, several of our dependencies
> are constrained by maintaining compatibility with versions used by Calcite.
> We've written tests to detect some of these incompatibilities. Could we add
> integration tests for these major hadoop distros that ensure we maintain
> compatibility rather then explicitly calling them out in our upgrade policy?
>
> Andrew
>
> On Tue, Aug 28, 2018 at 10:31 AM Chamikara Jayalath 
> wrote:
>
>> Thanks Tim for raising this and Thanks JB and Ismaël for all the great
>> points.
>>
>> I agree that one size fit all solution will not work when it comes to
>> dependencies. Based on past examples, clearly there are many cases where we
>> should proceed with caution and upgrade dependencies with care.
>>
>> That said, given that Beam respects semantic versioning and most of our
>> dependencies respect semantic versioning I think we should be able to
>> upgrade most minor (and patch) versions of dependencies with relative ease.
>> Current policy is to automatically create JIRAs if we are more than three
>> minor versions behind. I'm not sure if HBase respects semantic versioning.
>> If it does not, I think, it should be the exception not the norm.
>>
>> When it comes major version upgrades though we'll have to proceed with
>> caution. In addition to all the case-by-case reasoning Ismaël gave above
>> there's also the real possibility of a major version upgrade changing Beam
>> API (syntax or semantics) in a non backwards compatible way and breaking
>> the backwards compatibility guarantee offered by Beam. Current dependency
>> policy [1] try to capture this in a separate section and requires all PRs
>> that upgrade dependencies to contain a statement regarding backwards
>> compatibility.
>>
>> I agree that there might be many modifications we have to make to
>> existing policies when it comes to upgrading Beam dependencies in according
>> to industry standards. Current policies are there as a first version for us
>> to try out. We should definitely time to time reevaluate and update the
>> policies as needed. I'm also extremely eager to hear what others in the
>> community think about this.
>>
>> Thanks,
>> Cham
>>
>> [1] https://beam.apache.org/contribute/dependencies/
>>
>> On Tue, Aug 28, 2018 at 8:38 AM Ismaël Mejía  wrote:
>>
>>> I think we should refine the strategy on dependencies discussed
>>> recently. Sorry to come late with this (I did not follow closely the
>>> previous discussion), but the current approach is clearly not in line
>>> with the industry reality (at least not for IO connectors + Hadoop +
>>> Spark/Flink use).
>>>
>>> A really proactive approach to dependency updates is a good practice
>>> for the core dependencies we have e.g. Guava, Bytebuddy, Avro,
>>> Protobuf, etc, and of course for the case of cloud based IOs e.g. GCS,
>>> Bigquery, AWS S3, etc. However when we talk about self hosted data
>>> sources or processing systems this gets more complicated and I think
>>> we should be more flexible and do this case by case (and remove these
>>> from the auto update email reminder).
>>>
>>> Some open source projects have at least three maintained versions:
>>> - LTS – maps to what most of the people have installed (or the big
>>> data distributions use) e.g. HBase 1.1.x, Hadoop 2.6.x
>>> - Stable – current recommended version. HBase 1.4.x, Hadoop 2.8.x
>>> - Next – latest release. HBase 2.1.x Hadoop 3.1.x
>>>
>>> Following the most recent versions can be good to be close to the
>>> current development of other projects and some of the fixes, but these
>>> versions are commonly not deployed for most users and adopting a LTS
>>> or stable only approach won't satisfy all cases either. To understand
>>> why this is complex let’s see some historical issues:
>>>
>>> IO versioning
>>> * Elasticsearch. We delayed the move to version 6 until we heard of
>>> more active users needing it (more deployments). We support 2.x and
>>> 5.x (but 2.x went recently EOL). Support for 6.x is in progress.
>>> * SolrIO, stable version is 7.x, LTS is 6.x. We support only 5.x
>>> because most big data distributions still use 5.x (however 5.x has
>>> been EOL).
>>> * KafkaIO uses version 1.x but Kafka recently moved to 2.x, however
>>> most of the deployments of Kafka use earlier versions than 1.x. This
>>> module uses a single version with the kafka client as a provided
>>> dependency and so far it 

Re: [DISCUSS] Versioning, Hadoop related dependencies and enterprise users

2018-08-28 Thread Andrew Pilloud
The Beam SQL module faces similar problems, several of our dependencies are
constrained by maintaining compatibility with versions used by Calcite.
We've written tests to detect some of these incompatibilities. Could we add
integration tests for these major hadoop distros that ensure we maintain
compatibility rather then explicitly calling them out in our upgrade policy?

Andrew

On Tue, Aug 28, 2018 at 10:31 AM Chamikara Jayalath 
wrote:

> Thanks Tim for raising this and Thanks JB and Ismaël for all the great
> points.
>
> I agree that one size fit all solution will not work when it comes to
> dependencies. Based on past examples, clearly there are many cases where we
> should proceed with caution and upgrade dependencies with care.
>
> That said, given that Beam respects semantic versioning and most of our
> dependencies respect semantic versioning I think we should be able to
> upgrade most minor (and patch) versions of dependencies with relative ease.
> Current policy is to automatically create JIRAs if we are more than three
> minor versions behind. I'm not sure if HBase respects semantic versioning.
> If it does not, I think, it should be the exception not the norm.
>
> When it comes major version upgrades though we'll have to proceed with
> caution. In addition to all the case-by-case reasoning Ismaël gave above
> there's also the real possibility of a major version upgrade changing Beam
> API (syntax or semantics) in a non backwards compatible way and breaking
> the backwards compatibility guarantee offered by Beam. Current dependency
> policy [1] try to capture this in a separate section and requires all PRs
> that upgrade dependencies to contain a statement regarding backwards
> compatibility.
>
> I agree that there might be many modifications we have to make to existing
> policies when it comes to upgrading Beam dependencies in according to
> industry standards. Current policies are there as a first version for us to
> try out. We should definitely time to time reevaluate and update the
> policies as needed. I'm also extremely eager to hear what others in the
> community think about this.
>
> Thanks,
> Cham
>
> [1] https://beam.apache.org/contribute/dependencies/
>
> On Tue, Aug 28, 2018 at 8:38 AM Ismaël Mejía  wrote:
>
>> I think we should refine the strategy on dependencies discussed
>> recently. Sorry to come late with this (I did not follow closely the
>> previous discussion), but the current approach is clearly not in line
>> with the industry reality (at least not for IO connectors + Hadoop +
>> Spark/Flink use).
>>
>> A really proactive approach to dependency updates is a good practice
>> for the core dependencies we have e.g. Guava, Bytebuddy, Avro,
>> Protobuf, etc, and of course for the case of cloud based IOs e.g. GCS,
>> Bigquery, AWS S3, etc. However when we talk about self hosted data
>> sources or processing systems this gets more complicated and I think
>> we should be more flexible and do this case by case (and remove these
>> from the auto update email reminder).
>>
>> Some open source projects have at least three maintained versions:
>> - LTS – maps to what most of the people have installed (or the big
>> data distributions use) e.g. HBase 1.1.x, Hadoop 2.6.x
>> - Stable – current recommended version. HBase 1.4.x, Hadoop 2.8.x
>> - Next – latest release. HBase 2.1.x Hadoop 3.1.x
>>
>> Following the most recent versions can be good to be close to the
>> current development of other projects and some of the fixes, but these
>> versions are commonly not deployed for most users and adopting a LTS
>> or stable only approach won't satisfy all cases either. To understand
>> why this is complex let’s see some historical issues:
>>
>> IO versioning
>> * Elasticsearch. We delayed the move to version 6 until we heard of
>> more active users needing it (more deployments). We support 2.x and
>> 5.x (but 2.x went recently EOL). Support for 6.x is in progress.
>> * SolrIO, stable version is 7.x, LTS is 6.x. We support only 5.x
>> because most big data distributions still use 5.x (however 5.x has
>> been EOL).
>> * KafkaIO uses version 1.x but Kafka recently moved to 2.x, however
>> most of the deployments of Kafka use earlier versions than 1.x. This
>> module uses a single version with the kafka client as a provided
>> dependency and so far it works (but we don’t have multi version
>> tests).
>>
>> Runners versioning
>> * The move to Spark 1 to Spark 2 was decided after evaluating the
>> tradeoffs between maintaining multiple version support and to have
>> breaking changes with the issues of maintaining multiple versions.
>> This is a rare case but also with consequences. This dependency is
>> provided but we don't actively test issues on version migration.
>> * Flink moved to version 1.5, introducing incompatibility in
>> checkpointing (discussed recently and with not yet consensus on how to
>> handle).
>>
>> As you can see, it seems really hard to have a solution that fits all
>> cases. 

Re: [DISCUSS] Versioning, Hadoop related dependencies and enterprise users

2018-08-28 Thread Chamikara Jayalath
Thanks Tim for raising this and Thanks JB and Ismaël for all the great
points.

I agree that one size fit all solution will not work when it comes to
dependencies. Based on past examples, clearly there are many cases where we
should proceed with caution and upgrade dependencies with care.

That said, given that Beam respects semantic versioning and most of our
dependencies respect semantic versioning I think we should be able to
upgrade most minor (and patch) versions of dependencies with relative ease.
Current policy is to automatically create JIRAs if we are more than three
minor versions behind. I'm not sure if HBase respects semantic versioning.
If it does not, I think, it should be the exception not the norm.

When it comes major version upgrades though we'll have to proceed with
caution. In addition to all the case-by-case reasoning Ismaël gave above
there's also the real possibility of a major version upgrade changing Beam
API (syntax or semantics) in a non backwards compatible way and breaking
the backwards compatibility guarantee offered by Beam. Current dependency
policy [1] try to capture this in a separate section and requires all PRs
that upgrade dependencies to contain a statement regarding backwards
compatibility.

I agree that there might be many modifications we have to make to existing
policies when it comes to upgrading Beam dependencies in according to
industry standards. Current policies are there as a first version for us to
try out. We should definitely time to time reevaluate and update the
policies as needed. I'm also extremely eager to hear what others in the
community think about this.

Thanks,
Cham

[1] https://beam.apache.org/contribute/dependencies/

On Tue, Aug 28, 2018 at 8:38 AM Ismaël Mejía  wrote:

> I think we should refine the strategy on dependencies discussed
> recently. Sorry to come late with this (I did not follow closely the
> previous discussion), but the current approach is clearly not in line
> with the industry reality (at least not for IO connectors + Hadoop +
> Spark/Flink use).
>
> A really proactive approach to dependency updates is a good practice
> for the core dependencies we have e.g. Guava, Bytebuddy, Avro,
> Protobuf, etc, and of course for the case of cloud based IOs e.g. GCS,
> Bigquery, AWS S3, etc. However when we talk about self hosted data
> sources or processing systems this gets more complicated and I think
> we should be more flexible and do this case by case (and remove these
> from the auto update email reminder).
>
> Some open source projects have at least three maintained versions:
> - LTS – maps to what most of the people have installed (or the big
> data distributions use) e.g. HBase 1.1.x, Hadoop 2.6.x
> - Stable – current recommended version. HBase 1.4.x, Hadoop 2.8.x
> - Next – latest release. HBase 2.1.x Hadoop 3.1.x
>
> Following the most recent versions can be good to be close to the
> current development of other projects and some of the fixes, but these
> versions are commonly not deployed for most users and adopting a LTS
> or stable only approach won't satisfy all cases either. To understand
> why this is complex let’s see some historical issues:
>
> IO versioning
> * Elasticsearch. We delayed the move to version 6 until we heard of
> more active users needing it (more deployments). We support 2.x and
> 5.x (but 2.x went recently EOL). Support for 6.x is in progress.
> * SolrIO, stable version is 7.x, LTS is 6.x. We support only 5.x
> because most big data distributions still use 5.x (however 5.x has
> been EOL).
> * KafkaIO uses version 1.x but Kafka recently moved to 2.x, however
> most of the deployments of Kafka use earlier versions than 1.x. This
> module uses a single version with the kafka client as a provided
> dependency and so far it works (but we don’t have multi version
> tests).
>
> Runners versioning
> * The move to Spark 1 to Spark 2 was decided after evaluating the
> tradeoffs between maintaining multiple version support and to have
> breaking changes with the issues of maintaining multiple versions.
> This is a rare case but also with consequences. This dependency is
> provided but we don't actively test issues on version migration.
> * Flink moved to version 1.5, introducing incompatibility in
> checkpointing (discussed recently and with not yet consensus on how to
> handle).
>
> As you can see, it seems really hard to have a solution that fits all
> cases. Probably the only rule that I see from this list is that we
> should upgrade versions for connectors that have been deprecated or
> arrived to the EOL (e.g. Solr 5.x, Elasticsearch 2.x).
>
> For the case of the provided dependencies I wonder if as part of the
> tests we should provide tests with multiple versions (note that this
> is currently blocked by BEAM-4087).
>
> Any other ideas or opinions to see how we can handle this? What other
> people in the community think ? (Notice that this can have relation
> with the ongoing LTS discussion.
>
>
> On 

Re: [DISCUSS] Versioning, Hadoop related dependencies and enterprise users

2018-08-28 Thread Ismaël Mejía
I think we should refine the strategy on dependencies discussed
recently. Sorry to come late with this (I did not follow closely the
previous discussion), but the current approach is clearly not in line
with the industry reality (at least not for IO connectors + Hadoop +
Spark/Flink use).

A really proactive approach to dependency updates is a good practice
for the core dependencies we have e.g. Guava, Bytebuddy, Avro,
Protobuf, etc, and of course for the case of cloud based IOs e.g. GCS,
Bigquery, AWS S3, etc. However when we talk about self hosted data
sources or processing systems this gets more complicated and I think
we should be more flexible and do this case by case (and remove these
from the auto update email reminder).

Some open source projects have at least three maintained versions:
- LTS – maps to what most of the people have installed (or the big
data distributions use) e.g. HBase 1.1.x, Hadoop 2.6.x
- Stable – current recommended version. HBase 1.4.x, Hadoop 2.8.x
- Next – latest release. HBase 2.1.x Hadoop 3.1.x

Following the most recent versions can be good to be close to the
current development of other projects and some of the fixes, but these
versions are commonly not deployed for most users and adopting a LTS
or stable only approach won't satisfy all cases either. To understand
why this is complex let’s see some historical issues:

IO versioning
* Elasticsearch. We delayed the move to version 6 until we heard of
more active users needing it (more deployments). We support 2.x and
5.x (but 2.x went recently EOL). Support for 6.x is in progress.
* SolrIO, stable version is 7.x, LTS is 6.x. We support only 5.x
because most big data distributions still use 5.x (however 5.x has
been EOL).
* KafkaIO uses version 1.x but Kafka recently moved to 2.x, however
most of the deployments of Kafka use earlier versions than 1.x. This
module uses a single version with the kafka client as a provided
dependency and so far it works (but we don’t have multi version
tests).

Runners versioning
* The move to Spark 1 to Spark 2 was decided after evaluating the
tradeoffs between maintaining multiple version support and to have
breaking changes with the issues of maintaining multiple versions.
This is a rare case but also with consequences. This dependency is
provided but we don't actively test issues on version migration.
* Flink moved to version 1.5, introducing incompatibility in
checkpointing (discussed recently and with not yet consensus on how to
handle).

As you can see, it seems really hard to have a solution that fits all
cases. Probably the only rule that I see from this list is that we
should upgrade versions for connectors that have been deprecated or
arrived to the EOL (e.g. Solr 5.x, Elasticsearch 2.x).

For the case of the provided dependencies I wonder if as part of the
tests we should provide tests with multiple versions (note that this
is currently blocked by BEAM-4087).

Any other ideas or opinions to see how we can handle this? What other
people in the community think ? (Notice that this can have relation
with the ongoing LTS discussion.


On Tue, Aug 28, 2018 at 10:44 AM Tim Robertson
 wrote:
>
> Hi folks,
>
> I'd like to revisit the discussion around our versioning policy specifically 
> for the Hadoop ecosystem and make sure we are aware of the implications.
>
> As an example our policy today would have us on HBase 2.1 and I have 
> reminders to address this.
>
> However, currently the versions of HBase in the major hadoop distros are:
>
>  - Cloudera 5 on HBase 1.2 (Cloudera 6 is 2.1 but is only in beta)
>  - Hortonworks HDP3 on HBase 2.0 (only recently released so we can assume is 
> not widely adopted)
>  - AWS EMR HBase on 1.4
>
> On the versioning I think we might need a more nuanced approach to ensure 
> that we target real communities of existing and potential users. Enterprise 
> users need to stick to the supported versions in the distributions to 
> maintain support contracts from the vendors.
>
> Should our versioning policy have more room to consider on a case by case 
> basis?
>
> For Hadoop might we benefit from a strategy on which community of users Beam 
> is targeting?
>
> (OT: I'm collecting some thoughts on what we might consider to target 
> enterprise hadoop users - kerberos on all relevant IO, performance, leaking 
> beyond encryption zones with temporary files etc)
>
> Thanks,
> Tim


Re: Publishing release artifacts to custom artifactory

2018-08-28 Thread Alexey Romanenko
Thomas, thanks, looks great. Do you think we have to add this command to 
“Contribution Guide”?

Lukasz, yes, "-Poffline-repository"  can be omitted in this case. I don’t 
remember why I added this =)

> On 24 Aug 2018, at 22:15, Thomas Weise  wrote:
> 
> Alexey, publishing to custom repo with authentication is now possible, see 
> https://github.com/apache/beam/pull/6230 
>  with example.
> 
> 
> On Fri, Aug 24, 2018 at 1:08 PM Lukasz Cwik  > wrote:
> "-Poffline-repository" controls the addition of another maven repo to read 
> dependencies from, it doesn't impact project publishing and shouldn't be 
> needed.
> 
> On Fri, Aug 24, 2018 at 5:28 AM Alexey Romanenko  > wrote:
> Maybe my answer is not 100% relevant to initial topic (sorry for that in 
> advance) but it took me quite a time to find out how to properly install 
> artefacts into local maven repository with gradle.
> Finally, I came to this command (additional flags are skipped for the sake of 
> simplicity).
> 
> ./gradlew -Ppublishing 
> -Poffline-repository=file:/// <> --no-parallel 
> -PdistMgmtSnapshotsUrl=file:/// <> -p 
>  publish
> 
> For example,I used this command to publish KinesisIO artifacts on local maven 
> to test my changes against external Beam-based application:
> 
> ./gradlew -Ppublishing 
> -Poffline-repository=file:///Users/aromanenko/.m2/repository/ <> 
> --no-parallel -PdistMgmtSnapshotsUrl=file:///Users/aromanenko/.m2/repository/ 
> <> -p sdks/java/io/kinesis/ publish
> 
> Not sure, if this way is fully correct, but, at least, it's worked for me =)
> 
>> On 15 Aug 2018, at 16:32, Thomas Weise > > wrote:
>> 
>> Hey JB,
>> 
>> I think it has to be -PisRelease=false (otherwise the publish task isn't 
>> activated).
>> 
>> For -SNAPSHOT, I'm getting further with 
>> https://github.com/apache/beam/pull/6230 
>> 
>> 
>> I'm now looking for a good way to allow skipping gpg (when the version isn't 
>> -SNAPSHOT). Currently, it will run whenever the isRelease property is 
>> present.
>> 
>> Thanks
>> 
>> 
>> On Wed, Aug 15, 2018 at 4:42 AM Jean-Baptiste Onofré > > wrote:
>> Hi Thomas,
>> 
>> yes, I did on a local Nexus repository.
>> 
>> Did you try with -DisRelease=false ?
>> 
>> Regards
>> JB
>> 
>> On 15/08/2018 07:54, Thomas Weise wrote:
>> > Hi,
>> > 
>> > I wanted to see if anyone has used the Gradle build to publish the
>> > release/snapshot artifacts to custom repo?
>> > 
>> > So far I found the following (attempting to publish a particular Maven
>> > artifact):
>> > 
>> > ./gradlew :beam-runners-flink_2.11-job-server:publish -Ppublishing
>> > -PdistMgmtSnapshotsUrl=https://somecustomrepo/libs-snapshots 
>> > 
>> > --no-parallel --no-daemon
>> > 
>> > It gets to the point of attempting deploy to the remote repo, but fails
>> > due to hardcoded server IDs:
>> > 
>> > https://github.com/apache/beam/blob/b82263390af57a2d52c5d7b257470bc837606e4e/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy#L817
>> >  
>> > 
>> > 
>> > When the version isn't -SNAPSHOT, then I guess it would instead be
>> > -PisRelease, the build will fail because it expects gpg to run, which I
>> > don't need for what I'm trying to do.
>> > 
>> > Have others attempted third party builds for the Python SDK or Docker
>> > images? Any gotchas to share?
>> > 
>> > Thanks,
>> > Thomas
>> > 
>> > 
>> 
>> -- 
>> Jean-Baptiste Onofré
>> jbono...@apache.org 
>> http://blog.nanthrax.net 
>> Talend - http://www.talend.com 
> 



Re: [DISCUSS] Versioning, Hadoop related dependencies and enterprise users

2018-08-28 Thread Jean-Baptiste Onofré
Hi Tim,

regarding the IO, while ago (at the incubator time of the project), we
discussed how to deal with different versions of the backend API and
dependencies. I proposed to have a release cycle per IO and have a
subproject per IO version, like for instance:

sdks/java/io/elasticsearch-5
sdks/java/io/elasticsearch-6
...

I'm still thinking that the best option, allowing to really leverage the
backend version in the right way.

Regarding the release, it's like what I'm doing in the ServiceMix
Bundles: the IO can have their own release cycle. As we agreed on a
periodical release cycle, not sure if it's still required, but it could
be interesting (why not having a specific repository for IOs).

Regards
JB

On 28/08/2018 10:43, Tim Robertson wrote:
> Hi folks,
> 
> I'd like to revisit the discussion around our versioning policy
> specifically for the Hadoop ecosystem and make sure we are aware of the
> implications.
> 
> As an example our policy today would have us on HBase 2.1 and I have
> reminders to address this.
> 
> However, currently the versions of HBase in the major hadoop distros are:
> 
>  - Cloudera 5 on HBase 1.2 (Cloudera 6 is 2.1 but is only in beta)
>  - Hortonworks HDP3 on HBase 2.0 (only recently released so we can
> assume is not widely adopted)
>  - AWS EMR HBase on 1.4
> 
> On the versioning I think we might need a more nuanced approach to
> ensure that we target real communities of existing and potential users.
> Enterprise users need to stick to the supported versions in the
> distributions to maintain support contracts from the vendors.
> 
> Should our versioning policy have more room to consider on a case by
> case basis?
> 
> For Hadoop might we benefit from a strategy on which community of users
> Beam is targeting? 
> 
> (OT: I'm collecting some thoughts on what we might consider to target
> enterprise hadoop users - kerberos on all relevant IO, performance,
> leaking beyond encryption zones with temporary files etc)
> 
> Thanks,
> Tim


[DISCUSS] Versioning, Hadoop related dependencies and enterprise users

2018-08-28 Thread Tim Robertson
Hi folks,

I'd like to revisit the discussion around our versioning policy
specifically for the Hadoop ecosystem and make sure we are aware of the
implications.

As an example our policy today would have us on HBase 2.1 and I have
reminders to address this.

However, currently the versions of HBase in the major hadoop distros are:

 - Cloudera 5 on HBase 1.2 (Cloudera 6 is 2.1 but is only in beta)
 - Hortonworks HDP3 on HBase 2.0 (only recently released so we can assume
is not widely adopted)
 - AWS EMR HBase on 1.4

On the versioning I think we might need a more nuanced approach to ensure
that we target real communities of existing and potential users. Enterprise
users need to stick to the supported versions in the distributions to
maintain support contracts from the vendors.

Should our versioning policy have more room to consider on a case by case
basis?

For Hadoop might we benefit from a strategy on which community of users
Beam is targeting?

(OT: I'm collecting some thoughts on what we might consider to target
enterprise hadoop users - kerberos on all relevant IO, performance, leaking
beyond encryption zones with temporary files etc)

Thanks,
Tim


Re: JB's back

2018-08-28 Thread Mikhail Gryzykhin
Welcome back JB!

On Mon, Aug 27, 2018 at 11:10 PM Reuven Lax  wrote:

> Welcome back!
>
> On Mon, Aug 27, 2018 at 10:44 PM Jean-Baptiste Onofré 
> wrote:
>
>> Hi guys,
>>
>> Maybe you saw that I took some days off last week. I landed back last
>> night, so, just time to unstack my e-mails and I'm back ;)
>>
>> Regards
>> JB
>> --
>> Jean-Baptiste Onofré
>> jbono...@apache.org
>> http://blog.nanthrax.net
>> Talend - http://www.talend.com
>>
>


Re: JB's back

2018-08-28 Thread Reuven Lax
Welcome back!

On Mon, Aug 27, 2018 at 10:44 PM Jean-Baptiste Onofré 
wrote:

> Hi guys,
>
> Maybe you saw that I took some days off last week. I landed back last
> night, so, just time to unstack my e-mails and I'm back ;)
>
> Regards
> JB
> --
> Jean-Baptiste Onofré
> jbono...@apache.org
> http://blog.nanthrax.net
> Talend - http://www.talend.com
>