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/05/02 15:20:05 UTC

[GitHub] [flink] JingGe opened a new pull request, #19624: [FLINK-27476][test] build import option only focuses on maven main class files

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

   ## What is the purpose of the change
   
   ImportOption.DoNotIncludeTests.class used currently has some issue when running test with testContainer. It would be good to define the target class path precisely.
   
   
   ## Brief change log
   
     - build new import option `MavenMainClassesOnly` 
     - change pom.xml to use flink-shaded-guava
   
   
   ## Verifying this change
   
   `ImportOptions.MavenMainClassesOnly.class` could be used within @AnalyzeClasses(...)
   
   ## 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**)
   


-- 
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 pull request #19624: [FLINK-27476][test] build import option only focuses on maven main class files

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

   In the external connector repo, the `ProductionCodeArchitectureTest` has been moved to sub modules[1]. That's why we don't have the same issue in the flink repo. Violations could be found at [2] 
   
   
   [1] https://github.com/apache/flink-connector-elasticsearch/pull/10/files#diff-2f93acbdbdf55f0be081d4c7760ed5bd67706ab609b0657b63a8c031dd4dd55a
   [2] https://github.com/apache/flink-connector-elasticsearch/runs/6206623054?check_suite_focus=true#step:6:695


-- 
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 pull request #19624: [FLINK-27476][test] build import option only focuses on maven main class files

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

   > 
   
   there are other ImportOptions that are working on jar and archive: `ImportOption.DoNotIncludeJars` and `ImportOption.DoNotIncludeArchives`. I think the current `SourcePredicates#targetCode` is fine and could be used with them together, if we want to exclude jars.


-- 
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 pull request #19624: [FLINK-27476][test] build import option only focuses on maven main class files

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

   > Could you expand on what concrete issue you're trying to solve?
   > 
   > There is supposed to be _some_ issue with testcontainers; what exactly is it?
   
   The excluding style import option just excludes some known paths which is not a stable solution. It will break if, in this case, any test classes are found in other path. I have running the same archunit test 1. alone, 2. in Idea, 3. with ITCase which needs testContainer together. Only the third one failed. 


-- 
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 pull request #19624: [FLINK-27476][test] build import option only focuses on maven main class files

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

   > Have you tried whether `ImportOption.DoNotIncludeJars` would fix the issue?
   
   Yes, that could fix it. But I think the import option I built in this PR is still very helpful for the future development, because it is more precise. WDYT?


-- 
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 pull request #19624: [FLINK-27476][test] build import option only focuses on maven main class files

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

   @Airblader After talking with @zentol offline and checking the archunit doc again. I think, theoretically, class path should only be managed by the test via `@AnalyzeClasses(...ImportOptions...)`. It should work without adding the SourcePredicates#areProductionCode logic into the rule definition. There might be some corner case I am not aware of. May I have your thoughts of why developing the `SourcePredicates`? Is there anything that `SourcePredicates` could do but customized `ImportOption`s could not do? 


-- 
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 #19624: [FLINK-27476][test] build import option only focuses on maven main class files

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb0bc6c55a5928446f435eb49932ed478f6adc07",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "fb0bc6c55a5928446f435eb49932ed478f6adc07",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fb0bc6c55a5928446f435eb49932ed478f6adc07 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] zentol commented on pull request #19624: [FLINK-27476][test] build import option only focuses on maven main class files

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

   So where does archunit pick up the es-base classes from? Does it just read the classpath?
   
   I see that `SourcePredicates#areProductionCode` selects everything that doesn't match certain (test) paths; so if it scans a jar this naturally fails.
   Should we maybe adjust `SourcePredicates#targetCode` to additionally filter everything coming from a jar?


-- 
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 pull request #19624: [FLINK-27476][test] build import option only focuses on maven main class files

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

   @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] zentol commented on pull request #19624: [FLINK-27476][test] build import option only focuses on maven main class files

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

   If you use the new import option the source predicate filters nothing; if anything it should cost performance.


