Re: [DISCUSS] Inconsistent naming of intermediate results

2015-06-01 Thread Ufuk Celebi
I would like to get this done with the upcoming release to have a stable name for the documentation. Thinking about the names with Stephan, he had a great suggestion to rename them to "streams". I like this idea very much. The supported result variants make more sense when you think about them as

Re: [DISCUSS] Inconsistent naming of intermediate results

2015-06-01 Thread Aljoscha Krettek
+1 I like it. We are a streaming system underneath after all. On Jun 1, 2015 10:02 AM, "Ufuk Celebi" wrote: > I would like to get this done with the upcoming release to have a stable > name for the documentation. > > Thinking about the names with Stephan, he had a great suggestion to rename > the

MultipleLinearRegression - Strange results

2015-06-01 Thread Felix Neutatz
Hi, I want to use MultipleLinearRegression, but I got really strange results. So I tested it with the housing price dataset: http://archive.ics.uci.edu/ml/machine-learning-databases/housing/housing.data And here I get negative house prices - even when I use the training set as dataset: LabeledVec

[jira] [Created] (FLINK-2122) Make all internal streaming operators Checkpointable

2015-06-01 Thread Aljoscha Krettek (JIRA)
Aljoscha Krettek created FLINK-2122: --- Summary: Make all internal streaming operators Checkpointable Key: FLINK-2122 URL: https://issues.apache.org/jira/browse/FLINK-2122 Project: Flink Issu

[jira] [Created] (FLINK-2123) Fix CLI client logging

2015-06-01 Thread Ufuk Celebi (JIRA)
Ufuk Celebi created FLINK-2123: -- Summary: Fix CLI client logging Key: FLINK-2123 URL: https://issues.apache.org/jira/browse/FLINK-2123 Project: Flink Issue Type: Bug Components: Core

Re: MultipleLinearRegression - Strange results

2015-06-01 Thread Till Rohrmann
Since MLR uses stochastic gradient descent, you probably have to configure the step size right. SGD is very sensitive to the right step size choice. If the step size is too high, then the SGD algorithm does not converge. You can find the parameter description here [1]. Cheers, Till [1] http://ci.

[jira] [Created] (FLINK-2124) FromElementsFunction is not really Serializable

2015-06-01 Thread Aljoscha Krettek (JIRA)
Aljoscha Krettek created FLINK-2124: --- Summary: FromElementsFunction is not really Serializable Key: FLINK-2124 URL: https://issues.apache.org/jira/browse/FLINK-2124 Project: Flink Issue Typ

[jira] [Created] (FLINK-2125) String delimiter for SocketTextStream

2015-06-01 Thread JIRA
Márton Balassi created FLINK-2125: - Summary: String delimiter for SocketTextStream Key: FLINK-2125 URL: https://issues.apache.org/jira/browse/FLINK-2125 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-2126) Scala shell tests sporadically fail on travis

2015-06-01 Thread Robert Metzger (JIRA)
Robert Metzger created FLINK-2126: - Summary: Scala shell tests sporadically fail on travis Key: FLINK-2126 URL: https://issues.apache.org/jira/browse/FLINK-2126 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-2127) The GSA Documentation has trailing s

2015-06-01 Thread Andra Lungu (JIRA)
Andra Lungu created FLINK-2127: -- Summary: The GSA Documentation has trailing s Key: FLINK-2127 URL: https://issues.apache.org/jira/browse/FLINK-2127 Project: Flink Issue Type: Bug Com

[DISCUSS] Consolidate method naming between the batch and streaming API

2015-06-01 Thread Márton Balassi
Looking at the DataSet and DataStream APIs we have come to the conclusion with Aljoscha that there are a few methods that although providing the same functionality are named differently. These are the following: 1. rebalance (batch) / distribute (streaming): Rebalances the data sent to the

Re: [DISCUSS] Consolidate method naming between the batch and streaming API

2015-06-01 Thread Aljoscha Krettek
Yes, these renamings make sense. The partitionBy() is not yet in the master for streaming, though. On Mon, Jun 1, 2015 at 4:10 PM, Márton Balassi wrote: > Looking at the DataSet and DataStream APIs we have come to the conclusion > with Aljoscha that there are a few methods that although providing

ResendUnfulfillableException at the end of failing job

