Return-Path: X-Original-To: apmail-flink-user-archive@minotaur.apache.org Delivered-To: apmail-flink-user-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id EFF9019C4C for ; Wed, 9 Mar 2016 12:35:44 +0000 (UTC) Received: (qmail 38301 invoked by uid 500); 9 Mar 2016 12:35:39 -0000 Delivered-To: apmail-flink-user-archive@flink.apache.org Received: (qmail 38220 invoked by uid 500); 9 Mar 2016 12:35:39 -0000 Mailing-List: contact user-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@flink.apache.org Delivered-To: mailing list user@flink.apache.org Received: (qmail 38211 invoked by uid 99); 9 Mar 2016 12:35:39 -0000 Received: from mail-relay.apache.org (HELO mail-relay.apache.org) (140.211.11.15) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 09 Mar 2016 12:35:39 +0000 Received: from [192.168.178.102] (ip5b40315a.dynamic.kabel-deutschland.de [91.64.49.90]) by mail-relay.apache.org (ASF Mail Server at mail-relay.apache.org) with ESMTPSA id 7152B1A0040 for ; Wed, 9 Mar 2016 12:35:37 +0000 (UTC) Subject: Re: JDBCInputFormat preparation with Flink 1.1-SNAPSHOT and Scala 2.11 To: user@flink.apache.org References: <588AE178-9642-4D6D-BA00-D2DB7516AE38@opencorrelate.org> <56DFDE08.1040105@apache.org> <7E8F549D-DB52-4A27-86AE-4DAE71BB9244@correlatesystems.com> From: Chesnay Schepler Message-ID: <56E01899.2050309@apache.org> Date: Wed, 9 Mar 2016 13:35:37 +0100 User-Agent: Mozilla/5.0 (Windows NT 10.0; WOW64; rv:38.0) Gecko/20100101 Thunderbird/38.5.0 MIME-Version: 1.0 In-Reply-To: <7E8F549D-DB52-4A27-86AE-4DAE71BB9244@correlatesystems.com> Content-Type: multipart/alternative; boundary="------------080106020004020902000904" This is a multi-part message in MIME format. --------------080106020004020902000904 Content-Type: text/plain; charset=utf-8; format=flowed Content-Transfer-Encoding: 8bit now that i look back to my mail i may have given you the wrong idea about the prototype; to make sure we are on the same page: the only thing it enables is using the JDBCInputFormat without providing a separate TypeInformation. Still works with tuples, not POJO's. you can find the prototype here: https://github.com/zentol/flink/tree/3445_jdbc The JDBCInputFormat there implements ResultTypeQueryable. WIthin getProducedType it executes a dummy query, reads the ResultSetMetaData and generates a TypeInfo from it. On 09.03.2016 12:46, Prez Cannady wrote: > I suspected as much (the tuple size limitation). Creating my own > InputFormat seems to be the best solution, but before i go down that > rabbit hole I wanted to see at least a semi-trivial working example of > JDBCInputFormat with Scala 2.11. > > I’d appreciate a look at that prototype if its publicly available > (even if it is Java). I might glean a hint from it. > > Prez Cannady > p: 617 500 3378 > e: revprez@opencorrelate.org > GH: https://github.com/opencorrelate > LI: https://www.linkedin.com/in/revprez > >> On Mar 9, 2016, at 3:25 AM, Chesnay Schepler > > wrote: >> >> you can always create your own InputFormat, but there is no >> AbstractJDBCInputFormat if that's what you were looking for. >> >> When you say arbitrary tuple size, do you mean a) a size greater than >> 25, or b) tuples of different sizes? >> If a) unless you are fine with using nested tuples you won't get >> around the tuple size limitation. Since the user has to be aware of >> the nesting (since the fields can be accessed directly via tuple.f0 >> etc), this can't really be done in a general-purpose fashion. >> If b) this will straight-up not work with tuples. >> >> You could use POJO's though. then you could also group by column names. >> >> I'm not sure about Scala, but in the Java Stream API you can pass the >> InputFormat and the TypeInformation into createInput. >> >> I've recently did a prototype where the input type is determined >> automatically by querying the database. If this is a problem for you >> feel free to ping me. >> >> On 09.03.2016 03:17, Prez Cannady wrote: >>> I’m attempting to create a stream using JDBCInputFormat. Objective >>> is to convert each record into a tuple and then serialize for input >>> into a Kafka topic. Here’s what I have so far. >>> >>> ``` >>> val env = StreamExecutionEnvironment.getExecutionEnvironment >>> >>> val inputFormat = JDBCInputFormat.buildJDBCInputFormat() >>> .setDrivername("org.postgresql.Driver") >>> .setDBUrl("jdbc:postgresql:test") >>> .setQuery("select name from persons") >>> .finish() >>> >>> val stream : DataStream[Tuple1[String]] = env.createInput(...) >>> ``` >>> >>> I think this is essentially what I want to do. It would be nice if >>> I could return tuples of arbitrary length, but reading the code >>> suggests I have to commit to a defined arity. So I have some questions. >>> >>> 1. Is there a better way to read from a database (i.e., defining my >>> own `InputFormat` using Slick)? >>> 2. To get the above example working, what should I supply to >>> `createInput`? >>> >>> >>> Prez Cannady >>> p: 617 500 3378 >>> e: revprez@opencorrelate.org >>> GH: https://github.com/opencorrelate >>> LI: https://www.linkedin.com/in/revprez >>> >>> >>> >>> >>> >>> >>> >>> >>> >> > --------------080106020004020902000904 Content-Type: text/html; charset=utf-8 Content-Transfer-Encoding: 8bit
now that i look back to my mail i may have given you the wrong idea about the prototype; to make sure we are on the same page:
the only thing it enables is using the JDBCInputFormat without providing a separate TypeInformation. Still works with tuples, not POJO's.

