[jira] [Created] (FLINK-6324) Refine state access methods in OperatorStateStore

2017-04-18 Thread Tzu-Li (Gordon) Tai (JIRA)
Tzu-Li (Gordon) Tai created FLINK-6324:
--

 Summary: Refine state access methods in OperatorStateStore
 Key: FLINK-6324
 URL: https://issues.apache.org/jira/browse/FLINK-6324
 Project: Flink
  Issue Type: Improvement
  Components: DataStream API, State Backends, Checkpointing
Reporter: Tzu-Li (Gordon) Tai
Assignee: Tzu-Li (Gordon) Tai


This proposes to refine the OperatorStateStore interface by,
- deprecating Java serialization shortcuts
- rename getOperatorState to getListState

The Java serialization shortcuts can be deprecated because they were
previously introduced to provide a smoother migration path from older
savepoints. However, its usage should definitely be discouraged.

Renaming to {{getListState}} is a preparation of making the names of state
access methods contain information about both its redistribution pattern
on restore and the shape of its data structure, since the combination of
these two is orthogonal. This convention will also provide a better
naming pattern for more state access methods in the future, for example
{{getUnionListState}}. If the method name does not contain its
redistribution pattern (e.g., {{getListState}}), then it simply implies the
default repartitioning scheme (SPLIT_DISTRIBUTE).



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


Re: [DISCUSS] Code style / checkstyle

2017-04-18 Thread Chesnay Schepler
+1 to use earth rotation as the new standard time unit. Maybe more 
importantly, I'm absolutely in favor of merging it.


On 18.04.2017 20:39, Aljoscha Krettek wrote:

I rebased the PR [1] on current master. Is there any strong objection against 
merging this (minus the two last commits which introduce curly-brace-style 
checking). If not, I would like to merge this after two earth rotations, i.e. 
after all the time zones have had some time to react.

The complete set of checks has been listed by Chesnay (via Greg) before but the 
gist of it is that we only add common-sense checks that most people should be 
able to agree upon so that we avoid edit wars (especially when it comes to 
whitespace, import order and Javadoc paragraph styling).

[1] https://github.com/apache/flink/pull/3567

On 5. Apr 2017, at 23:54, Chesnay Schepler  wrote:

I agree to just allow both. While I definitely prefer 1) i can see why someone 
might prefer 2).

Wouldn't want to delay this anymore; can't find to add this to flink-metrics 
and flink-python...

On 03.04.2017 18:33, Aljoscha Krettek wrote:

I think enough people did already look at the checkstyle rules proposed in the 
PR.

On most of the rules reaching consensus is easy so I propose to enable all 
rules except those regarding placement of curly braces and control flow 
formatting. The two styles in the Flink code base are:

1)
if () {
} else {
}

try {
} catch () {
}

and

2)

if () {
}
else {
}

try {
}
catch () {
}

I think it’s hard to reach consensus on these so I suggest to keep allowing 
both styles.

Any comments very welcome! :-)

Best,
Aljoscha

On 19. Mar 2017, at 17:09, Aljoscha Krettek  wrote:

I played around with this over the week end and it turns out that the required 
changes in flink-streaming-java are not that big. I created a PR with a proposed 
checkstyle.xml and the required code changes: 
https://github.com/apache/flink/pull/3567 
. There’s a longer description of 
the style in the PR. The commits add continuously more invasive changes so we can 
start with the more lightweight changes if we want to.

If we want to go forward with this I would also encourage other people to use 
this for different modules and see how it turns out.

Best,
Aljoscha


On 18 Mar 2017, at 08:00, Aljoscha Krettek mailto:aljos...@apache.org>> wrote:

I added an issue for adding a custom checkstyle.xml for flink-streaming-java so that 
we can gradually add checks: https://issues.apache.org/jira/browse/FLINK-6107 
. I outlined the procedure in 
the Jira. We can use this as a pilot project and see how it goes and then gradually 
also apply to other modules.

What do you think?


On 6 Mar 2017, at 12:42, Stephan Ewen mailto:se...@apache.org>> wrote:

A singular "all reformat in one instant" will cause immense damage to the
project, in my opinion.

- There are so many pull requests that we are having a hard time keeping
up, and merging will a lot more time intensive.
- I personally have many forked branches with WIP features that will
probably never go in if the branches become unmergeable. I expect that to
be true for many other committers and contributors.
- Some companies have Flink forks and are rebasing patches onto master
regularly. They will be completely screwed by a full reformat.

If we do something, the only thing that really is possible is:

(1) Define a style. Ideally not too far away from Flink's style.
(2) Apply it to new projects/modules
(3) Coordinate carefully to pull it into existing modules, module by
module. Leaving time to adopt pull requests bit by bit, and allowing forks
to go through minor merges, rather than total conflict.



On Wed, Mar 1, 2017 at 5:57 PM, Henry Saputra mailto:henry.sapu...@gmail.com>>
wrote:


It is actually Databricks Scala guide to help contributions to Apache Spark
so not really official Spark Scala guide.
The style guide feels bit more like asking people to write Scala in Java
mode so I am -1 to follow the style for Apache Flink Scala if that what you
are recommending.

If the "unification" means ONE style guide for both Java and Scala I would
vote -1 to it because both languages have different semantics and styles to
make them readable and understandable.

We could start with improving the Scala maven style guide to follow more
Scala official style guide [1] and add IntelliJ Idea or Eclipse plugin
style to follow suit.

Java side has bit more strict style check but we could make it tighter but
embracing more Google Java guide closely with minor exceptions

- Henry

[1] http://docs.scala-lang.org/style/ 

On Mon, Feb 27, 2017 at 11:54 AM, Stavros Kontopoulos <
st.kontopou...@gmail.com > wrote:


+1 to provide and enforcing a unified code style for both java and scala.
Unification should apply when it makes sense like comments though.

Eventually code base should be re-

Re: Rescaling state

2017-04-18 Thread Aljoscha Krettek
Hi,
yes, right now this can only happen when a Job is stopped and restarted from a 
savepoint. In the future we might be able to do it without restarting but I 
think that is quite a ways in the future.

Best,
Aljoscha
> On 18. Apr 2017, at 20:12, Vishnu Viswanath  
> wrote:
> 
> Hi All,
> 
> Regarding the scale up and down feature mentioned in this talk:
> https://www.youtube.com/watch?v=IHSMnlWXkZ4&list=PLDX4T_cnKjD2UC6wJr_wRbIvtlMtkc-n2&index=18
> 
> Can this only be done when restarting the job from a save point? Is there a
> case where the state is transferred over from one node to another during
> the job execution.
> 
> Thanks,
> Vishnu



Re: [DISCUSS] Code style / checkstyle

2017-04-18 Thread Aljoscha Krettek
I rebased the PR [1] on current master. Is there any strong objection against 
merging this (minus the two last commits which introduce curly-brace-style 
checking). If not, I would like to merge this after two earth rotations, i.e. 
after all the time zones have had some time to react.

The complete set of checks has been listed by Chesnay (via Greg) before but the 
gist of it is that we only add common-sense checks that most people should be 
able to agree upon so that we avoid edit wars (especially when it comes to 
whitespace, import order and Javadoc paragraph styling).

[1] https://github.com/apache/flink/pull/3567
> On 5. Apr 2017, at 23:54, Chesnay Schepler  wrote:
> 
> I agree to just allow both. While I definitely prefer 1) i can see why 
> someone might prefer 2).
> 
> Wouldn't want to delay this anymore; can't find to add this to flink-metrics 
> and flink-python...
> 
> On 03.04.2017 18:33, Aljoscha Krettek wrote:
>> I think enough people did already look at the checkstyle rules proposed in 
>> the PR.
>> 
>> On most of the rules reaching consensus is easy so I propose to enable all 
>> rules except those regarding placement of curly braces and control flow 
>> formatting. The two styles in the Flink code base are:
>> 
>> 1)
>> if () {
>> } else {
>> }
>> 
>> try {
>> } catch () {
>> }
>> 
>> and
>> 
>> 2)
>> 
>> if () {
>> }
>> else {
>> }
>> 
>> try {
>> }
>> catch () {
>> }
>> 
>> I think it’s hard to reach consensus on these so I suggest to keep allowing 
>> both styles.
>> 
>> Any comments very welcome! :-)
>> 
>> Best,
>> Aljoscha
>>> On 19. Mar 2017, at 17:09, Aljoscha Krettek  wrote:
>>> 
>>> I played around with this over the week end and it turns out that the 
>>> required changes in flink-streaming-java are not that big. I created a PR 
>>> with a proposed checkstyle.xml and the required code changes: 
>>> https://github.com/apache/flink/pull/3567 
>>> . There’s a longer description 
>>> of the style in the PR. The commits add continuously more invasive changes 
>>> so we can start with the more lightweight changes if we want to.
>>> 
>>> If we want to go forward with this I would also encourage other people to 
>>> use this for different modules and see how it turns out.
>>> 
>>> Best,
>>> Aljoscha
>>> 
 On 18 Mar 2017, at 08:00, Aljoscha Krettek >>> > wrote:
 
 I added an issue for adding a custom checkstyle.xml for 
 flink-streaming-java so that we can gradually add checks: 
 https://issues.apache.org/jira/browse/FLINK-6107 
 . I outlined the 
 procedure in the Jira. We can use this as a pilot project and see how it 
 goes and then gradually also apply to other modules.
 
 What do you think?
 