-- 
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] zentol commented on pull request #19624: [FLINK-27476][test] build import option only focuses on maven main class files

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

   Have you tried whether `ImportOption.DoNotIncludeJars` would fix the issue?


-- 
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 #19624: [FLINK-27476][test] build import option only focuses on maven main class files

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


##########
flink-architecture-tests/flink-architecture-tests-base/src/main/java/org/apache/flink/architecture/common/ImportOptions.java:
##########
@@ -57,4 +70,19 @@ public boolean includes(Location location) {
             return !location.matches(SHADED);
         }
     }
+
+    /** {@link Pattern} based predicate. */
+    private static class PatternPredicate implements Predicate<Location> {

Review Comment:
   fair enough, thanks!



-- 
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] zentol commented on pull request #19624: [FLINK-27476][test] build import option only focuses on maven main class files

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

   I don't necessarily mind it, but do think that we should figure out the responsibilities a bit.
   
   Currently, a test can select certain classes to test via the ImportOptions. This newly added import option would push the responsibility of selecting production code to the test.
   But then we have rules (like the Visibility thing) that select production code on their own.
   
   So who's responsible for selecting production source code?
   
   And if we're going with both the user and rule having the capability to do so, then they should rely on the same logic.


-- 
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] Airblader commented on pull request #19624: [FLINK-27476][test] build import option only focuses on maven main class files

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

   Originally this was done through the predicate because the idea was to build the class graph only once. Since test code is now tested per-module anyway, I don't see a reason why this couldn't instead be done on the classpath scanning level already.


-- 
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] zentol commented on a diff in pull request #19624: [FLINK-27476][test] build import option only focuses on maven main class files

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


##########
flink-architecture-tests/flink-architecture-tests-base/src/main/java/org/apache/flink/architecture/common/ImportOptions.java:
##########
@@ -57,4 +70,19 @@ public boolean includes(Location location) {
             return !location.matches(SHADED);
         }
     }
+
+    /** {@link Pattern} based predicate. */
+    private static class PatternPredicate implements Predicate<Location> {

Review Comment:
   this class seems unnecessary. Why not just do it like `ExcludeShadedImportOption`.



##########
flink-architecture-tests/flink-architecture-tests-base/src/main/java/org/apache/flink/architecture/common/ImportOptions.java:
##########
@@ -18,6 +18,8 @@
 
 package org.apache.flink.architecture.common;
 
+import org.apache.flink.shaded.guava30.com.google.common.base.Predicate;

Review Comment:
   why would you use guava for this when https://docs.oracle.com/javase/8/docs/api/java/util/function/Predicate.html exists? 



-- 
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] zentol merged pull request #19624: [FLINK-27476][test] build import option only focuses on maven main class files

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


-- 
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 pull request #19624: [FLINK-27476][test] build import option only focuses on maven main class files

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

   Thanks for the feedback. If I understood you correctly, we should keep the `SourcePredicates#areProductionCode` in the production code test rule definition to provide better performance, shouldn't we?


-- 
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] zentol commented on pull request #19624: [FLINK-27476][test] build import option only focuses on maven main class files

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

   Can you give me an example for one such "other path" (and ideally a violation as well)? I'm curious why we'd have issues now with the external repo when the same tests seem to work just fine in the flink repo.


-- 
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] zentol commented on pull request #19624: [FLINK-27476][test] build import option only focuses on maven main class files

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

   Like this:
   `.map(location -> !location.isJar() && predicates.includes(location))`


-- 
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 pull request #19624: [FLINK-27476][test] build import option only focuses on maven main class files

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

   Conceptually, you are right. As far as I understood, they have two different responsibilities. `SourcePredicates` are used to build the rules with their default classes scope. Same Rule will be used in different tests and each test will have its individual classes scope via `@AnalyzeClasses(...ImportOptions...)`and might be different from case to case even using the same rule. In the end, both of them together are responsible for defining the classes scope when the test is running. 


-- 
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 pull request #19624: [FLINK-27476][test] build import option only focuses on maven main class files

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

   Thanks for your feedback @zentol, I addressed your comments, please take a look.


-- 
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