2015-06-01 Thread Márton Balassi
While experimenting in a cluster setting I was experiencing some hardware failures causing some taskmanagers to be unregistered and as a result also failing my streaming jobs. In the logs after the taskmanager dies I see some akka Exceptions. I think they are harmless compared to loosing taskmanage

Scala Tuple2 not recognized as a Scala Tuple

2015-06-01 Thread Pieter-Jan Van Aeken
>From the Java API, I get an instance of_ DataSet_​ which I wrap with the method available in _org.apache.flink.api.scala.package. _This works fine when I print out the result, but when I try to write as CSV, I get an exception: java.lang.IllegalArgumentException: requirement failed: CSV output ca

Re: ResendUnfulfillableException at the end of failing job

2015-06-01 Thread Ufuk Celebi
Thanks for posting this. From what I've found online, this seems to be a problem related to Akka. For example there is this PR to fix this: https://github.com/akka/akka/issues/16623 Can you check whether this should have been fixed in the Akka version we are using and post to the Akka ML if so?

Re: [DISCUSS] Consolidate method naming between the batch and streaming API

2015-06-01 Thread Gyula Fóra
+1 for the changes proposed by Marton (before the release) Aljoscha Krettek ezt írta (időpont: 2015. jún. 1., H, 16:32): > Yes, these renamings make sense. The partitionBy() is not yet in the > master for streaming, though. > > On Mon, Jun 1, 2015 at 4:10 PM, Márton Balassi > wrote: > > Looking

Re: Scala Tuple2 not recognized as a Scala Tuple

2015-06-01 Thread Pieter-Jan Van Aeken
Found the solution: Creating the TypeInformation in Scala and then passing it to Java where it is used explicitly in the returns method resolved the issue. Op Maandag, 01/06/2015 om 16:59 schreef Pieter-Jan Van Aeken: >From the Java API, I get an instance of_ DataSet_​ which I wrap with the metho

Re: MultipleLinearRegression - Strange results

2015-06-01 Thread Alexander Alexandrov
I've seen some work on adaptive learning rates in the past days. Maybe we can think about extending the base algorithm and comparing the use case setting for the IMPRO-3 project. @Felix you can discuss this with the others on Wednesday, Manu will be also there and can give some feedback, I'll try

Re: Scala Tuple2 not recognized as a Scala Tuple

2015-06-01 Thread Aljoscha Krettek
Great, good to hear! On Jun 1, 2015 5:39 PM, "Pieter-Jan Van Aeken" < pieterjan.vanae...@euranova.eu> wrote: > Found the solution: Creating the TypeInformation in Scala and then > passing it to Java where it is used explicitly in the returns method > resolved the issue. > > Op Maandag, 01/06/2015

Re: [DISCUSS] Consolidate method naming between the batch and streaming API

2015-06-01 Thread Fabian Hueske
Thanks for bringing up this point! +1 for the renaming. @Marton: Is this a "complete" list, i.e., did you go through both APIs or might there be more methods that are semantically identical but named differently? 2015-06-01 17:31 GMT+02:00 Gyula Fóra : > +1 for the changes proposed by Marton (be

Re: [DISCUSS] Consolidate method naming between the batch and streaming API

2015-06-01 Thread Stephan Ewen
+1 Good list and choices, Marton! On Mon, Jun 1, 2015 at 5:45 PM, Fabian Hueske wrote: > Thanks for bringing up this point! > > +1 for the renaming. > @Marton: Is this a "complete" list, i.e., did you go through both APIs or > might there be more methods that are semantically identical but name

Old new web interface

2015-06-01 Thread Matthias Schumacher
Hi all, last year I've been working on a web interface for Flink 0.7. It combines the web interface from the Job Manager and web client, an additional user management and scheduling jobs. I want to bring my work up to date and have heard it's a new web interface for the Job Manager in work.Co

Re: MultipleLinearRegression - Strange results

2015-06-01 Thread Sachin Goel
You can set the learning rate to be 1/sqrt(iteration number). This usually works. Regards Sachin Goel On Mon, Jun 1, 2015 at 9:09 PM, Alexander Alexandrov < alexander.s.alexand...@gmail.com> wrote: > I've seen some work on adaptive learning rates in the past days. > > Maybe we can think about ex

Re: [ANNOUNCE] Apache Mahout 0.10.1 Released

2015-06-01 Thread Dmitriy Lyubimov
we need to add published links to javadoc/scaladoc stuff. Nice job btw sorting this out. http://apache.github.io/mahout/0.10.1/docs/mahout-math/ http://apache.github.io/mahout/0.10.1/docs/mahout-math-scala

Re: [DISCUSS] Consolidate method naming between the batch and streaming API

2015-06-01 Thread Márton Balassi
@Fabian: I hope that this is the complete list, correct me f I am wrong. :) I am opening a small PR with the changes on top of Aljoscha's one that exposes the streaming partitioning then. On Mon, Jun 1, 2015 at 6:01 PM, Stephan Ewen wrote: > +1 > > Good list and choices, Marton! > > On Mon, Jun

