You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by rmetzger <gi...@git.apache.org> on 2015/11/23 17:04:55 UTC

[GitHub] flink pull request: [FLINK-2996] Introduce configuration parameter...

GitHub user rmetzger opened a pull request:

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

    [FLINK-2996] Introduce configuration parameter for BlobServer port

    There were at least two users on the Flink mailing list running Flink behind a firewall which is open only for certain ports.
    Therefore, I've introduced a new configuration parameter for the BlobServer to allow specifying a port, a list of ports or a range of ports.
    
    As soon as this PR gets accepted, I'll implement the same of the ApplicationMaster/JobManager.

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

    $ git pull https://github.com/rmetzger/flink flink2996

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

    https://github.com/apache/flink/pull/1394.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 #1394
    
----
commit 8f5cebf7f05c3bc1f32df9b876f9a9c58b1eecc5
Author: Robert Metzger <rm...@apache.org>
Date:   2015-11-23T15:55:50Z

    [FLINK-2996] Introduce configuration parameter for BlobServer port

commit a4e8c64484cf9b3359d0a7d5906791043cbd46c4
Author: Robert Metzger <rm...@apache.org>
Date:   2015-11-23T16:02:56Z

    wip

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2996] Introduce configuration parameter...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2996] Introduce configuration parameter...

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

    https://github.com/apache/flink/pull/1394#discussion_r45733642
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRangeTest.java ---
    @@ -0,0 +1,108 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.flink.runtime.blob;
    +
    +import org.apache.flink.configuration.ConfigConstants;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.util.NetUtils;
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import java.io.IOException;
    +import java.net.ServerSocket;
    +
    +/**
    + * Tests to ensure that the BlobServer properly starts on a specified range of available ports.
    + */
    +public class BlobServerRangeTest {
    +
    +
    --- End diff --
    
    two line linebreak


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2996] Introduce configuration parameter...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on the pull request:

    https://github.com/apache/flink/pull/1394#issuecomment-159218757
  
    Looks good :+1: 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2996] Introduce configuration parameter...

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

    https://github.com/apache/flink/pull/1394#discussion_r45733770
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRangeTest.java ---
    @@ -0,0 +1,108 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.flink.runtime.blob;
    +
    +import org.apache.flink.configuration.ConfigConstants;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.util.NetUtils;
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import java.io.IOException;
    +import java.net.ServerSocket;
    +
    +/**
    + * Tests to ensure that the BlobServer properly starts on a specified range of available ports.
    + */
    +public class BlobServerRangeTest {
    --- End diff --
    
    Should extend `TestLogger` otherwise you don't see in which test you are.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2996] Introduce configuration parameter...

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

    https://github.com/apache/flink/pull/1394#discussion_r45733147
  
    --- Diff: flink-core/src/test/java/org/apache/flink/util/NetUtilsTest.java ---
    @@ -94,4 +99,53 @@ public void testIPv6URLEncoding() {
     			fail(e.getMessage());
     		}
     	}
    +
    +	@Test
    +	public void testFreePortRangeUtility() {
    +		// inspired by Hadoop's example for "yarn.app.mapreduce.am.job.client.port-range"
    +		String rangeDefinition = "50000-50050, 50100-50200,51234 "; // this also contains some whitespaces
    +		Set<Integer> ports = NetUtils.getPortRangeFromString(rangeDefinition);
    +		Assert.assertEquals(51+101+1, ports.size());
    +		// check first range
    +		Assert.assertThat(ports, hasItems(50000, 50001, 50002, 50050));
    --- End diff --
    
    What if `ports` does not contain `50003`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2996] Introduce configuration parameter...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on the pull request:

    https://github.com/apache/flink/pull/1394#issuecomment-159291184
  
    +1 for merging


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2996] Introduce configuration parameter...

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

    https://github.com/apache/flink/pull/1394#discussion_r45733750
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRangeTest.java ---
    @@ -0,0 +1,108 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.flink.runtime.blob;
    +
    +import org.apache.flink.configuration.ConfigConstants;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.util.NetUtils;
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import java.io.IOException;
    +import java.net.ServerSocket;
    +
    +/**
    + * Tests to ensure that the BlobServer properly starts on a specified range of available ports.
    + */
    +public class BlobServerRangeTest {
    +
    +
    +	/**
    +	 * Start blob server on 0 = pick an ephemeral port
    +	 */
    +	@Test
    +	public void testOnEphemeralPort() {
    +		Configuration conf = new Configuration();
    +		conf.setString(ConfigConstants.BLOB_SERVER_PORT, "0");
    +		try {
    +			BlobServer srv = new BlobServer(conf);
    +		} catch (IOException e) {
    --- End diff --
    
    Why don't you let the exception propagate. This will be enough to fail the test and you will see the stack trace at the proper position.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2996] Introduce configuration parameter...

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

    https://github.com/apache/flink/pull/1394#discussion_r45733621
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java ---
    @@ -137,12 +139,29 @@ else if (config.containsKey(ConfigConstants.STATE_BACKEND) &&
     			this.shutdownHook = null;
     		}
     
    -		// start the server
    -		try {
    -			this.serverSocket = new ServerSocket(0, backlog);
    +		//  ----------------------- start the server -------------------
    +
    +		String serverPortRange = config.getString(ConfigConstants.BLOB_SERVER_PORT, ConfigConstants.DEFAULT_BLOB_SERVER_PORT);
    +		Iterator<Integer> ports = NetUtils.getPortRangeFromString(serverPortRange).iterator();
    +
    +		ServerSocket socketAttempt = null;
    +		while(ports.hasNext()) {
    --- End diff --
    
    Why do you have to test all ports here. Isn't it enough to take the first port for which you can open a socket? Or is it important that you always take the last available port?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2996] Introduce configuration parameter...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1394#issuecomment-159281472
  
    I've addressed Till's comments, thank you for the review.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2996] Introduce configuration parameter...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on the pull request:

    https://github.com/apache/flink/pull/1394#issuecomment-159266975
  
    I had some comments concerning the port selection. I think it's not necessary to traverse the complete list of available ports. The first open port should be enough.
    
    Besides that, LGTM.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2996] Introduce configuration parameter...

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

    https://github.com/apache/flink/pull/1394#discussion_r45738417
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java ---
    @@ -137,12 +139,29 @@ else if (config.containsKey(ConfigConstants.STATE_BACKEND) &&
     			this.shutdownHook = null;
     		}
     
    -		// start the server
    -		try {
    -			this.serverSocket = new ServerSocket(0, backlog);
    +		//  ----------------------- start the server -------------------
    +
    +		String serverPortRange = config.getString(ConfigConstants.BLOB_SERVER_PORT, ConfigConstants.DEFAULT_BLOB_SERVER_PORT);
    +		Iterator<Integer> ports = NetUtils.getPortRangeFromString(serverPortRange).iterator();
    +
    +		ServerSocket socketAttempt = null;
    +		while(ports.hasNext()) {
    --- End diff --
    
    Thnx, I'll fix that. Stupid mistake


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---