You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by StephanEwen <gi...@git.apache.org> on 2018/02/19 18:15:16 UTC

[GitHub] flink pull request #5528: [FLINK-6489] [FLINK-8696] [shell scripts] Remove J...

GitHub user StephanEwen opened a pull request:

    https://github.com/apache/flink/pull/5528

    [FLINK-6489] [FLINK-8696] [shell scripts] Remove JobManager local mode from the Shell Scripts

    ## What is the purpose of the change
    
    The JobManager local mode is problematic:
      - The JobManager does not start all components properly, making some operations fail/timeout unexpectedly
      - It introduces code complexity in maintaining the JobManager entry point that also spawns an embedded TaskManager
    
    The JobManager local mode is also unnecessary. The `start-cluster.sh` script sets up a proper local cluster without additional configuration.
    
    This pull request removes the local mode from all UNIX scripts and changes the Windows `start-local.bat` to a  `start-cluster.bat` that also starts two separate processes for JobManager and TaskManager.
    
    ## Brief change log
    
      - Changes the `start-local.bat` to a  `start-cluster.bat` which spawns two processes (using `start` command). The two processes show up as separate windows and can be stopped separately.
      - Removes the `start-local-sh` and `stop-local.sh` scripts.
      - Removes the passing of the *local* parameter between the shell scripts and always passes *cluster* as the execution mode parameter to the JobManager.
    
    The next step would be to remove the execution mode from the JobManagers command line argument parsing and setup logic.
    
    ## Verifying this change
    
      - Building Flink, starting a cluster via the shell scripts (Linux, MacOS, Windows) and checking the WebUI that the cluster is properly up. No actual job execution needed.
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): (yes / **no)**
      - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / **no)**
      - The serializers: (yes / **no** / don't know)
      - The runtime per-record code paths (performance sensitive): (yes / **no** / don't know)
      - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes / **no** / don't know)
      - The S3 file system connector: (yes / **no** / don't know)
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (yes / **no)**
      - If yes, how is the feature documented? (not applicable / **docs** / JavaDocs / not documented)
    
    Docs are updated to not refer to `start-local.xy` any more.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/StephanEwen/incubator-flink remove_local_mode

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/5528.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #5528
    
----
commit 99d564928f8299d2ad868ed5c23a9c88d8899a47
Author: Stephan Ewen <se...@...>
Date:   2018-02-16T20:14:41Z

    [FLINK-6489] [shell scripts] Remove local mode from Windows start-local.bat
    
    Instead, this uses 'start' to start JobManager and TaskManager background processes.

commit f142deebd951a498f2247b2ce243c15b19a81e84
Author: Stephan Ewen <se...@...>
Date:   2018-02-16T20:15:49Z

    [FLINK-8696] [shell scripts] Remove JobManager local mode from shell scripts

commit 11e87c4e2e879afb753354660ce3e33fb38a60d8
Author: Stephan Ewen <se...@...>
Date:   2018-02-19T18:13:12Z

    [FLINK-6489] [FLINK-8696] [docs] Update docs to use 'start-cluster.sh' instead of 'start-local.sh'
    
    (and likewise for 'start-cluster.bat' vs. 'start-local.bat')

----


---

[GitHub] flink pull request #5528: [FLINK-6489] [FLINK-8696] [shell scripts] Remove J...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5528#discussion_r169595831
  
    --- Diff: flink-dist/src/main/flink-bin/bin/start-cluster.bat ---
    @@ -0,0 +1,77 @@
    +::###############################################################################
    +::  Licensed to the Apache Software Foundation (ASF) under one
    +::  or more contributor license agreements.  See the NOTICE file
    +::  distributed with this work for additional information
    +::  regarding copyright ownership.  The ASF licenses this file
    +::  to you under the Apache License, Version 2.0 (the
    +::  "License"); you may not use this file except in compliance
    +::  with the License.  You may obtain a copy of the License at
    +::
    +::      http://www.apache.org/licenses/LICENSE-2.0
    +::
    +::  Unless required by applicable law or agreed to in writing, software
    +::  distributed under the License is distributed on an "AS IS" BASIS,
    +::  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +::  See the License for the specific language governing permissions and
    +:: limitations under the License.
    +::###############################################################################
    +
    +@echo off
    +setlocal EnableDelayedExpansion
    +
    +SET bin=%~dp0
    +SET FLINK_ROOT_DIR=%bin%..
    +SET FLINK_LIB_DIR=%FLINK_ROOT_DIR%\lib
    +SET FLINK_CONF_DIR=%FLINK_ROOT_DIR%\conf
    +SET FLINK_LOG_DIR=%FLINK_ROOT_DIR%\log
    +
    +SET JVM_ARGS=-Xms1024m -Xmx1024m
    +
    +SET FLINK_CLASSPATH=%FLINK_LIB_DIR%\*
    +
    +SET logname_jm=flink-%username%-jobmanager-%computername%.log
    +SET logname_tm=flink-%username%-taskmanager-%computername%.log
    +SET log_jm=%FLINK_LOG_DIR%\%logname_jm%
    +SET log_tm=%FLINK_LOG_DIR%\%logname_tm%
    +SET outname_jm=flink-%username%-jobmanager-%computername%.out
    +SET outname_tm=flink-%username%-taskmanager-%computername%.out
    +SET out_jm=%FLINK_LOG_DIR%\%outname_jm%
    +SET out_tm=%FLINK_LOG_DIR%\%outname_tm%
    +
    +SET log_setting_jm=-Dlog.file="%log_jm%" -Dlogback.configurationFile=file:"%FLINK_CONF_DIR%/logback.xml" -Dlog4j.configuration=file:"%FLINK_CONF_DIR%/log4j.properties"
    +SET log_setting_tm=-Dlog.file="%log_tm%" -Dlogback.configurationFile=file:"%FLINK_CONF_DIR%/logback.xml" -Dlog4j.configuration=file:"%FLINK_CONF_DIR%/log4j.properties"
    +
    +:: Log rotation (quick and dirty)
    +CD "%FLINK_LOG_DIR%"
    +for /l %%x in (5, -1, 1) do ( 
    +SET /A y = %%x+1 
    +RENAME "%logname_jm%.%%x" "%logname_jm%.!y!" 2> nul
    +RENAME "%logname_tm%.%%x" "%logname_tm%.!y!" 2> nul
    +RENAME "%outname_jm%.%%x" "%outname_jm%.!y!"  2> nul
    +RENAME "%outname_tm%.%%x" "%outname_tm%.!y!"  2> nul
    +)
    +RENAME "%logname_jm%" "%logname_jm%.0"  2> nul
    +RENAME "%logname_tm%" "%logname_tm%.0"  2> nul
    +RENAME "%outname_jm%" "%outname_jm%.0"  2> nul
    +RENAME "%outname_tm%" "%outname_tm%.0"  2> nul
    +DEL "%logname_jm%.6"  2> nul
    +DEL "%logname_tm%.6"  2> nul
    +DEL "%outname_jm%.6"  2> nul
    +DEL "%outname_tm%.6"  2> nul
    +
    +for %%X in (java.exe) do (set FOUND=%%~$PATH:X)
    +if not defined FOUND (
    +    echo java.exe was not found in PATH variable
    +    goto :eof
    +)
    +
    +echo Starting a local cluster with one JobManager process and one TaskManager process.
    +
    +echo You can terminate the processes via CTRL-C in the spawned shell windows.
    +
    +echo Web interface by default on http://localhost:8081/.
    --- End diff --
    
    This should be updated to `9065`.


---

[GitHub] flink issue #5528: [FLINK-6489] [FLINK-8696] [shell scripts] Remove JobManag...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on the issue:

    https://github.com/apache/flink/pull/5528
  
    There's still a reference to `start-local.sh` in `common.sh`:
    ```
    function start_cluster {
      if [[ "$CLUSTER_MODE" == "local" ]]; then
        $FLINK_DIR/bin/start-local.sh
      elif [[ "$CLUSTER_MODE" == "cluster" ]]; then
        $FLINK_DIR/bin/start-cluster.sh
      else
        echo "Unrecognized cluster mode: $CLUSTER_MODE"
        exit
      fi
    ```


---

[GitHub] flink pull request #5528: [FLINK-6489] [FLINK-8696] [shell scripts] Remove J...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5528#discussion_r169622022
  
    --- Diff: flink-dist/src/main/flink-bin/bin/start-cluster.bat ---
    @@ -0,0 +1,77 @@
    +::###############################################################################
    +::  Licensed to the Apache Software Foundation (ASF) under one
    +::  or more contributor license agreements.  See the NOTICE file
    +::  distributed with this work for additional information
    +::  regarding copyright ownership.  The ASF licenses this file
    +::  to you under the Apache License, Version 2.0 (the
    +::  "License"); you may not use this file except in compliance
    +::  with the License.  You may obtain a copy of the License at
    +::
    +::      http://www.apache.org/licenses/LICENSE-2.0
    +::
    +::  Unless required by applicable law or agreed to in writing, software
    +::  distributed under the License is distributed on an "AS IS" BASIS,
    +::  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +::  See the License for the specific language governing permissions and
    +:: limitations under the License.
    +::###############################################################################
    +
    +@echo off
    +setlocal EnableDelayedExpansion
    +
    +SET bin=%~dp0
    +SET FLINK_ROOT_DIR=%bin%..
    +SET FLINK_LIB_DIR=%FLINK_ROOT_DIR%\lib
    +SET FLINK_CONF_DIR=%FLINK_ROOT_DIR%\conf
    +SET FLINK_LOG_DIR=%FLINK_ROOT_DIR%\log
    +
    +SET JVM_ARGS=-Xms1024m -Xmx1024m
    +
    +SET FLINK_CLASSPATH=%FLINK_LIB_DIR%\*
    +
    +SET logname_jm=flink-%username%-jobmanager-%computername%.log
    +SET logname_tm=flink-%username%-taskmanager-%computername%.log
    +SET log_jm=%FLINK_LOG_DIR%\%logname_jm%
    +SET log_tm=%FLINK_LOG_DIR%\%logname_tm%
    +SET outname_jm=flink-%username%-jobmanager-%computername%.out
    +SET outname_tm=flink-%username%-taskmanager-%computername%.out
    +SET out_jm=%FLINK_LOG_DIR%\%outname_jm%
    +SET out_tm=%FLINK_LOG_DIR%\%outname_tm%
    +
    +SET log_setting_jm=-Dlog.file="%log_jm%" -Dlogback.configurationFile=file:"%FLINK_CONF_DIR%/logback.xml" -Dlog4j.configuration=file:"%FLINK_CONF_DIR%/log4j.properties"
    +SET log_setting_tm=-Dlog.file="%log_tm%" -Dlogback.configurationFile=file:"%FLINK_CONF_DIR%/logback.xml" -Dlog4j.configuration=file:"%FLINK_CONF_DIR%/log4j.properties"
    +
    +:: Log rotation (quick and dirty)
    +CD "%FLINK_LOG_DIR%"
    +for /l %%x in (5, -1, 1) do ( 
    +SET /A y = %%x+1 
    +RENAME "%logname_jm%.%%x" "%logname_jm%.!y!" 2> nul
    +RENAME "%logname_tm%.%%x" "%logname_tm%.!y!" 2> nul
    +RENAME "%outname_jm%.%%x" "%outname_jm%.!y!"  2> nul
    +RENAME "%outname_tm%.%%x" "%outname_tm%.!y!"  2> nul
    +)
    +RENAME "%logname_jm%" "%logname_jm%.0"  2> nul
    +RENAME "%logname_tm%" "%logname_tm%.0"  2> nul
    +RENAME "%outname_jm%" "%outname_jm%.0"  2> nul
    +RENAME "%outname_tm%" "%outname_tm%.0"  2> nul
    +DEL "%logname_jm%.6"  2> nul
    +DEL "%logname_tm%.6"  2> nul
    +DEL "%outname_jm%.6"  2> nul
    +DEL "%outname_tm%.6"  2> nul
    +
    +for %%X in (java.exe) do (set FOUND=%%~$PATH:X)
    +if not defined FOUND (
    +    echo java.exe was not found in PATH variable
    +    goto :eof
    +)
    +
    +echo Starting a local cluster with one JobManager process and one TaskManager process.
    +
    +echo You can terminate the processes via CTRL-C in the spawned shell windows.
    +
    +echo Web interface by default on http://localhost:8081/.
    --- End diff --
    
    ok 👍 


---

[GitHub] flink issue #5528: [FLINK-6489] [FLINK-8696] [shell scripts] Remove JobManag...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the issue:

    https://github.com/apache/flink/pull/5528
  
    Merged in 74c5570c9fa94d35e47899b0dcdc74a5d18750f6


---

[GitHub] flink issue #5528: [FLINK-6489] [FLINK-8696] [shell scripts] Remove JobManag...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on the issue:

    https://github.com/apache/flink/pull/5528
  
    I've tried using the start-local.bat scripts and found 2 issues:
    1)
    The following warning is printed in the JobManager and TaskManager windows:
    ```
    WARNING: An illegal reflective access operation has occurred
    WARNING: Illegal reflective access by org.apache.hadoop.security.authentication.util.KerberosUtil (file:/C:/Users/Zento/Documents/GitHub/flink/flink-dist/target/flink-1.5-SNAPSHOT-bin/flink-1.5-SNAPSHOT/lib/flink-shaded-hadoop2-uber-1.5-SNAPSHOT.jar) to method sun.security.krb5.Config.getInstance()
    WARNING: Please consider reporting this to the maintainers of org.apache.hadoop.security.authentication.util.KerberosUtil
    WARNING: Use --illegal-access=warn to enable warnings of further illegal reflective access operations
    WARNING: All illegal access operations will be denied in a future release
    ```
    2)
    You cannot use them out-of-the-box as they start the old JobManager/TaskManager components, but the client works against FLIP-6 by default.
    3)
    After explicitly configuring `mode: "old"` in `flink-conf.yaml` the job submissions times out. In the JM/TM logs I found this funky exception:
    ```
    2018-02-21 10:44:09,363 ERROR akka.remote.Remoting                                          - [B cannot be cast to [C
    java.lang.ClassCastException: [B cannot be cast to [C
            at akka.remote.artery.FastHash$.ofString(LruBoundedCache.scala:18)
            at akka.remote.serialization.ActorRefResolveCache.hash(ActorRefResolveCache.scala:61)
            at akka.remote.serialization.ActorRefResolveCache.hash(ActorRefResolveCache.scala:55)
            at akka.remote.artery.LruBoundedCache.getOrCompute(LruBoundedCache.scala:110)
            at akka.remote.RemoteActorRefProvider.resolveActorRef(RemoteActorRefProvider.scala:403)
            at akka.actor.SerializedActorRef.readResolve(ActorRef.scala:433)
            at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
            at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(Unknown Source)
            at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(Unknown Source)
            at java.base/java.lang.reflect.Method.invoke(Unknown Source)
            at java.base/java.io.ObjectStreamClass.invokeReadResolve(Unknown Source)
            at java.base/java.io.ObjectInputStream.readOrdinaryObject(Unknown Source)
            at java.base/java.io.ObjectInputStream.readObject0(Unknown Source)
            at java.base/java.io.ObjectInputStream.defaultReadFields(Unknown Source)
            at java.base/java.io.ObjectInputStream.readSerialData(Unknown Source)
            at java.base/java.io.ObjectInputStream.readOrdinaryObject(Unknown Source)
            at java.base/java.io.ObjectInputStream.readObject0(Unknown Source)
            at java.base/java.io.ObjectInputStream.readObject(Unknown Source)
            at akka.serialization.JavaSerializer$$anonfun$1.apply(Serializer.scala:328)
            at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
            at akka.serialization.JavaSerializer.fromBinary(Serializer.scala:328)
            at akka.serialization.Serialization.akka$serialization$Serialization$$deserializeByteArray(Serialization.scala:156)
            at akka.serialization.Serialization$$anonfun$deserialize$2.apply(Serialization.scala:142)
            at scala.util.Try$.apply(Try.scala:192)
            at akka.serialization.Serialization.deserialize(Serialization.scala:136)
            at akka.remote.MessageSerializer$.deserialize(MessageSerializer.scala:30)
            at akka.remote.DefaultMessageDispatcher.payload$lzycompute$1(Endpoint.scala:64)
            at akka.remote.DefaultMessageDispatcher.payload$1(Endpoint.scala:64)
            at akka.remote.DefaultMessageDispatcher.dispatch(Endpoint.scala:82)
            at akka.remote.EndpointReader$$anonfun$akka$remote$EndpointReader$$deliverAndAck$1.apply(Endpoint.scala:1047)
            at akka.remote.EndpointReader$$anonfun$akka$remote$EndpointReader$$deliverAndAck$1.apply(Endpoint.scala:1046)
            at scala.collection.Iterator$class.foreach(Iterator.scala:891)
            at scala.collection.AbstractIterator.foreach(Iterator.scala:1334)
            at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
            at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
            at akka.remote.EndpointReader.akka$remote$EndpointReader$$deliverAndAck(Endpoint.scala:1046)
            at akka.remote.EndpointReader$$anonfun$receive$2.applyOrElse(Endpoint.scala:980)
            at akka.actor.Actor$class.aroundReceive(Actor.scala:502)
            at akka.remote.EndpointActor.aroundReceive(Endpoint.scala:446)
            at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526)
            at akka.actor.ActorCell.invoke(ActorCell.scala:495)
            at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257)
            at akka.dispatch.Mailbox.run(Mailbox.scala:224)
            at akka.dispatch.Mailbox.exec(Mailbox.scala:234)
            at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
            at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
            at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
            at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
    ```