> On 6 Mar 2017, at 12:42, Stephan Ewen  > wrote:
> 
> A singular "all reformat in one instant" will cause immense damage to the
> project, in my opinion.
> 
> - There are so many pull requests that we are having a hard time keeping
> up, and merging will a lot more time intensive.
> - I personally have many forked branches with WIP features that will
> probably never go in if the branches become unmergeable. I expect that to
> be true for many other committers and contributors.
> - Some companies have Flink forks and are rebasing patches onto master
> regularly. They will be completely screwed by a full reformat.
> 
> If we do something, the only thing that really is possible is:
> 
> (1) Define a style. Ideally not too far away from Flink's style.
> (2) Apply it to new projects/modules
> (3) Coordinate carefully to pull it into existing modules, module by
> module. Leaving time to adopt pull requests bit by bit, and allowing forks
> to go through minor merges, rather than total conflict.
> 
> 
> 
> On Wed, Mar 1, 2017 at 5:57 PM, Henry Saputra  >
> wrote:
> 
>> It is actually Databricks Scala guide to help contributions to Apache 
>> Spark
>> so not really official Spark Scala guide.
>> The style guide feels bit more like asking people to write Scala in Java
>> mode so I am -1 to follow the style for Apache Flink Scala if that what 
>> you
>> are recommending.
>> 
>> If the "unification" means ONE style guide for both Java and Scala I 
>> would
>> vote -1 to it because both languages have different semantics and styles 
>> to
>> make them readable and understandable.
>> 
>> We could start with improving the Scala maven style guide to follow more
>> Scala official style guide [1] and add IntelliJ Idea or Eclipse plugin
>> style to follow suit.
>> 
>> Java side has bit more strict style check but we could make it tighter 
>> but
>> embracing more Google Java guide closely with min

Rescaling state

2017-04-18 Thread Vishnu Viswanath
Hi All,

Regarding the scale up and down feature mentioned in this talk:
https://www.youtube.com/watch?v=IHSMnlWXkZ4&list=PLDX4T_cnKjD2UC6wJr_wRbIvtlMtkc-n2&index=18

Can this only be done when restarting the job from a save point? Is there a
case where the state is transferred over from one node to another during
the job execution.

Thanks,
Vishnu


[jira] [Created] (FLINK-6323) Add new Sink interface with access to more meta data

2017-04-18 Thread Aljoscha Krettek (JIRA)
Aljoscha Krettek created FLINK-6323:
---

 Summary: Add new Sink interface with access to more meta data
 Key: FLINK-6323
 URL: https://issues.apache.org/jira/browse/FLINK-6323
 Project: Flink
  Issue Type: Improvement
  Components: DataStream API
Reporter: Aljoscha Krettek


The current {{SinkFunction}} cannot access the timestamps of elements which 
resulted in the (somewhat hacky) {{FlinkKafkaProducer010}}. Due to other 
limitations {{GenericWriteAheadSink}} is currently also a {{StreamOperator}} 
and not a {{SinkFunction}}.

We should add a new interface for sinks that takes a context parameter, similar 
to {{ProcessFunction}}. This will allow sinks to query additional meta data 
about the element that they're receiving. 

This is one ML thread where a user ran into a problem caused by this: 
http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Why-I-am-getting-Null-pointer-exception-while-accessing-RuntimeContext-in-FlinkKafkaProducer010-td12633.html#a12635



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Created] (FLINK-6322) Mesos task labels

2017-04-18 Thread Eron Wright (JIRA)
Eron Wright  created FLINK-6322:
---

 Summary: Mesos task labels
 Key: FLINK-6322
 URL: https://issues.apache.org/jira/browse/FLINK-6322
 Project: Flink
  Issue Type: Improvement
  Components: Mesos
Reporter: Eron Wright 
Priority: Minor



Task labels serve many purposes in Mesos, such a tagging tasks for 
log-aggregation purposes.   

