diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/application/ApplicationExceptionsMessageParameters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/application/ApplicationExceptionsMessageParameters.java index cee07b8b20966..62db2c9351c30 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/application/ApplicationExceptionsMessageParameters.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/application/ApplicationExceptionsMessageParameters.java @@ -30,7 +30,7 @@ /** {@link MessageParameters} for {@link ApplicationExceptionsHandler}. */ public class ApplicationExceptionsMessageParameters extends ApplicationMessageParameters { - private final UpperLimitExceptionParameter upperLimitExceptionParameter = + public final UpperLimitExceptionParameter upperLimitExceptionParameter = new UpperLimitExceptionParameter(); @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobExceptionsMessageParameters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobExceptionsMessageParameters.java index 46e1d02716db4..679badb27860c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobExceptionsMessageParameters.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobExceptionsMessageParameters.java @@ -30,10 +30,10 @@ /** {@link MessageParameters} for {@link JobExceptionsHandler}. */ public class JobExceptionsMessageParameters extends JobMessageParameters { - private final UpperLimitExceptionParameter upperLimitExceptionParameter = + public final UpperLimitExceptionParameter upperLimitExceptionParameter = new UpperLimitExceptionParameter(); - private final FailureLabelFilterParameter failureLabelExceptionParameter = + public final FailureLabelFilterParameter failureLabelExceptionParameter = new FailureLabelFilterParameter(); @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/application/ApplicationExceptionsMessageParametersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/application/ApplicationExceptionsMessageParametersTest.java new file mode 100644 index 0000000000000..4a2e13d9fb910 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/application/ApplicationExceptionsMessageParametersTest.java @@ -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.rest.messages.application; + +import org.apache.flink.api.common.ApplicationID; +import org.apache.flink.runtime.rest.messages.MessageParameters; + +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link ApplicationExceptionsMessageParameters}. */ +class ApplicationExceptionsMessageParametersTest { + + @Test + void testMaxExceptionsQueryParameterIsRendered() throws Exception { + ApplicationID applicationId = ApplicationID.generate(); + ApplicationExceptionsMessageParameters parameters = + new ApplicationExceptionsMessageParameters(); + parameters.applicationPathParameter.resolve(applicationId); + parameters.upperLimitExceptionParameter.resolveFromString("20"); + + String resolvedUrl = + MessageParameters.resolveUrl(ApplicationExceptionsHeaders.URL, parameters); + + assertThat(resolvedUrl) + .isEqualTo("/applications/" + applicationId + "/exceptions?maxExceptions=20"); + } + + @Test + void testMaxExceptionsQueryParameterIsOmittedWhenUnresolved() { + ApplicationID applicationId = ApplicationID.generate(); + ApplicationExceptionsMessageParameters parameters = + new ApplicationExceptionsMessageParameters(); + parameters.applicationPathParameter.resolve(applicationId); + + String resolvedUrl = + MessageParameters.resolveUrl(ApplicationExceptionsHeaders.URL, parameters); + + assertThat(resolvedUrl).isEqualTo("/applications/" + applicationId + "/exceptions"); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/JobExceptionsMessageParametersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/JobExceptionsMessageParametersTest.java new file mode 100644 index 0000000000000..b30006282731f --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/JobExceptionsMessageParametersTest.java @@ -0,0 +1,66 @@ +/* + * 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.rest.messages.job; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.rest.messages.JobExceptionsHeaders; +import org.apache.flink.runtime.rest.messages.MessageParameters; + +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link JobExceptionsMessageParameters}. */ +class JobExceptionsMessageParametersTest { + + @Test + void testMaxExceptionsQueryParameterIsRendered() throws Exception { + JobID jobId = new JobID(); + JobExceptionsMessageParameters parameters = new JobExceptionsMessageParameters(); + parameters.jobPathParameter.resolve(jobId); + parameters.upperLimitExceptionParameter.resolveFromString("20"); + + String resolvedUrl = MessageParameters.resolveUrl(JobExceptionsHeaders.URL, parameters); + + assertThat(resolvedUrl).isEqualTo("/jobs/" + jobId + "/exceptions?maxExceptions=20"); + } + + @Test + void testFailureLabelFilterQueryParameterIsRendered() throws Exception { + JobID jobId = new JobID(); + JobExceptionsMessageParameters parameters = new JobExceptionsMessageParameters(); + parameters.jobPathParameter.resolve(jobId); + parameters.failureLabelExceptionParameter.resolveFromString("type:system"); + + String resolvedUrl = MessageParameters.resolveUrl(JobExceptionsHeaders.URL, parameters); + + assertThat(resolvedUrl).contains("failureLabelFilter="); + } + + @Test + void testOptionalQueryParametersAreOmittedWhenUnresolved() { + JobID jobId = new JobID(); + JobExceptionsMessageParameters parameters = new JobExceptionsMessageParameters(); + parameters.jobPathParameter.resolve(jobId); + + String resolvedUrl = MessageParameters.resolveUrl(JobExceptionsHeaders.URL, parameters); + + assertThat(resolvedUrl).isEqualTo("/jobs/" + jobId + "/exceptions"); + } +}