---

[GitHub] flink issue #5528: [FLINK-6489] [FLINK-8696] [shell scripts] Remove JobManag...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the issue:

    https://github.com/apache/flink/pull/5528
  
    Okay, after a quick check with @zentol 
      - Some issues were caused by Java 9. Apparently Flink on Windows is incompatible with Java 9
      - We need to change the JobManager and TaskManager to use the flip-6 entry points.
      - The `common.sh` in `flink-end-to-end-tests` needs to be updated.
    



---

[GitHub] flink issue #5528: [FLINK-6489] [FLINK-8696] [shell scripts] Remove JobManag...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on the issue:

    https://github.com/apache/flink/pull/5528
  
    The JM/TM can connect to each other though, and the job submission through the UI is also working. Metrics appear to be broken though.


---

[GitHub] flink issue #5528: [FLINK-6489] [FLINK-8696] [shell scripts] Remove JobManag...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on the issue:

    https://github.com/apache/flink/pull/5528
  
    I tried it again with java 8. The job submission through the client now also works for me, and the warning is no longer printed.


---

[GitHub] flink issue #5528: [FLINK-6489] [FLINK-8696] [shell scripts] Remove JobManag...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the issue:

    https://github.com/apache/flink/pull/5528
  
    Tested it locally again, works (with current flip-6 Web UI at 9065)
    
    Adjusted the test-infra scripts to remove local mode. Tose are only executed on Travis, so waiting until the CI build passes...