I propose a new configuration setting for a list of 'key=value' labels to be 
applied to TM instances.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Created] (FLINK-6321) RocksDB state backend Checkpointing is not working with KeydCEP.

2017-04-18 Thread Shashank Agarwal (JIRA)
Shashank Agarwal created FLINK-6321:
---

 Summary: RocksDB state backend Checkpointing is not working with 
KeydCEP.
 Key: FLINK-6321
 URL: https://issues.apache.org/jira/browse/FLINK-6321
 Project: Flink
  Issue Type: Bug
  Components: CEP, State Backends, Checkpointing
Affects Versions: 1.2.0
 Environment: yarn-cluster, RocksDB State backend, Checkpointing every 
1000 ms
Reporter: Shashank Agarwal


Checkpointing is not working with RocksDBStateBackend when using CEP. It's 
working fine with FsStateBackend and MemoryStateBackend. Application failing 
every-time.

'''
04/18/2017 21:53:20 Job execution switched to status FAILING.
AsynchronousException{java.lang.Exception: Could not materialize checkpoint 46 
for operator KeyedCEPPatternOperator -> Map (1/4).}
at 
org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:980)
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.Exception: Could not materialize checkpoint 46 for 
operator KeyedCEPPatternOperator -> Map (1/4).
... 6 more
Caused by: java.util.concurrent.CancellationException
at java.util.concurrent.FutureTask.report(FutureTask.java:121)
at java.util.concurrent.FutureTask.get(FutureTask.java:192)
at 
org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:40)
at 
org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:915)
... 5 more

'''





--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Created] (FLINK-6320) Flakey JobManagerHAJobGraphRecoveryITCase

2017-04-18 Thread Nico Kruber (JIRA)
Nico Kruber created FLINK-6320:
--

 Summary: Flakey JobManagerHAJobGraphRecoveryITCase
 Key: FLINK-6320
 URL: https://issues.apache.org/jira/browse/FLINK-6320
 Project: Flink
  Issue Type: Bug
  Components: Tests
Affects Versions: 1.3.0
Reporter: Nico Kruber


it looks as if there is a race condition in the cleanup of 
{{JobManagerHAJobGraphRecoveryITCase}}.

{code}
Tests run: 2, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 50.271 sec <<< 
FAILURE! - in org.apache.flink.test.recovery.JobManagerHAJobGraphRecoveryITCase
testJobPersistencyWhenJobManagerShutdown(org.apache.flink.test.recovery.JobManagerHAJobGraphRecoveryITCase)
  Time elapsed: 0.129 sec  <<< ERROR!
java.io.FileNotFoundException: File does not exist: 
/tmp/9b63934b-789d-428c-aa9e-47d5d8fa1e32/recovery/submittedJobGraphf763d61fba47
at org.apache.commons.io.FileUtils.forceDelete(FileUtils.java:2275)
at org.apache.commons.io.FileUtils.cleanDirectory(FileUtils.java:1653)
at org.apache.commons.io.FileUtils.deleteDirectory(FileUtils.java:1535)
at org.apache.commons.io.FileUtils.forceDelete(FileUtils.java:2270)
at org.apache.commons.io.FileUtils.cleanDirectory(FileUtils.java:1653)
at 
org.apache.flink.test.recovery.JobManagerHAJobGraphRecoveryITCase.cleanUp(JobManagerHAJobGraphRecoveryITCase.java:112)
{code}
Full log: https://s3.amazonaws.com/archive.travis-ci.org/jobs/223124016/log.txt

Maybe a rule-based temporary directory is a better solution:
{code:java}
@Rule
public TemporaryFolder tempFolder = new TemporaryFolder();
{code}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Created] (FLINK-6319) Add timeout when shutting SystemProcessingTimeService down

2017-04-18 Thread Till Rohrmann (JIRA)
Till Rohrmann created FLINK-6319:


 Summary: Add timeout when shutting SystemProcessingTimeService down
 Key: FLINK-6319
 URL: https://issues.apache.org/jira/browse/FLINK-6319
 Project: Flink
  Issue Type: Improvement
  Components: Local Runtime
Affects Versions: 1.3.0
Reporter: Till Rohrmann
Priority: Minor


