-
Notifications
You must be signed in to change notification settings - Fork 13k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[FLINK-9028][flip6] perform parameters checking before starting cluster #5726
Changes from 2 commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,58 @@ | ||
/* | ||
* 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.resourcemanager; | ||
|
||
import org.apache.flink.configuration.Configuration; | ||
import org.apache.flink.configuration.ResourceManagerOptions; | ||
import org.junit.Assert; | ||
import org.junit.Test; | ||
|
||
/** | ||
* Tests to guard {@link ResourceManagerRuntimeServices}. | ||
*/ | ||
public class ResourceManagerRuntimeServicesTest { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. missing |
||
|
||
/** | ||
* Test to guard {@link ResourceManagerRuntimeServices#calculateCutoffMB(Configuration, long)}. | ||
*/ | ||
@Test | ||
public void calculateCutoffMB() throws Exception { | ||
|
||
Configuration config = new Configuration(); | ||
long containerMemoryMB = 1000; | ||
|
||
config.setFloat(ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_RATIO, 0.1f); | ||
config.setInteger(ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_MIN, 128); | ||
|
||
Assert.assertEquals(128, ResourceManagerRuntimeServices.calculateCutoffMB(config, containerMemoryMB)); | ||
|
||
config.setFloat(ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_RATIO, 0.2f); | ||
Assert.assertEquals(200, ResourceManagerRuntimeServices.calculateCutoffMB(config, containerMemoryMB)); | ||
|
||
config.setInteger(ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_MIN, 1000); | ||
|
||
try { | ||
ResourceManagerRuntimeServices.calculateCutoffMB(config, containerMemoryMB); | ||
} catch (Exception expected) { | ||
// we expected it. | ||
return; | ||
} | ||
Assert.fail(); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -39,6 +39,8 @@ | |
import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; | ||
import org.apache.flink.runtime.jobgraph.JobGraph; | ||
import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; | ||
import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServices; | ||
import org.apache.flink.runtime.taskexecutor.TaskManagerServices; | ||
import org.apache.flink.util.FlinkException; | ||
import org.apache.flink.util.Preconditions; | ||
import org.apache.flink.util.ShutdownHookUtil; | ||
|
@@ -409,6 +411,12 @@ public void terminateCluster(ApplicationId applicationId) throws FlinkException | |
} | ||
} | ||
|
||
private void checkConfig(ClusterSpecification clusterSpecification) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This should throw a checked exception. Something like |
||
long taskManagerMemorySize = clusterSpecification.getTaskManagerMemoryMB(); | ||
long cutoff = ResourceManagerRuntimeServices.calculateCutoffMB(flinkConfiguration, taskManagerMemorySize); | ||
TaskManagerServices.calculateHeapSizeMB(taskManagerMemorySize - cutoff, flinkConfiguration); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I somehow don't like that we calculate some value just in order to check whether an unchecked exception is thrown. This should be imo more explicit. Or at least it should get a comment explaining what's going on here. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hmm...This is a bit tricky, the |
||
} | ||
|
||
/** | ||
* This method will block until the ApplicationMaster/JobManager have been deployed on YARN. | ||
* | ||
|
@@ -423,6 +431,9 @@ protected ClusterClient<ApplicationId> deployInternal( | |
@Nullable JobGraph jobGraph, | ||
boolean detached) throws Exception { | ||
|
||
// ------------------ Check if configuration is valid -------------------- | ||
checkConfig(clusterSpecification); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. let's not abbreviate and name it |
||
|
||
if (UserGroupInformation.isSecurityEnabled()) { | ||
// note: UGI::hasKerberosCredentials inaccurately reports false | ||
// for logins based on a keytab (fixed in Hadoop 2.6.1, see HADOOP-10786), | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I would keep this method in the
ContaineredTaskManagerParameters
class.