you can find the prototype here: https://github.com/zentol/flink/tree/3445_jdbc

The JDBCInputFormat there implements ResultTypeQueryable. WIthin getProducedType it executes a dummy query, reads the ResultSetMetaData and generates a TypeInfo from it.

On 09.03.2016 12:46, Prez Cannady wrote:
I suspected as much (the tuple size limitation).  Creating my own InputFormat seems to be the best solution, but before i go down that rabbit hole I wanted to see at least a semi-trivial working example of JDBCInputFormat with Scala 2.11.

I’d appreciate a look at that prototype if its publicly available (even if it is Java). I might glean a hint from it.

Prez Cannady  
On Mar 9, 2016, at 3:25 AM, Chesnay Schepler <chesnay@apache.org> wrote:

you can always create your own InputFormat, but there is no AbstractJDBCInputFormat if that's what you were looking for.

When you say arbitrary tuple size, do you mean a) a size greater than 25, or b) tuples of different sizes?
If a) unless you are fine with using nested tuples you won't get around the tuple size limitation. Since the user has to be aware of the nesting (since the fields can be accessed directly via tuple.f0 etc), this can't really be done in a general-purpose fashion.
If b) this will straight-up not work with tuples.

You could use POJO's though. then you could also group by column names.

I'm not sure about Scala, but in the Java Stream API you can pass the InputFormat and the TypeInformation into createInput.

I've recently did a prototype where the input type is determined automatically by querying the database. If this is a problem for you feel free to ping me.

On 09.03.2016 03:17, Prez Cannady wrote:
I’m attempting to create a stream using JDBCInputFormat.  Objective is to convert each record into a tuple and then serialize for input into a Kafka topic.  Here’s what I have so far.

```
val env = StreamExecutionEnvironment.getExecutionEnvironment

val inputFormat = JDBCInputFormat.buildJDBCInputFormat()
      .setDrivername("org.postgresql.Driver")
      .setDBUrl("jdbc:postgresql:test")
      .setQuery("select name from persons")
      .finish()

val stream : DataStream[Tuple1[String]] = env.createInput(...)
```

I think this is essentially what I want to do.  It would be nice if I could return tuples of arbitrary length, but reading the code suggests I have to commit to a defined arity.  So I have some questions.

1. Is there a better way to read from a database (i.e., defining my own `InputFormat` using Slick)?
2. To get the above example working, what should I supply to `createInput`?


Prez Cannady  











--------------080106020004020902000904--