A user noted that we simply call {{shutdownNow}} on the 
{{SystemProcessingTimeService's}} {{ScheduledThreadpoolExecutor}} when calling 
{{SystemProcessingTimeService.shutdownService}}. {{shutdowNow}} will halt all 
waiting tasks but it won't wait until the currently running tasks have been 
completed. This can lead to unwanted runtime behaviours such as wrong 
termination orders when shutting down tasks (as reported in FLINK-4973).

I propose to add a small timeout to wait for currently running tasks to 
complete. Even though this problem cannot be completely solved since timer 
tasks might take longer than the specified timeout, a timeout for waiting for 
running tasks to complete will mitigate the problem.

We can do this by calling {{timerServicer.awaitTermination(timeout, 
timeoutUnit);}} after the {{shutdowNow}} call.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Created] (FLINK-6318) NFA serialisation doesn't work with TypeSerializers that load classes

2017-04-18 Thread Aljoscha Krettek (JIRA)
Aljoscha Krettek created FLINK-6318:
---

 Summary: NFA serialisation doesn't work with TypeSerializers that 
load classes
 Key: FLINK-6318
 URL: https://issues.apache.org/jira/browse/FLINK-6318
 Project: Flink
  Issue Type: Improvement
  Components: CEP
Reporter: Aljoscha Krettek


This is from a user reported stack trace:

{code}
04/12/2017 10:05:04 Job execution switched to status FAILING.
java.lang.RuntimeException: Could not deserialize NFA.
at org.apache.flink.cep.nfa.NFA$Serializer.deserialize(NFA.java:538)
at org.apache.flink.cep.nfa.NFA$Serializer.deserialize(NFA.java:469)
at 
org.apache.flink.contrib.streaming.state.RocksDBValueState.value(RocksDBValueState.java:81)
at 
org.apache.flink.cep.operator.AbstractKeyedCEPPatternOperator.getNFA(AbstractKeyedCEPPatternOperator.java:124)
at 
org.apache.flink.cep.operator.AbstractCEPBasePatternOperator.processElement(AbstractCEPBasePatternOperator.java:72)
at 
org.apache.flink.cep.operator.AbstractKeyedCEPPatternOperator.processElement(AbstractKeyedCEPPatternOperator.java:162)
at 
org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:185)
at 
org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:63)
at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:272)
at org.apache.flink.runtime.taskmanager.Task.run(Task.java:655)
at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.ClassNotFoundException: 
co.ronak.nto.Job$$anon$18$$anon$21$$anon$3
at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:331)
at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
at java.lang.Class.forName0(Native Method)
at java.lang.Class.forName(Class.java:348)
at java.io.ObjectInputStream.resolveClass(ObjectInputStream.java:626)
at 
java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1613)
at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1518)
at 
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1774)
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351)
at 
java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2000)
at 
java.io.ObjectInputStream.defaultReadObject(ObjectInputStream.java:501)
at 
org.apache.flink.api.scala.typeutils.TraversableSerializer.readObject(TraversableSerializer.scala:53)
at sun.reflect.GeneratedMethodAccessor5.invoke(Unknown Source)
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:497)
at 
java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass.java:1017)
at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1900)
at 
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1801)
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351)
at 
java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2000)
at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1924)
at 
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1801)
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351)
at java.io.ObjectInputStream.readArray(ObjectInputStream.java:1707)
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1345)
at 
java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2000)
at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1924)
at 
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1801)
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351)
at 
java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2000)
at 
java.io.ObjectInputStream.defaultReadObject(ObjectInputStream.java:501)
at 
org.apache.flink.cep.NonDuplicatingTypeSerializer.readObject(NonDuplicatingTypeSerializer.java:190)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:497)
at 
java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass.java:1017)
at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1900)
at 
java.io.ObjectInputStream.readOrdinaryObject(Objec

[jira] [Created] (FLINK-6317) History server - wrong default directory

2017-04-18 Thread JIRA
Lorenz Bühmann created FLINK-6317:
-

 Summary: History server - wrong default directory
 Key: FLINK-6317
 URL: https://issues.apache.org/jira/browse/FLINK-6317
 Project: Flink
  Issue Type: Bug
  Components: Web Client
Affects Versions: 1.2.0
Reporter: Lorenz Bühmann
Priority: Minor


When the history server is started without a directory specified in the 
configuration file, it will use some random directory located in the Java Temp 
directory. Unfortunately, a file separator is missing:

{code:title=HistoryServer.java@L139-L143|borderStyle=solid}
String webDirectory = 
config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR);
if (webDirectory == null) {
webDirectory = System.getProperty("java.io.tmpdir") + 
"flink-web-history-" + UUID.randomUUID();
}
webDir = new File(webDirectory);
{code}

It should be 

{code}
webDirectory = System.getProperty("java.io.tmpdir") + File.separator +  
"flink-web-history-" + UUID.randomUUID();
{code}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)