You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/04/06 12:00:05 UTC

[GitHub] [flink] alpreu opened a new pull request, #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

alpreu opened a new pull request, #19377:
URL: https://github.com/apache/flink/pull/19377

   ## What is the purpose of the change
   
   Connectors should not depend on non-public API. To ensure that connectors satisfy this criteria we introduce a new ArchUnit test/rule for them.
   
   
   ## Brief change log
   
     - Added new ArchUnit rule checking connector dependencies
     - Updated ArchUnit violations
   
   
   ## Verifying this change
   
   This change added tests and can be verified as follows:
   
     - Run the test and check that violations are detected (see archunit-violations/d5f08b81-d148-4c1d-8666-d111e4b28600)
     - Manually introduce a violation by adding non-public API code to a connector and check if test detects it
   
   ## 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, Kubernetes/Yarn, 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)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] JingGe commented on a diff in pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
JingGe commented on code in PR #19377:
URL: https://github.com/apache/flink/pull/19377#discussion_r855236604


##########
flink-architecture-tests/flink-architecture-tests-production/src/main/java/org/apache/flink/architecture/rules/ConnectorRules.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.architecture.rules;
+
+import org.apache.flink.annotation.Public;
+import org.apache.flink.annotation.PublicEvolving;
+
+import com.tngtech.archunit.base.DescribedPredicate;
+import com.tngtech.archunit.core.domain.JavaClass;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.lang.ArchRule;
+import com.tngtech.archunit.thirdparty.com.google.common.base.Joiner;
+
+import static com.tngtech.archunit.base.DescribedPredicate.not;
+import static com.tngtech.archunit.core.domain.JavaModifier.PUBLIC;
+import static com.tngtech.archunit.core.domain.properties.HasModifiers.Predicates.modifier;
+import static com.tngtech.archunit.library.freeze.FreezingArchRule.freeze;
+import static org.apache.flink.architecture.common.GivenJavaClasses.noJavaClassesThat;
+import static org.apache.flink.architecture.common.Predicates.areDirectlyAnnotatedWithAtLeastOneOf;
+import static org.apache.flink.architecture.common.SourcePredicates.areProductionCode;
+
+/** Rules for Flink connectors. */
+public class ConnectorRules {
+    private static final String[] CONNECTOR_PACKAGES = {
+        "org.apache.flink.connector..", "org.apache.flink.streaming.connectors.."
+    };
+
+    private static DescribedPredicate<JavaClass> areNotPublicAndResideOutsideOfPackages(
+            String... packageIdentifiers) {
+        return JavaClass.Predicates.resideOutsideOfPackages(packageIdentifiers)
+                .and(
+                        not(areDirectlyAnnotatedWithAtLeastOneOf(
+                                        Public.class, PublicEvolving.class))
+                                .and(not(modifier(PUBLIC))))

Review Comment:
   I think the rule is fine without this check. Do you have a concrete violation if we don't have it?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] JingGe commented on a diff in pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
JingGe commented on code in PR #19377:
URL: https://github.com/apache/flink/pull/19377#discussion_r855893792


##########
flink-architecture-tests/flink-architecture-tests-production/src/main/java/org/apache/flink/architecture/rules/ConnectorRules.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.architecture.rules;
+
+import org.apache.flink.annotation.Public;
+import org.apache.flink.annotation.PublicEvolving;
+
+import com.tngtech.archunit.base.DescribedPredicate;
+import com.tngtech.archunit.core.domain.JavaClass;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.lang.ArchRule;
+import com.tngtech.archunit.thirdparty.com.google.common.base.Joiner;
+
+import static com.tngtech.archunit.base.DescribedPredicate.not;
+import static com.tngtech.archunit.core.domain.JavaModifier.PUBLIC;
+import static com.tngtech.archunit.core.domain.properties.HasModifiers.Predicates.modifier;
+import static com.tngtech.archunit.library.freeze.FreezingArchRule.freeze;
+import static org.apache.flink.architecture.common.GivenJavaClasses.noJavaClassesThat;
+import static org.apache.flink.architecture.common.Predicates.areDirectlyAnnotatedWithAtLeastOneOf;
+import static org.apache.flink.architecture.common.SourcePredicates.areProductionCode;
+
+/** Rules for Flink connectors. */
+public class ConnectorRules {
+    private static final String[] CONNECTOR_PACKAGES = {
+        "org.apache.flink.connector..", "org.apache.flink.streaming.connectors.."
+    };
+
+    private static DescribedPredicate<JavaClass> areNotPublicAndResideOutsideOfPackages(
+            String... packageIdentifiers) {
+        return JavaClass.Predicates.resideOutsideOfPackages(packageIdentifiers)
+                .and(
+                        not(areDirectlyAnnotatedWithAtLeastOneOf(
+                                        Public.class, PublicEvolving.class))
+                                .and(not(modifier(PUBLIC))))

Review Comment:
   yeah, it should be fine. The question is that is there a case that a protected class outside of the connector packages, which has the @Public or @PublicEvolving, will be used by connectors?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] alpreu commented on pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
alpreu commented on PR #19377:
URL: https://github.com/apache/flink/pull/19377#issuecomment-1091684319

   @flinkbot run azure


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] JingGe commented on a diff in pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
JingGe commented on code in PR #19377:
URL: https://github.com/apache/flink/pull/19377#discussion_r845293143


##########
flink-architecture-tests/flink-architecture-tests-production/src/main/java/org/apache/flink/architecture/rules/ConnectorRules.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.architecture.rules;
+
+import com.tngtech.archunit.base.DescribedPredicate;
+import com.tngtech.archunit.core.domain.JavaClass;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.lang.ArchRule;
+import com.tngtech.archunit.thirdparty.com.google.common.base.Joiner;
+
+import static com.tngtech.archunit.base.DescribedPredicate.not;
+import static com.tngtech.archunit.core.domain.JavaModifier.PUBLIC;
+import static com.tngtech.archunit.core.domain.properties.HasModifiers.Predicates.modifier;
+import static com.tngtech.archunit.library.freeze.FreezingArchRule.freeze;
+import static org.apache.flink.architecture.common.GivenJavaClasses.noJavaClassesThat;
+import static org.apache.flink.architecture.common.SourcePredicates.areProductionCode;
+
+/** Rules for Flink connectors. */
+public class ConnectorRules {
+    private static final String[] CONNECTOR_PACKAGES = {
+        "org.apache.flink.connector..", "org.apache.flink.streaming.connectors.."
+    };
+
+    private static DescribedPredicate<JavaClass> areNotPublicAndResideOutsideOfPackages(
+            String... packageIdentifiers) {
+        return JavaClass.Predicates.resideOutsideOfPackages(packageIdentifiers)
+                .and(not(modifier(PUBLIC)))
+                .as(
+                        "are not public and reside outside of packages ['%s']",
+                        Joiner.on("', '").join(packageIdentifiers));
+    }
+
+    @ArchTest
+    public static final ArchRule CONNECTOR_CLASSES_ONLY_DEPEND_ON_PUBLIC_API =
+            freeze(
+                    noJavaClassesThat(areProductionCode())
+                            .and()
+                            .resideInAnyPackage(CONNECTOR_PACKAGES)
+                            .and()
+                            .arePublic()

Review Comment:
   I think all classes should follow this rule.



##########
flink-architecture-tests/flink-architecture-tests-production/src/main/java/org/apache/flink/architecture/rules/ConnectorRules.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.architecture.rules;
+
+import com.tngtech.archunit.base.DescribedPredicate;
+import com.tngtech.archunit.core.domain.JavaClass;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.lang.ArchRule;
+import com.tngtech.archunit.thirdparty.com.google.common.base.Joiner;
+
+import static com.tngtech.archunit.base.DescribedPredicate.not;
+import static com.tngtech.archunit.core.domain.JavaModifier.PUBLIC;
+import static com.tngtech.archunit.core.domain.properties.HasModifiers.Predicates.modifier;
+import static com.tngtech.archunit.library.freeze.FreezingArchRule.freeze;
+import static org.apache.flink.architecture.common.GivenJavaClasses.noJavaClassesThat;
+import static org.apache.flink.architecture.common.SourcePredicates.areProductionCode;
+
+/** Rules for Flink connectors. */
+public class ConnectorRules {
+    private static final String[] CONNECTOR_PACKAGES = {
+        "org.apache.flink.connector..", "org.apache.flink.streaming.connectors.."
+    };
+
+    private static DescribedPredicate<JavaClass> areNotPublicAndResideOutsideOfPackages(
+            String... packageIdentifiers) {
+        return JavaClass.Predicates.resideOutsideOfPackages(packageIdentifiers)
+                .and(not(modifier(PUBLIC)))

Review Comment:
   Is publicEvolving also allowed?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] MartijnVisser commented on pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
MartijnVisser commented on PR #19377:
URL: https://github.com/apache/flink/pull/19377#issuecomment-1115803846

   @flinkbot run azure


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] flinkbot commented on pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
flinkbot commented on PR #19377:
URL: https://github.com/apache/flink/pull/19377#issuecomment-1090192823

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3350ff8faf5811e5168a105af4e96ecd6e490fb1",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3350ff8faf5811e5168a105af4e96ecd6e490fb1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3350ff8faf5811e5168a105af4e96ecd6e490fb1 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] alpreu commented on pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
alpreu commented on PR #19377:
URL: https://github.com/apache/flink/pull/19377#issuecomment-1108756932

   As discussed offline, we only consider classes to be part of the public API if they have the annotation of {Public, PublicEvolving, Experimental} independent of their class modifier


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] JingGe commented on a diff in pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
JingGe commented on code in PR #19377:
URL: https://github.com/apache/flink/pull/19377#discussion_r855893792


##########
flink-architecture-tests/flink-architecture-tests-production/src/main/java/org/apache/flink/architecture/rules/ConnectorRules.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.architecture.rules;
+
+import org.apache.flink.annotation.Public;
+import org.apache.flink.annotation.PublicEvolving;
+
+import com.tngtech.archunit.base.DescribedPredicate;
+import com.tngtech.archunit.core.domain.JavaClass;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.lang.ArchRule;
+import com.tngtech.archunit.thirdparty.com.google.common.base.Joiner;
+
+import static com.tngtech.archunit.base.DescribedPredicate.not;
+import static com.tngtech.archunit.core.domain.JavaModifier.PUBLIC;
+import static com.tngtech.archunit.core.domain.properties.HasModifiers.Predicates.modifier;
+import static com.tngtech.archunit.library.freeze.FreezingArchRule.freeze;
+import static org.apache.flink.architecture.common.GivenJavaClasses.noJavaClassesThat;
+import static org.apache.flink.architecture.common.Predicates.areDirectlyAnnotatedWithAtLeastOneOf;
+import static org.apache.flink.architecture.common.SourcePredicates.areProductionCode;
+
+/** Rules for Flink connectors. */
+public class ConnectorRules {
+    private static final String[] CONNECTOR_PACKAGES = {
+        "org.apache.flink.connector..", "org.apache.flink.streaming.connectors.."
+    };
+
+    private static DescribedPredicate<JavaClass> areNotPublicAndResideOutsideOfPackages(
+            String... packageIdentifiers) {
+        return JavaClass.Predicates.resideOutsideOfPackages(packageIdentifiers)
+                .and(
+                        not(areDirectlyAnnotatedWithAtLeastOneOf(
+                                        Public.class, PublicEvolving.class))
+                                .and(not(modifier(PUBLIC))))

Review Comment:
   yeah, it should be fine. The question is that is there a case that a package private class outside of the connector packages, which has the @Public or @PublicEvolving, will be used by connectors?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] alpreu commented on a diff in pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
alpreu commented on code in PR #19377:
URL: https://github.com/apache/flink/pull/19377#discussion_r855243602


##########
flink-architecture-tests/flink-architecture-tests-production/src/main/java/org/apache/flink/architecture/rules/ConnectorRules.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.architecture.rules;
+
+import org.apache.flink.annotation.Public;
+import org.apache.flink.annotation.PublicEvolving;
+
+import com.tngtech.archunit.base.DescribedPredicate;
+import com.tngtech.archunit.core.domain.JavaClass;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.lang.ArchRule;
+import com.tngtech.archunit.thirdparty.com.google.common.base.Joiner;
+
+import static com.tngtech.archunit.base.DescribedPredicate.not;
+import static com.tngtech.archunit.core.domain.JavaModifier.PUBLIC;
+import static com.tngtech.archunit.core.domain.properties.HasModifiers.Predicates.modifier;
+import static com.tngtech.archunit.library.freeze.FreezingArchRule.freeze;
+import static org.apache.flink.architecture.common.GivenJavaClasses.noJavaClassesThat;
+import static org.apache.flink.architecture.common.Predicates.areDirectlyAnnotatedWithAtLeastOneOf;
+import static org.apache.flink.architecture.common.SourcePredicates.areProductionCode;
+
+/** Rules for Flink connectors. */
+public class ConnectorRules {
+    private static final String[] CONNECTOR_PACKAGES = {
+        "org.apache.flink.connector..", "org.apache.flink.streaming.connectors.."
+    };
+
+    private static DescribedPredicate<JavaClass> areNotPublicAndResideOutsideOfPackages(
+            String... packageIdentifiers) {
+        return JavaClass.Predicates.resideOutsideOfPackages(packageIdentifiers)
+                .and(
+                        not(areDirectlyAnnotatedWithAtLeastOneOf(
+                                        Public.class, PublicEvolving.class))
+                                .and(not(modifier(PUBLIC))))

Review Comment:
   What part of the rule are you referring to? I added the part about the annotation due to the discussion from the other comment



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] MartijnVisser commented on a diff in pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
MartijnVisser commented on code in PR #19377:
URL: https://github.com/apache/flink/pull/19377#discussion_r847088827


##########
flink-architecture-tests/flink-architecture-tests-production/src/main/java/org/apache/flink/architecture/rules/ConnectorRules.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.architecture.rules;
+
+import com.tngtech.archunit.base.DescribedPredicate;
+import com.tngtech.archunit.core.domain.JavaClass;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.lang.ArchRule;
+import com.tngtech.archunit.thirdparty.com.google.common.base.Joiner;
+
+import static com.tngtech.archunit.base.DescribedPredicate.not;
+import static com.tngtech.archunit.core.domain.JavaModifier.PUBLIC;
+import static com.tngtech.archunit.core.domain.properties.HasModifiers.Predicates.modifier;
+import static com.tngtech.archunit.library.freeze.FreezingArchRule.freeze;
+import static org.apache.flink.architecture.common.GivenJavaClasses.noJavaClassesThat;
+import static org.apache.flink.architecture.common.SourcePredicates.areProductionCode;
+
+/** Rules for Flink connectors. */
+public class ConnectorRules {
+    private static final String[] CONNECTOR_PACKAGES = {
+        "org.apache.flink.connector..", "org.apache.flink.streaming.connectors.."
+    };
+
+    private static DescribedPredicate<JavaClass> areNotPublicAndResideOutsideOfPackages(
+            String... packageIdentifiers) {
+        return JavaClass.Predicates.resideOutsideOfPackages(packageIdentifiers)
+                .and(not(modifier(PUBLIC)))

Review Comment:
   It should be number 2, based on the table for 'Existing guarantees' at https://cwiki.apache.org/confluence/display/FLINK/FLIP-196%3A+Source+API+stability+guarantees. This would also be in line with https://github.com/apache/flink/tree/master/flink-annotations/src/main/java/org/apache/flink/annotation I think?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] MartijnVisser commented on pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
MartijnVisser commented on PR #19377:
URL: https://github.com/apache/flink/pull/19377#issuecomment-1122794396

   @flinkbot run azure


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] JingGe commented on a diff in pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
JingGe commented on code in PR #19377:
URL: https://github.com/apache/flink/pull/19377#discussion_r855627519


##########
flink-architecture-tests/flink-architecture-tests-production/src/main/java/org/apache/flink/architecture/rules/ConnectorRules.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.architecture.rules;
+
+import org.apache.flink.annotation.Public;
+import org.apache.flink.annotation.PublicEvolving;
+
+import com.tngtech.archunit.base.DescribedPredicate;
+import com.tngtech.archunit.core.domain.JavaClass;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.lang.ArchRule;
+import com.tngtech.archunit.thirdparty.com.google.common.base.Joiner;
+
+import static com.tngtech.archunit.base.DescribedPredicate.not;
+import static com.tngtech.archunit.core.domain.JavaModifier.PUBLIC;
+import static com.tngtech.archunit.core.domain.properties.HasModifiers.Predicates.modifier;
+import static com.tngtech.archunit.library.freeze.FreezingArchRule.freeze;
+import static org.apache.flink.architecture.common.GivenJavaClasses.noJavaClassesThat;
+import static org.apache.flink.architecture.common.Predicates.areDirectlyAnnotatedWithAtLeastOneOf;
+import static org.apache.flink.architecture.common.SourcePredicates.areProductionCode;
+
+/** Rules for Flink connectors. */
+public class ConnectorRules {
+    private static final String[] CONNECTOR_PACKAGES = {
+        "org.apache.flink.connector..", "org.apache.flink.streaming.connectors.."
+    };
+
+    private static DescribedPredicate<JavaClass> areNotPublicAndResideOutsideOfPackages(
+            String... packageIdentifiers) {
+        return JavaClass.Predicates.resideOutsideOfPackages(packageIdentifiers)
+                .and(
+                        not(areDirectlyAnnotatedWithAtLeastOneOf(
+                                        Public.class, PublicEvolving.class))
+                                .and(not(modifier(PUBLIC))))

Review Comment:
   Sorry for the unclear description. I meant without using this one: `and(not(modifier(PUBLIC)))`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] alpreu commented on a diff in pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
alpreu commented on code in PR #19377:
URL: https://github.com/apache/flink/pull/19377#discussion_r843988471


##########
flink-architecture-tests/flink-architecture-tests-production/src/main/java/org/apache/flink/architecture/rules/ConnectorRules.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.architecture.rules;
+
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.annotation.Internal;
+
+import com.tngtech.archunit.base.DescribedPredicate;
+import com.tngtech.archunit.core.domain.JavaClass;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.lang.ArchRule;
+import com.tngtech.archunit.thirdparty.com.google.common.base.Joiner;
+
+import static com.tngtech.archunit.base.DescribedPredicate.not;
+import static com.tngtech.archunit.core.domain.JavaModifier.PUBLIC;
+import static com.tngtech.archunit.core.domain.properties.HasModifiers.Predicates.modifier;
+import static com.tngtech.archunit.library.freeze.FreezingArchRule.freeze;
+import static org.apache.flink.architecture.common.GivenJavaClasses.noJavaClassesThat;
+import static org.apache.flink.architecture.common.SourcePredicates.areProductionCode;
+
+/** Rules for Flink connectors. */
+public class ConnectorRules {
+    private static final String[] CONNECTOR_PACKAGES = {
+        "org.apache.flink.connector..", "org.apache.flink.streaming.connectors.."
+    };
+
+    private static DescribedPredicate<JavaClass> areNotPublicAndResideOutsideOfPackages(
+            String... packageIdentifiers) {
+        return JavaClass.Predicates.resideOutsideOfPackages(packageIdentifiers)
+                .and(not(modifier(PUBLIC)))
+                .as(
+                        "are not public and reside outside of packages ['%s']",
+                        Joiner.on("', '").join(packageIdentifiers));
+    }
+
+    @ArchTest
+    public static final ArchRule CONNECTOR_CLASSES_ONLY_DEPEND_ON_PUBLIC_API =
+            freeze(
+                    noJavaClassesThat(areProductionCode())
+                            .and()
+                            .resideInAnyPackage(CONNECTOR_PACKAGES)
+                            .and()
+                            .arePublic()
+                            .and()
+                            .areNotAnnotatedWith(Internal.class)
+                            .and()
+                            .areNotAnnotatedWith(Deprecated.class)
+                            .and()
+                            .areNotAnnotatedWith(Experimental.class)
+                            .should()
+                            .dependOnClassesThat(

Review Comment:
   I think the fluent API style does not work very well here. It actually is negated by the `noJavaClassesThat` at the start of the rule :/



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] alpreu commented on a diff in pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
alpreu commented on code in PR #19377:
URL: https://github.com/apache/flink/pull/19377#discussion_r847180722


##########
flink-architecture-tests/flink-architecture-tests-production/src/main/java/org/apache/flink/architecture/rules/ConnectorRules.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.architecture.rules;
+
+import com.tngtech.archunit.base.DescribedPredicate;
+import com.tngtech.archunit.core.domain.JavaClass;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.lang.ArchRule;
+import com.tngtech.archunit.thirdparty.com.google.common.base.Joiner;
+
+import static com.tngtech.archunit.base.DescribedPredicate.not;
+import static com.tngtech.archunit.core.domain.JavaModifier.PUBLIC;
+import static com.tngtech.archunit.core.domain.properties.HasModifiers.Predicates.modifier;
+import static com.tngtech.archunit.library.freeze.FreezingArchRule.freeze;
+import static org.apache.flink.architecture.common.GivenJavaClasses.noJavaClassesThat;
+import static org.apache.flink.architecture.common.SourcePredicates.areProductionCode;
+
+/** Rules for Flink connectors. */
+public class ConnectorRules {
+    private static final String[] CONNECTOR_PACKAGES = {
+        "org.apache.flink.connector..", "org.apache.flink.streaming.connectors.."
+    };
+
+    private static DescribedPredicate<JavaClass> areNotPublicAndResideOutsideOfPackages(
+            String... packageIdentifiers) {
+        return JavaClass.Predicates.resideOutsideOfPackages(packageIdentifiers)
+                .and(not(modifier(PUBLIC)))

Review Comment:
   I updated the rule to require the Flink annotations as well



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] MartijnVisser commented on a diff in pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
MartijnVisser commented on code in PR #19377:
URL: https://github.com/apache/flink/pull/19377#discussion_r847088827


##########
flink-architecture-tests/flink-architecture-tests-production/src/main/java/org/apache/flink/architecture/rules/ConnectorRules.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.architecture.rules;
+
+import com.tngtech.archunit.base.DescribedPredicate;
+import com.tngtech.archunit.core.domain.JavaClass;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.lang.ArchRule;
+import com.tngtech.archunit.thirdparty.com.google.common.base.Joiner;
+
+import static com.tngtech.archunit.base.DescribedPredicate.not;
+import static com.tngtech.archunit.core.domain.JavaModifier.PUBLIC;
+import static com.tngtech.archunit.core.domain.properties.HasModifiers.Predicates.modifier;
+import static com.tngtech.archunit.library.freeze.FreezingArchRule.freeze;
+import static org.apache.flink.architecture.common.GivenJavaClasses.noJavaClassesThat;
+import static org.apache.flink.architecture.common.SourcePredicates.areProductionCode;
+
+/** Rules for Flink connectors. */
+public class ConnectorRules {
+    private static final String[] CONNECTOR_PACKAGES = {
+        "org.apache.flink.connector..", "org.apache.flink.streaming.connectors.."
+    };
+
+    private static DescribedPredicate<JavaClass> areNotPublicAndResideOutsideOfPackages(
+            String... packageIdentifiers) {
+        return JavaClass.Predicates.resideOutsideOfPackages(packageIdentifiers)
+                .and(not(modifier(PUBLIC)))

Review Comment:
   It should be number 2



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] JingGe commented on a diff in pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
JingGe commented on code in PR #19377:
URL: https://github.com/apache/flink/pull/19377#discussion_r846309162


##########
flink-architecture-tests/flink-architecture-tests-production/src/main/java/org/apache/flink/architecture/rules/ConnectorRules.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.architecture.rules;
+
+import com.tngtech.archunit.base.DescribedPredicate;
+import com.tngtech.archunit.core.domain.JavaClass;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.lang.ArchRule;
+import com.tngtech.archunit.thirdparty.com.google.common.base.Joiner;
+
+import static com.tngtech.archunit.base.DescribedPredicate.not;
+import static com.tngtech.archunit.core.domain.JavaModifier.PUBLIC;
+import static com.tngtech.archunit.core.domain.properties.HasModifiers.Predicates.modifier;
+import static com.tngtech.archunit.library.freeze.FreezingArchRule.freeze;
+import static org.apache.flink.architecture.common.GivenJavaClasses.noJavaClassesThat;
+import static org.apache.flink.architecture.common.SourcePredicates.areProductionCode;
+
+/** Rules for Flink connectors. */
+public class ConnectorRules {
+    private static final String[] CONNECTOR_PACKAGES = {
+        "org.apache.flink.connector..", "org.apache.flink.streaming.connectors.."
+    };
+
+    private static DescribedPredicate<JavaClass> areNotPublicAndResideOutsideOfPackages(
+            String... packageIdentifiers) {
+        return JavaClass.Predicates.resideOutsideOfPackages(packageIdentifiers)
+                .and(not(modifier(PUBLIC)))

Review Comment:
   Now I got your point. we need to make clear what is the definition of "public API" for this rule:
   
   1. class with the public visibility.
   or
   2. class with the public visibility and has Flink @Public or @PublicEvolving annotation.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] alpreu commented on a diff in pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
alpreu commented on code in PR #19377:
URL: https://github.com/apache/flink/pull/19377#discussion_r846159554


##########
flink-architecture-tests/flink-architecture-tests-production/src/main/java/org/apache/flink/architecture/rules/ConnectorRules.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.architecture.rules;
+
+import com.tngtech.archunit.base.DescribedPredicate;
+import com.tngtech.archunit.core.domain.JavaClass;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.lang.ArchRule;
+import com.tngtech.archunit.thirdparty.com.google.common.base.Joiner;
+
+import static com.tngtech.archunit.base.DescribedPredicate.not;
+import static com.tngtech.archunit.core.domain.JavaModifier.PUBLIC;
+import static com.tngtech.archunit.core.domain.properties.HasModifiers.Predicates.modifier;
+import static com.tngtech.archunit.library.freeze.FreezingArchRule.freeze;
+import static org.apache.flink.architecture.common.GivenJavaClasses.noJavaClassesThat;
+import static org.apache.flink.architecture.common.SourcePredicates.areProductionCode;
+
+/** Rules for Flink connectors. */
+public class ConnectorRules {
+    private static final String[] CONNECTOR_PACKAGES = {
+        "org.apache.flink.connector..", "org.apache.flink.streaming.connectors.."
+    };
+
+    private static DescribedPredicate<JavaClass> areNotPublicAndResideOutsideOfPackages(
+            String... packageIdentifiers) {
+        return JavaClass.Predicates.resideOutsideOfPackages(packageIdentifiers)
+                .and(not(modifier(PUBLIC)))

Review Comment:
   It is included already because we're using the class modifier here, not the Flink annotation



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] alpreu commented on a diff in pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
alpreu commented on code in PR #19377:
URL: https://github.com/apache/flink/pull/19377#discussion_r846082122


##########
flink-architecture-tests/flink-architecture-tests-production/src/main/java/org/apache/flink/architecture/rules/ConnectorRules.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.architecture.rules;
+
+import com.tngtech.archunit.base.DescribedPredicate;
+import com.tngtech.archunit.core.domain.JavaClass;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.lang.ArchRule;
+import com.tngtech.archunit.thirdparty.com.google.common.base.Joiner;
+
+import static com.tngtech.archunit.base.DescribedPredicate.not;
+import static com.tngtech.archunit.core.domain.JavaModifier.PUBLIC;
+import static com.tngtech.archunit.core.domain.properties.HasModifiers.Predicates.modifier;
+import static com.tngtech.archunit.library.freeze.FreezingArchRule.freeze;
+import static org.apache.flink.architecture.common.GivenJavaClasses.noJavaClassesThat;
+import static org.apache.flink.architecture.common.SourcePredicates.areProductionCode;
+
+/** Rules for Flink connectors. */
+public class ConnectorRules {
+    private static final String[] CONNECTOR_PACKAGES = {
+        "org.apache.flink.connector..", "org.apache.flink.streaming.connectors.."
+    };
+
+    private static DescribedPredicate<JavaClass> areNotPublicAndResideOutsideOfPackages(
+            String... packageIdentifiers) {
+        return JavaClass.Predicates.resideOutsideOfPackages(packageIdentifiers)
+                .and(not(modifier(PUBLIC)))

Review Comment:
   The JavaDoc for PublicEvolving states
   ``` 
   Classes and methods with this annotation are intended for public use and have stable behavior.
   However, their interfaces and signatures are not considered to be stable and might be changed
   across versions.
   ```
   
   So I think connector developers should be able to rely on them. For example the new `Sink` interface is still PublicEvolving, so if we prohibit this then no connector could depend on it for at least another release cycle



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] JingGe commented on a diff in pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
JingGe commented on code in PR #19377:
URL: https://github.com/apache/flink/pull/19377#discussion_r846138318


##########
flink-architecture-tests/flink-architecture-tests-production/src/main/java/org/apache/flink/architecture/rules/ConnectorRules.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.architecture.rules;
+
+import com.tngtech.archunit.base.DescribedPredicate;
+import com.tngtech.archunit.core.domain.JavaClass;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.lang.ArchRule;
+import com.tngtech.archunit.thirdparty.com.google.common.base.Joiner;
+
+import static com.tngtech.archunit.base.DescribedPredicate.not;
+import static com.tngtech.archunit.core.domain.JavaModifier.PUBLIC;
+import static com.tngtech.archunit.core.domain.properties.HasModifiers.Predicates.modifier;
+import static com.tngtech.archunit.library.freeze.FreezingArchRule.freeze;
+import static org.apache.flink.architecture.common.GivenJavaClasses.noJavaClassesThat;
+import static org.apache.flink.architecture.common.SourcePredicates.areProductionCode;
+
+/** Rules for Flink connectors. */
+public class ConnectorRules {
+    private static final String[] CONNECTOR_PACKAGES = {
+        "org.apache.flink.connector..", "org.apache.flink.streaming.connectors.."
+    };
+
+    private static DescribedPredicate<JavaClass> areNotPublicAndResideOutsideOfPackages(
+            String... packageIdentifiers) {
+        return JavaClass.Predicates.resideOutsideOfPackages(packageIdentifiers)
+                .and(not(modifier(PUBLIC)))

Review Comment:
   then we should include publicEvolving here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] alpreu commented on a diff in pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
alpreu commented on code in PR #19377:
URL: https://github.com/apache/flink/pull/19377#discussion_r855898749


##########
flink-architecture-tests/flink-architecture-tests-production/src/main/java/org/apache/flink/architecture/rules/ConnectorRules.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.architecture.rules;
+
+import org.apache.flink.annotation.Public;
+import org.apache.flink.annotation.PublicEvolving;
+
+import com.tngtech.archunit.base.DescribedPredicate;
+import com.tngtech.archunit.core.domain.JavaClass;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.lang.ArchRule;
+import com.tngtech.archunit.thirdparty.com.google.common.base.Joiner;
+
+import static com.tngtech.archunit.base.DescribedPredicate.not;
+import static com.tngtech.archunit.core.domain.JavaModifier.PUBLIC;
+import static com.tngtech.archunit.core.domain.properties.HasModifiers.Predicates.modifier;
+import static com.tngtech.archunit.library.freeze.FreezingArchRule.freeze;
+import static org.apache.flink.architecture.common.GivenJavaClasses.noJavaClassesThat;
+import static org.apache.flink.architecture.common.Predicates.areDirectlyAnnotatedWithAtLeastOneOf;
+import static org.apache.flink.architecture.common.SourcePredicates.areProductionCode;
+
+/** Rules for Flink connectors. */
+public class ConnectorRules {
+    private static final String[] CONNECTOR_PACKAGES = {
+        "org.apache.flink.connector..", "org.apache.flink.streaming.connectors.."
+    };
+
+    private static DescribedPredicate<JavaClass> areNotPublicAndResideOutsideOfPackages(
+            String... packageIdentifiers) {
+        return JavaClass.Predicates.resideOutsideOfPackages(packageIdentifiers)
+                .and(
+                        not(areDirectlyAnnotatedWithAtLeastOneOf(
+                                        Public.class, PublicEvolving.class))
+                                .and(not(modifier(PUBLIC))))

Review Comment:
   I see, yeah let's just remove it, we can always add it back later



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] MartijnVisser merged pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
MartijnVisser merged PR #19377:
URL: https://github.com/apache/flink/pull/19377


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] alpreu commented on a diff in pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
alpreu commented on code in PR #19377:
URL: https://github.com/apache/flink/pull/19377#discussion_r846082122


##########
flink-architecture-tests/flink-architecture-tests-production/src/main/java/org/apache/flink/architecture/rules/ConnectorRules.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.architecture.rules;
+
+import com.tngtech.archunit.base.DescribedPredicate;
+import com.tngtech.archunit.core.domain.JavaClass;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.lang.ArchRule;
+import com.tngtech.archunit.thirdparty.com.google.common.base.Joiner;
+
+import static com.tngtech.archunit.base.DescribedPredicate.not;
+import static com.tngtech.archunit.core.domain.JavaModifier.PUBLIC;
+import static com.tngtech.archunit.core.domain.properties.HasModifiers.Predicates.modifier;
+import static com.tngtech.archunit.library.freeze.FreezingArchRule.freeze;
+import static org.apache.flink.architecture.common.GivenJavaClasses.noJavaClassesThat;
+import static org.apache.flink.architecture.common.SourcePredicates.areProductionCode;
+
+/** Rules for Flink connectors. */
+public class ConnectorRules {
+    private static final String[] CONNECTOR_PACKAGES = {
+        "org.apache.flink.connector..", "org.apache.flink.streaming.connectors.."
+    };
+
+    private static DescribedPredicate<JavaClass> areNotPublicAndResideOutsideOfPackages(
+            String... packageIdentifiers) {
+        return JavaClass.Predicates.resideOutsideOfPackages(packageIdentifiers)
+                .and(not(modifier(PUBLIC)))

Review Comment:
   The JavaDoc for PublicEvolving states
   ``` 
   Classes and methods with this annotation are intended for public use and have stable behavior.
   However, their interfaces and signatures are not considered to be stable and might be changed
   across versions.```
   
   So I think connector developers should be able to rely on them. For example the new `Sink` interface is still PublicEvolving, so if we prohibit this then no connector could depend on it for at least another release cycle



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] JingGe commented on a diff in pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
JingGe commented on code in PR #19377:
URL: https://github.com/apache/flink/pull/19377#discussion_r843980119


##########
flink-architecture-tests/flink-architecture-tests-production/src/main/java/org/apache/flink/architecture/rules/ConnectorRules.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.architecture.rules;
+
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.annotation.Internal;
+
+import com.tngtech.archunit.base.DescribedPredicate;
+import com.tngtech.archunit.core.domain.JavaClass;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.lang.ArchRule;
+import com.tngtech.archunit.thirdparty.com.google.common.base.Joiner;
+
+import static com.tngtech.archunit.base.DescribedPredicate.not;
+import static com.tngtech.archunit.core.domain.JavaModifier.PUBLIC;
+import static com.tngtech.archunit.core.domain.properties.HasModifiers.Predicates.modifier;
+import static com.tngtech.archunit.library.freeze.FreezingArchRule.freeze;
+import static org.apache.flink.architecture.common.GivenJavaClasses.noJavaClassesThat;
+import static org.apache.flink.architecture.common.SourcePredicates.areProductionCode;
+
+/** Rules for Flink connectors. */
+public class ConnectorRules {
+    private static final String[] CONNECTOR_PACKAGES = {
+        "org.apache.flink.connector..", "org.apache.flink.streaming.connectors.."
+    };
+
+    private static DescribedPredicate<JavaClass> areNotPublicAndResideOutsideOfPackages(
+            String... packageIdentifiers) {
+        return JavaClass.Predicates.resideOutsideOfPackages(packageIdentifiers)
+                .and(not(modifier(PUBLIC)))
+                .as(
+                        "are not public and reside outside of packages ['%s']",
+                        Joiner.on("', '").join(packageIdentifiers));
+    }
+
+    @ArchTest
+    public static final ArchRule CONNECTOR_CLASSES_ONLY_DEPEND_ON_PUBLIC_API =
+            freeze(
+                    noJavaClassesThat(areProductionCode())
+                            .and()
+                            .resideInAnyPackage(CONNECTOR_PACKAGES)
+                            .and()
+                            .arePublic()
+                            .and()
+                            .areNotAnnotatedWith(Internal.class)
+                            .and()
+                            .areNotAnnotatedWith(Deprecated.class)
+                            .and()
+                            .areNotAnnotatedWith(Experimental.class)
+                            .should()
+                            .dependOnClassesThat(

Review Comment:
   hmmm, does the code mean "Connector production code should depend on non public API outside of CONNECTOR_PACKAGES"?



##########
flink-architecture-tests/flink-architecture-tests-production/src/main/java/org/apache/flink/architecture/rules/ConnectorRules.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.architecture.rules;
+
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.annotation.Internal;
+
+import com.tngtech.archunit.base.DescribedPredicate;
+import com.tngtech.archunit.core.domain.JavaClass;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.lang.ArchRule;
+import com.tngtech.archunit.thirdparty.com.google.common.base.Joiner;
+
+import static com.tngtech.archunit.base.DescribedPredicate.not;
+import static com.tngtech.archunit.core.domain.JavaModifier.PUBLIC;
+import static com.tngtech.archunit.core.domain.properties.HasModifiers.Predicates.modifier;
+import static com.tngtech.archunit.library.freeze.FreezingArchRule.freeze;
+import static org.apache.flink.architecture.common.GivenJavaClasses.noJavaClassesThat;
+import static org.apache.flink.architecture.common.SourcePredicates.areProductionCode;
+
+/** Rules for Flink connectors. */
+public class ConnectorRules {
+    private static final String[] CONNECTOR_PACKAGES = {
+        "org.apache.flink.connector..", "org.apache.flink.streaming.connectors.."
+    };
+
+    private static DescribedPredicate<JavaClass> areNotPublicAndResideOutsideOfPackages(
+            String... packageIdentifiers) {
+        return JavaClass.Predicates.resideOutsideOfPackages(packageIdentifiers)
+                .and(not(modifier(PUBLIC)))
+                .as(
+                        "are not public and reside outside of packages ['%s']",
+                        Joiner.on("', '").join(packageIdentifiers));
+    }
+
+    @ArchTest
+    public static final ArchRule CONNECTOR_CLASSES_ONLY_DEPEND_ON_PUBLIC_API =
+            freeze(
+                    noJavaClassesThat(areProductionCode())
+                            .and()
+                            .resideInAnyPackage(CONNECTOR_PACKAGES)
+                            .and()
+                            .arePublic()
+                            .and()
+                            .areNotAnnotatedWith(Internal.class)
+                            .and()
+                            .areNotAnnotatedWith(Deprecated.class)
+                            .and()
+                            .areNotAnnotatedWith(Experimental.class)

Review Comment:
   Both Internal and experimental classes should follow this rule too.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] alpreu commented on a diff in pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
alpreu commented on code in PR #19377:
URL: https://github.com/apache/flink/pull/19377#discussion_r855846859


##########
flink-architecture-tests/flink-architecture-tests-production/src/main/java/org/apache/flink/architecture/rules/ConnectorRules.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.architecture.rules;
+
+import org.apache.flink.annotation.Public;
+import org.apache.flink.annotation.PublicEvolving;
+
+import com.tngtech.archunit.base.DescribedPredicate;
+import com.tngtech.archunit.core.domain.JavaClass;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.lang.ArchRule;
+import com.tngtech.archunit.thirdparty.com.google.common.base.Joiner;
+
+import static com.tngtech.archunit.base.DescribedPredicate.not;
+import static com.tngtech.archunit.core.domain.JavaModifier.PUBLIC;
+import static com.tngtech.archunit.core.domain.properties.HasModifiers.Predicates.modifier;
+import static com.tngtech.archunit.library.freeze.FreezingArchRule.freeze;
+import static org.apache.flink.architecture.common.GivenJavaClasses.noJavaClassesThat;
+import static org.apache.flink.architecture.common.Predicates.areDirectlyAnnotatedWithAtLeastOneOf;
+import static org.apache.flink.architecture.common.SourcePredicates.areProductionCode;
+
+/** Rules for Flink connectors. */
+public class ConnectorRules {
+    private static final String[] CONNECTOR_PACKAGES = {
+        "org.apache.flink.connector..", "org.apache.flink.streaming.connectors.."
+    };
+
+    private static DescribedPredicate<JavaClass> areNotPublicAndResideOutsideOfPackages(
+            String... packageIdentifiers) {
+        return JavaClass.Predicates.resideOutsideOfPackages(packageIdentifiers)
+                .and(
+                        not(areDirectlyAnnotatedWithAtLeastOneOf(
+                                        Public.class, PublicEvolving.class))
+                                .and(not(modifier(PUBLIC))))

Review Comment:
   I think it's fair to keep it, annotated classes should also be public java classes



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] MartijnVisser commented on pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
MartijnVisser commented on PR #19377:
URL: https://github.com/apache/flink/pull/19377#issuecomment-1122308024

   @flinkbot run azure


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] MartijnVisser commented on a diff in pull request #19377: [FLINK-27048][test] Add ArchUnit rule that connectors should only depend on public API

Posted by GitBox <gi...@apache.org>.
MartijnVisser commented on code in PR #19377:
URL: https://github.com/apache/flink/pull/19377#discussion_r847088827


##########
flink-architecture-tests/flink-architecture-tests-production/src/main/java/org/apache/flink/architecture/rules/ConnectorRules.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.architecture.rules;
+
+import com.tngtech.archunit.base.DescribedPredicate;
+import com.tngtech.archunit.core.domain.JavaClass;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.lang.ArchRule;
+import com.tngtech.archunit.thirdparty.com.google.common.base.Joiner;
+
+import static com.tngtech.archunit.base.DescribedPredicate.not;
+import static com.tngtech.archunit.core.domain.JavaModifier.PUBLIC;
+import static com.tngtech.archunit.core.domain.properties.HasModifiers.Predicates.modifier;
+import static com.tngtech.archunit.library.freeze.FreezingArchRule.freeze;
+import static org.apache.flink.architecture.common.GivenJavaClasses.noJavaClassesThat;
+import static org.apache.flink.architecture.common.SourcePredicates.areProductionCode;
+
+/** Rules for Flink connectors. */
+public class ConnectorRules {
+    private static final String[] CONNECTOR_PACKAGES = {
+        "org.apache.flink.connector..", "org.apache.flink.streaming.connectors.."
+    };
+
+    private static DescribedPredicate<JavaClass> areNotPublicAndResideOutsideOfPackages(
+            String... packageIdentifiers) {
+        return JavaClass.Predicates.resideOutsideOfPackages(packageIdentifiers)
+                .and(not(modifier(PUBLIC)))

Review Comment:
   It should be number 2, based on the table for 'Existing guarantees' at https://cwiki.apache.org/confluence/display/FLINK/FLIP-196%3A+Source+API+stability+guarantees



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org