You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@maven.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2022/10/14 14:01:00 UTC

[jira] [Commented] (MRESOLVER-278) BREAKING: Make Session extend AutoCloseable (and introduce onCloseHandlers)

    [ https://issues.apache.org/jira/browse/MRESOLVER-278?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17617747#comment-17617747 ] 

ASF GitHub Bot commented on MRESOLVER-278:
------------------------------------------

michael-o commented on code in PR #201:
URL: https://github.com/apache/maven-resolver/pull/201#discussion_r995792085


##########
maven-resolver-impl/src/main/java/org/eclipse/aether/internal/impl/checksum/SummaryFileTrustedChecksumsSource.java:
##########
@@ -245,4 +255,35 @@ public void close()
             // nop
         }
     }
+
+    @SuppressWarnings( "unchecked" )
+    private void saveSessionRecordedLines( RepositorySystemSession session )
+    {
+        Map<Path, Set<String>> recordedLines = (Map<Path, Set<String>>) session.getData().get( RECORDING_KEY );
+        if ( recordedLines == null || recordedLines.isEmpty() )
+        {
+            return;
+        }
+
+        ArrayList<Exception> exceptions = new ArrayList<>();
+        for ( Map.Entry<Path, Set<String>> entry : recordedLines.entrySet() )
+        {
+            Path file = entry.getKey();
+            Set<String> lines = entry.getValue();
+            if ( !lines.isEmpty() )
+            {
+                LOGGER.info( "Saving {} checksums to '{}'", lines.size(), file );
+                try
+                {
+                    Files.createDirectories( file.getParent() );
+                    Files.write( file, lines );

Review Comment:
   This is now truly ONE file per checksum algo?



##########
maven-resolver-api/src/test/java/org/eclipse/aether/DefaultRepositorySystemSessionTest.java:
##########
@@ -140,7 +140,10 @@ public void testCopyRepositorySystemSession() throws Exception
 
         for ( Method method : methods )
         {
-            assertEquals( method.getName(), method.invoke( session ) == null, method.invoke( newSession ) == null );
+            if ( method.getParameterCount() == 0 )
+            {
+                assertEquals( method.getName(), method.invoke( session ) == null, method.invoke( newSession ) == null );
+            }

Review Comment:
   This is an unrelated cleanup, no?



##########
maven-resolver-api/src/main/java/org/eclipse/aether/DefaultRepositorySystemSession.java:
##########
@@ -873,4 +888,41 @@ public Collection<FileTransformer> getTransformersForArtifact( Artifact artifact
         }
     }
 
+    private final CopyOnWriteArrayList<Consumer<RepositorySystemSession>> onCloseHandlers
+            = new CopyOnWriteArrayList<>();
+
+    @Override
+    public void addOnCloseHandler( Consumer<RepositorySystemSession> handler )
+    {
+        verifyStateForMutation();
+        requireNonNull( handler, "handler cannot be null" );
+        onCloseHandlers.add( 0, handler );
+    }
+
+    @Override
+    public boolean isClosed()
+    {
+        return closed.get();
+    }
+
+    @Override
+    public void close()
+    {
+        if ( closed.compareAndSet( false, true ) )
+        {
+            ArrayList<Exception> exceptions = new ArrayList<>();
+            for ( Consumer<RepositorySystemSession> onCloseHandler : onCloseHandlers )
+            {
+                try
+                {
+                    onCloseHandler.accept( this );
+                }
+                catch ( Exception e )
+                {
+                    exceptions.add( e );
+                }
+            }
+            MultiRuntimeException.mayThrow( "session onClose handler failures", exceptions );

Review Comment:
   on-close



##########
maven-resolver-api/src/main/java/org/eclipse/aether/DefaultRepositorySystemSession.java:
##########
@@ -873,4 +888,41 @@ public Collection<FileTransformer> getTransformersForArtifact( Artifact artifact
         }
     }
 
+    private final CopyOnWriteArrayList<Consumer<RepositorySystemSession>> onCloseHandlers
+            = new CopyOnWriteArrayList<>();
+
+    @Override
+    public void addOnCloseHandler( Consumer<RepositorySystemSession> handler )
+    {
+        verifyStateForMutation();
+        requireNonNull( handler, "handler cannot be null" );
+        onCloseHandlers.add( 0, handler );

Review Comment:
   Shouldn't this be just like with locks: FILO?



##########
maven-resolver-impl/src/main/java/org/eclipse/aether/internal/impl/checksum/SummaryFileTrustedChecksumsSource.java:
##########
@@ -201,41 +214,38 @@ private String calculateSummaryPath( boolean originAware,
         return fileName + "." + checksumAlgorithmFactory.getFileExtension();
     }
 
-    /**
-     * Note: this implementation will work only in single-thread (T1) model. While not ideal, the "workaround" is
-     * possible in both, Maven and Maven Daemon: force single threaded execution model while "recording" (in mvn:
-     * do not pass any {@code -T} CLI parameter, while for mvnd use {@code -1} CLI parameter.
-     * 
-     * TODO: this will need to be reworked for at least two reasons: a) avoid duplicates in summary file and b)
-     * support multi threaded builds (probably will need "on session close" hook).
-     */
     private class SummaryFileWriter implements Writer
     {
         private final Path basedir;
 
         private final boolean originAware;
 
-        private SummaryFileWriter( Path basedir, boolean originAware )
+        private final ConcurrentHashMap<Path, Set<String>> recordedLines;
+
+        private SummaryFileWriter( Path basedir,
+                                   boolean originAware,
+                                   ConcurrentHashMap<Path, Set<String>> recordedLines )
         {
             this.basedir = basedir;
             this.originAware = originAware;
+            this.recordedLines = recordedLines;
         }
 
         @Override
         public void addTrustedArtifactChecksums( Artifact artifact, ArtifactRepository artifactRepository,
                                                  List<ChecksumAlgorithmFactory> checksumAlgorithmFactories,
-                                                 Map<String, String> trustedArtifactChecksums ) throws IOException
+                                                 Map<String, String> trustedArtifactChecksums )
         {
             for ( ChecksumAlgorithmFactory checksumAlgorithmFactory : checksumAlgorithmFactories )
             {
                 String checksum = requireNonNull(
                         trustedArtifactChecksums.get( checksumAlgorithmFactory.getName() ) );
-                String summaryLine = ArtifactIdUtils.toId( artifact ) + " " + checksum + "\n";
+                String summaryLine = ArtifactIdUtils.toId( artifact ) + " " + checksum;
                 Path summaryPath = basedir.resolve(
                         calculateSummaryPath( originAware, artifactRepository, checksumAlgorithmFactory ) );
-                Files.createDirectories( summaryPath.getParent() );
-                Files.write( summaryPath, summaryLine.getBytes( StandardCharsets.UTF_8 ),
-                        StandardOpenOption.CREATE, StandardOpenOption.WRITE, StandardOpenOption.APPEND );
+
+                recordedLines.computeIfAbsent( summaryPath, p -> Collections.synchronizedSet( new TreeSet<>() ) )
+                        .add( summaryLine );

Review Comment:
   Do we really need it sorted or will insertion order just suffice?



##########
maven-resolver-impl/src/main/java/org/eclipse/aether/internal/impl/synccontext/DefaultSyncContextFactory.java:
##########
@@ -122,50 +114,39 @@ public void initService( final ServiceLocator locator )
     public SyncContext newInstance( final RepositorySystemSession session, final boolean shared )
     {
         requireNonNull( session, "session cannot be null" );
-        NamedLockFactoryAdapter adapter =
-                (NamedLockFactoryAdapter) session.getData().computeIfAbsent(
-                        ADAPTER_KEY,
-                        () -> createAdapter( session )
-                );
+        NamedLockFactoryAdapter adapter = getOrCreateSessionAdapter( session );
         return adapter.newInstance( session, shared );
     }
 
-    private NamedLockFactoryAdapter createAdapter( final RepositorySystemSession session )
+    private NamedLockFactoryAdapter getOrCreateSessionAdapter( final RepositorySystemSession session )
     {
-        String nameMapperName = ConfigUtils.getString( session, DEFAULT_NAME_MAPPER_NAME, NAME_MAPPER_KEY );
-        String namedLockFactoryName = ConfigUtils.getString( session, DEFAULT_FACTORY_NAME, FACTORY_KEY );
-        NameMapper nameMapper = nameMappers.get( nameMapperName );
-        if ( nameMapper == null )
-        {
-            throw new IllegalArgumentException( "Unknown NameMapper name: " + nameMapperName
-                    + ", known ones: " + nameMappers.keySet() );
-        }
-        NamedLockFactory namedLockFactory = namedLockFactories.get( namedLockFactoryName );
-        if ( namedLockFactory == null )
+        return (NamedLockFactoryAdapter) session.getData().computeIfAbsent( ADAPTER_KEY, () ->
         {
-            throw new IllegalArgumentException( "Unknown NamedLockFactory name: " + namedLockFactoryName
-                    + ", known ones: " + namedLockFactories.keySet() );
-        }
-        NamedLockFactoryAdapter adapter = new NamedLockFactoryAdapter( nameMapper, namedLockFactory );
-        createdAdapters.add( adapter );
-        return adapter;
+            String nameMapperName = ConfigUtils.getString( session, DEFAULT_NAME_MAPPER_NAME, NAME_MAPPER_KEY );
+            String namedLockFactoryName = ConfigUtils.getString( session, DEFAULT_FACTORY_NAME, FACTORY_KEY );
+            NameMapper nameMapper = nameMappers.get( nameMapperName );
+            if ( nameMapper == null )
+            {
+                throw new IllegalArgumentException( "Unknown NameMapper name: " + nameMapperName
+                        + ", known ones: " + nameMappers.keySet() );
+            }
+            NamedLockFactory namedLockFactory = namedLockFactories.get( namedLockFactoryName );
+            if ( namedLockFactory == null )
+            {
+                throw new IllegalArgumentException( "Unknown NamedLockFactory name: " + namedLockFactoryName
+                        + ", known ones: " + namedLockFactories.keySet() );
+            }
+            session.addOnCloseHandler( this::shutDownSessionAdapter );
+            return new NamedLockFactoryAdapter( nameMapper, namedLockFactory );
+        } );
     }
 
-    @PreDestroy
-    public void shutdown()
+    private void shutDownSessionAdapter( RepositorySystemSession session )
     {
-        LOGGER.debug( "Shutting down created adapters..." );
-        createdAdapters.forEach( adapter ->
-                {
-                    try
-                    {
-                        adapter.shutdown();
-                    }
-                    catch ( Exception e )
-                    {
-                        LOGGER.warn( "Could not shutdown: {}", adapter, e );
-                    }
-                }
-        );
+        NamedLockFactoryAdapter adapter = (NamedLockFactoryAdapter) session.getData().get( ADAPTER_KEY );
+        if ( adapter != null )
+        {
+            adapter.shutdown();
+        }

Review Comment:
   So the adapter showdown is now handled by the AutoCloseable for each?



##########
maven-resolver-api/src/main/java/org/eclipse/aether/RepositorySystemSession.java:
##########
@@ -271,4 +272,57 @@
     @Deprecated
     FileTransformerManager getFileTransformerManager();
 
+    /**
+     * Adds "on close" handler to this session, it must not be {@code null}. Note: when handlers are invoked, the
+     * passed in (this) session is ALREADY CLOSED (the {@link #isClosed()} method returns {@code true}). This implies,
+     * that handlers cannot use {@link RepositorySystem} to resolve/collect/and so on, handlers are meant to perform
+     * some internal cleanup on session close. Attempt to add handler to closed session will throw.
+     *
+     * @since TBD
+     */
+    void addOnCloseHandler( Consumer<RepositorySystemSession> handler );
+
+    /**
+     * Returns {@code true} if this instance was already closed. Closed sessions should NOT be used anymore.
+     *
+     * @return {@code true} if session was closed.
+     * @since TBD
+     */
+    boolean isClosed();
+
+    /**
+     * Closes this session and possibly releases related resources by invoking "on close" handlers added by
+     * {@link #addOnCloseHandler(Consumer<RepositorySystemSession>)} method. This method may be invoked multiple times,
+     * but only first invocation will actually invoke handlers, subsequent invocations will be no-op.
+     * <p>
+     * When close action happens, all the registered handlers will be invoked (even if some throws), and at end
+     * of operation a {@link MultiRuntimeException} may be thrown signaling that some handler(s) failed. This exception
+     * may be ignored, is at the discretion of caller.
+     * <p>
+     * Important: ideally it is the session creator who should be responsible to close the session. The "nested"
+     * (delegating, wrapped) sessions {@link AbstractForwardingRepositorySystemSession} and alike) by default
+     * (without overriding the {@link AbstractForwardingRepositorySystemSession#close()} method) are prevented to close
+     * session, and it is the "recommended" behaviour as well. On the other hand, "nested" session may receive new
+     * "on close" handler registrations, but those registrations are passed to delegated session, and will be invoked
+     * once the "top level" (delegated) session is closed.
+     * <p>
+     * New session "copy" instances created using copy-constructor
+     * {@link DefaultRepositorySystemSession#DefaultRepositorySystemSession(RepositorySystemSession)} result in new,
+     * independent session instances, and they do NOT share states like "read-only", "closed" and "on close handlers"
+     * with the copied session. Hence, they should be treated as "top level" sessions as well.
+     * <p>
+     * The recommended pattern for "top level" sessions is the usual try-with-resource:
+     *
+     * <pre> {@code
+     * try ( RepositorySystemSession session = create session... )

Review Comment:
   This confuses me. In Maven the repo session is passed around like a whore, how can this be achived in Maven? It can't, no?



##########
maven-resolver-impl/src/main/java/org/eclipse/aether/internal/impl/checksum/SummaryFileTrustedChecksumsSource.java:
##########
@@ -201,41 +214,38 @@ private String calculateSummaryPath( boolean originAware,
         return fileName + "." + checksumAlgorithmFactory.getFileExtension();
     }
 
-    /**
-     * Note: this implementation will work only in single-thread (T1) model. While not ideal, the "workaround" is
-     * possible in both, Maven and Maven Daemon: force single threaded execution model while "recording" (in mvn:
-     * do not pass any {@code -T} CLI parameter, while for mvnd use {@code -1} CLI parameter.
-     * 
-     * TODO: this will need to be reworked for at least two reasons: a) avoid duplicates in summary file and b)
-     * support multi threaded builds (probably will need "on session close" hook).
-     */
     private class SummaryFileWriter implements Writer
     {
         private final Path basedir;
 
         private final boolean originAware;
 
-        private SummaryFileWriter( Path basedir, boolean originAware )
+        private final ConcurrentHashMap<Path, Set<String>> recordedLines;
+
+        private SummaryFileWriter( Path basedir,
+                                   boolean originAware,
+                                   ConcurrentHashMap<Path, Set<String>> recordedLines )

Review Comment:
   Why do we need the semantics of a set?





> BREAKING: Make Session extend AutoCloseable (and introduce onCloseHandlers)
> ---------------------------------------------------------------------------
>
>                 Key: MRESOLVER-278
>                 URL: https://issues.apache.org/jira/browse/MRESOLVER-278
>             Project: Maven Resolver
>          Issue Type: New Feature
>          Components: Resolver
>            Reporter: Tamas Cservenak
>            Assignee: Tamas Cservenak
>            Priority: Major
>             Fix For: 1.9.0
>
>
> So far, in (vanilla) Maven, the lifecycle of session was on par with lifecycle of SISU container, as Maven does something like this:
>  * boot, create container
>  * create session
>  * work
>  * destroy container
>  * exit JVM
> So, Maven execution is 1 session 1 container, are on par.
> This is not true for cases where container (and resolver components) are reused across several sessions, like mvnd does. Also, current code on master (named locks adapter) uses {{@PreDestroy}} to shut down adapters, that is invoked when container is shut down, while the adapters are created per-session. This means that long-living mvnd daemons will shut down the unused adapter only when daemon itself is shut down, even is session for which adapter was created is long gone/done.
> While Maven has "session scoped" notion, resolver has not. Hence, simplest and cleanest solution is to make RepositorySystemSession have a method to "close", denoting that "this session is done". Also, if we can provide hooks for "onSessionClose", this resolves all the problems, as for example the adapter, that is created per session, could be now cleanly shut down at session end.
> One gotcha: this change implies a {*}breaking change for integrators of resolver{*}:  integrator should make sure they close the session after they are done with it.
> Example changes needed for resolver users: [https://github.com/apache/maven/pull/822]
> The "pattern" to make use of this feature in resolver is following:
>  * stuff something into session data, ideally using computeWhenAbsent
>  * if absent, register a callback hook as well
>  * in callback, get the stuffed thing from session and if present, do something with it



--
This message was sent by Atlassian Jira
(v8.20.10#820010)