Re: ClassNotFoundException : org.apache.flink.api.common.operators.util.UserCodeObjectWrapper, while trying to run locally

2015-06-01 Thread Robert Metzger
Hi Santosh, sorry for the late reply. I'm not 100% sure whats causing the issue. I'm not sure if the ThreadPoolExecutor (which seems to start a Flink LocalExecutor in the worker threads) is causing some issues. Are the executors starting local Flink instances in parallel? The last "Caused By" say

Re: Storm compatibility layer currently does not support Storm's SimpleJoin example

2015-06-01 Thread Robert Metzger
Great to see that you two are working together on the storm compatibility layer. Please let the other Flink committers know when Matthias PR is in a state that we can review it again (= when you think its ready). Given the feedback from Peter and the long list of missing features and the current r

Re: Get the current TM hostname from the ExecutionContext

2015-06-01 Thread Robert Metzger
+1 to add this. On Fri, May 29, 2015 at 12:02 PM, Ufuk Celebi wrote: > Hey Alexander, > > no, it's not possible at the moment. +1 for the idea. I also needed it > once. > > I think you will have to look at RuntimeEnvironment, Task, RuntimeContext. > > – Ufuk >

[jira] [Created] (FLINK-2128) ScalaShellITSuite failing

2015-06-01 Thread Ufuk Celebi (JIRA)
Ufuk Celebi created FLINK-2128: -- Summary: ScalaShellITSuite failing Key: FLINK-2128 URL: https://issues.apache.org/jira/browse/FLINK-2128 Project: Flink Issue Type: Bug Components: Sca

Re: ClassNotFoundException : org.apache.flink.api.common.operators.util.UserCodeObjectWrapper, while trying to run locally

2015-06-01 Thread Stephan Ewen
The stack trace looks really like the class "UserCodeObjectWrapper" was not found. That is a core Flink runtime class and the fact that it is missing indicates that there is something inconsistent with your libraries. Either some JAR files are missing or damaged. Did you use a downloaded binary d

[jira] [Created] (FLINK-2129) User state checkpointing works only for serializable state

2015-06-01 Thread Stephan Ewen (JIRA)
Stephan Ewen created FLINK-2129: --- Summary: User state checkpointing works only for serializable state Key: FLINK-2129 URL: https://issues.apache.org/jira/browse/FLINK-2129 Project: Flink Issue

[jira] [Created] (FLINK-2130) RabbitMQ source does not fail when failing to retrieve elements

2015-06-01 Thread Stephan Ewen (JIRA)
Stephan Ewen created FLINK-2130: --- Summary: RabbitMQ source does not fail when failing to retrieve elements Key: FLINK-2130 URL: https://issues.apache.org/jira/browse/FLINK-2130 Project: Flink

[jira] [Created] (FLINK-2131) Add Initialization schemes for K-means clustering

2015-06-01 Thread Sachin Goel (JIRA)
Sachin Goel created FLINK-2131: -- Summary: Add Initialization schemes for K-means clustering Key: FLINK-2131 URL: https://issues.apache.org/jira/browse/FLINK-2131 Project: Flink Issue Type: Task

Re: [DISCUSS] Consolidate method naming between the batch and streaming API

2015-06-01 Thread Szabó Péter
Great proposal! We should use consistent naming for the two API. Peter 2015-06-01 21:11 GMT+02:00 Márton Balassi : > @Fabian: I hope that this is the complete list, correct me f I am wrong. :) > > I am opening a small PR with the changes on top of Aljoscha's one that > exposes the streaming part