---

[GitHub] flink pull request #5528: [FLINK-6489] [FLINK-8696] [shell scripts] Remove J...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen closed the pull request at:

    https://github.com/apache/flink/pull/5528


---

[GitHub] flink pull request #5528: [FLINK-6489] [FLINK-8696] [shell scripts] Remove J...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5528#discussion_r169612616
  
    --- Diff: flink-dist/src/main/flink-bin/bin/start-cluster.bat ---
    @@ -0,0 +1,77 @@
    +::###############################################################################
    +::  Licensed to the Apache Software Foundation (ASF) under one
    +::  or more contributor license agreements.  See the NOTICE file
    +::  distributed with this work for additional information
    +::  regarding copyright ownership.  The ASF licenses this file
    +::  to you under the Apache License, Version 2.0 (the
    +::  "License"); you may not use this file except in compliance
    +::  with the License.  You may obtain a copy of the License at
    +::
    +::      http://www.apache.org/licenses/LICENSE-2.0
    +::
    +::  Unless required by applicable law or agreed to in writing, software
    +::  distributed under the License is distributed on an "AS IS" BASIS,
    +::  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +::  See the License for the specific language governing permissions and
    +:: limitations under the License.
    +::###############################################################################
    +
    +@echo off
    +setlocal EnableDelayedExpansion
    +
    +SET bin=%~dp0
    +SET FLINK_ROOT_DIR=%bin%..
    +SET FLINK_LIB_DIR=%FLINK_ROOT_DIR%\lib
    +SET FLINK_CONF_DIR=%FLINK_ROOT_DIR%\conf
    +SET FLINK_LOG_DIR=%FLINK_ROOT_DIR%\log
    +
    +SET JVM_ARGS=-Xms1024m -Xmx1024m
    +
    +SET FLINK_CLASSPATH=%FLINK_LIB_DIR%\*
    +
    +SET logname_jm=flink-%username%-jobmanager-%computername%.log
    +SET logname_tm=flink-%username%-taskmanager-%computername%.log
    +SET log_jm=%FLINK_LOG_DIR%\%logname_jm%
    +SET log_tm=%FLINK_LOG_DIR%\%logname_tm%
    +SET outname_jm=flink-%username%-jobmanager-%computername%.out
    +SET outname_tm=flink-%username%-taskmanager-%computername%.out
    +SET out_jm=%FLINK_LOG_DIR%\%outname_jm%
    +SET out_tm=%FLINK_LOG_DIR%\%outname_tm%
    +
    +SET log_setting_jm=-Dlog.file="%log_jm%" -Dlogback.configurationFile=file:"%FLINK_CONF_DIR%/logback.xml" -Dlog4j.configuration=file:"%FLINK_CONF_DIR%/log4j.properties"
    +SET log_setting_tm=-Dlog.file="%log_tm%" -Dlogback.configurationFile=file:"%FLINK_CONF_DIR%/logback.xml" -Dlog4j.configuration=file:"%FLINK_CONF_DIR%/log4j.properties"
    +
    +:: Log rotation (quick and dirty)
    +CD "%FLINK_LOG_DIR%"
    +for /l %%x in (5, -1, 1) do ( 
    +SET /A y = %%x+1 
    +RENAME "%logname_jm%.%%x" "%logname_jm%.!y!" 2> nul
    +RENAME "%logname_tm%.%%x" "%logname_tm%.!y!" 2> nul
    +RENAME "%outname_jm%.%%x" "%outname_jm%.!y!"  2> nul
    +RENAME "%outname_tm%.%%x" "%outname_tm%.!y!"  2> nul
    +)
    +RENAME "%logname_jm%" "%logname_jm%.0"  2> nul
    +RENAME "%logname_tm%" "%logname_tm%.0"  2> nul
    +RENAME "%outname_jm%" "%outname_jm%.0"  2> nul
    +RENAME "%outname_tm%" "%outname_tm%.0"  2> nul
    +DEL "%logname_jm%.6"  2> nul
    +DEL "%logname_tm%.6"  2> nul
    +DEL "%outname_jm%.6"  2> nul
    +DEL "%outname_tm%.6"  2> nul
    +
    +for %%X in (java.exe) do (set FOUND=%%~$PATH:X)
    +if not defined FOUND (
    +    echo java.exe was not found in PATH variable
    +    goto :eof
    +)
    +
    +echo Starting a local cluster with one JobManager process and one TaskManager process.
    +
    +echo You can terminate the processes via CTRL-C in the spawned shell windows.
    +
    +echo Web interface by default on http://localhost:8081/.
    --- End diff --
    
    I would like to keep it at 8081, because the flip-6 endpoint will switch to that in the next days anyways and they will probably not be aware that this would need to be updated again ;-)


---

[GitHub] flink issue #5528: [FLINK-6489] [FLINK-8696] [shell scripts] Remove JobManag...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the issue:

    https://github.com/apache/flink/pull/5528
  
    Hmm, weird, I tried it out under Windows (7) two days ago and it worked quite well.
    
    Let me check into what you found there...


---