You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by "Flaugh24 (via GitHub)" <gi...@apache.org> on 2023/05/30 17:42:16 UTC

[GitHub] [ignite-3] Flaugh24 opened a new pull request, #2121: IGNITE-19565 wip

Flaugh24 opened a new pull request, #2121:
URL: https://github.com/apache/ignite-3/pull/2121

   (no 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: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] Pochatkin commented on a diff in pull request #2121: IGNITE-19565 Use JobClassLoader for loading job classes

Posted by "Pochatkin (via GitHub)" <gi...@apache.org>.
Pochatkin commented on code in PR #2121:
URL: https://github.com/apache/ignite-3/pull/2121#discussion_r1218074181


##########
modules/code-deployment/src/main/java/org/apache/ignite/internal/deployunit/DeploymentManagerImpl.java:
##########
@@ -262,8 +262,15 @@ public CompletableFuture<UnitStatuses> statusAsync(String id) {
     }
 
     @Override
-    public Path path(String id, Version version) {
-        return deployer.unitPath(id, version);
+    public CompletableFuture<Path> path(String id, Version version) {
+        return CompletableFuture.supplyAsync(() -> {
+            Path path = deployer.unitPath(id, version);
+            if (path.toFile().exists()) {

Review Comment:
   Files.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: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] Pochatkin commented on a diff in pull request #2121: IGNITE-19565 Use JobClassLoader for loading job classes

Posted by "Pochatkin (via GitHub)" <gi...@apache.org>.
Pochatkin commented on code in PR #2121:
URL: https://github.com/apache/ignite-3/pull/2121#discussion_r1217686673


##########
modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java:
##########
@@ -533,6 +528,28 @@ public class IgniteImpl implements Ignite {
 
         sql = new IgniteSqlImpl(qryEngine);
 
+        deploymentManager = new DeploymentManagerImpl(
+                clusterSvc,
+                metaStorageMgr,
+                workDir,
+                nodeConfigRegistry.getConfiguration(DeploymentConfiguration.KEY),
+                cmgMgr
+        );
+
+        JobClassLoaderFactory jobClassLoaderFactory = new JobClassLoaderFactory(
+                unitName -> {

Review Comment:
   With deploymentManager instance you may compute latest version in JobClassLoaderFactory. All Version instaces are comparable



-- 
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: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] Flaugh24 commented on pull request #2121: IGNITE-19565 Use JobClassLoader for loading job classes

Posted by "Flaugh24 (via GitHub)" <gi...@apache.org>.
Flaugh24 commented on PR #2121:
URL: https://github.com/apache/ignite-3/pull/2121#issuecomment-1572053328

   https://issues.apache.org/jira/browse/IGNITE-19565


-- 
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: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] Pochatkin commented on a diff in pull request #2121: IGNITE-19565 Use JobClassLoader for loading job classes

Posted by "Pochatkin (via GitHub)" <gi...@apache.org>.
Pochatkin commented on code in PR #2121:
URL: https://github.com/apache/ignite-3/pull/2121#discussion_r1218078510


##########
modules/compute/src/main/java/org/apache/ignite/internal/compute/JobClassLoaderFactory.java:
##########
@@ -71,22 +67,48 @@ public JobClassLoaderFactory(Function<String, Version> detectLastUnitVersion, Ig
      * @param units The units of the job.
      * @return The class loader.
      */
-    public JobClassLoader createClassLoader(List<DeploymentUnit> units) {
-        URL[] classPath = units.stream()
-                .map(this::constructPath)
-                .flatMap(JobClassLoaderFactory::collectClasspath)
-                .toArray(URL[]::new);
-
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("Classpath for job: " + Arrays.toString(classPath));
-        }
+    public CompletableFuture<JobClassLoader> createClassLoader(List<DeploymentUnit> units) {
+        Map<Integer, Stream<URL>> map = new TreeMap<>();
+
+        CompletableFuture[] futures = IntStream.range(0, units.size())
+                .mapToObj(id -> {
+                    return constructPath(units.get(id))
+                            .thenApply(JobClassLoaderFactory::collectClasspath)
+                            .thenApply(stream -> map.put(id, stream));
+                }).toArray(CompletableFuture[]::new);
+
+        return CompletableFuture.allOf(futures).thenApply(v -> {
+            return map.values().stream()
+                    .flatMap(Function.identity())
+                    .toArray(URL[]::new);
+        })
+                .thenApply(it -> new JobClassLoader(it, getClass().getClassLoader()))
+                .whenComplete((cl, err) -> {
+                    if (err != null) {
+                        LOG.error("Failed to create class loader", err);
+                    } else {
+                        System.out.println(map);

Review Comment:
   Remove redundant prints



-- 
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: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] PakhomovAlexander merged pull request #2121: IGNITE-19565 Use JobClassLoader for loading job classes

Posted by "PakhomovAlexander (via GitHub)" <gi...@apache.org>.
PakhomovAlexander merged PR #2121:
URL: https://github.com/apache/ignite-3/pull/2121


-- 
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: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] Pochatkin commented on a diff in pull request #2121: IGNITE-19565 Use JobClassLoader for loading job classes

Posted by "Pochatkin (via GitHub)" <gi...@apache.org>.
Pochatkin commented on code in PR #2121:
URL: https://github.com/apache/ignite-3/pull/2121#discussion_r1221710843


##########
modules/api/src/main/java/org/apache/ignite/compute/DeploymentUnit.java:
##########
@@ -41,6 +41,17 @@ public DeploymentUnit(String name, Version version) {
         this.version = version;
     }
 
+    /**
+     * Constructor.
+     *
+     * @param name Name of the deployment unit.
+     * @param version Version of the deployment unit.
+     */
+    public DeploymentUnit(String name, String version) {
+        this.name = name;

Review Comment:
   this(name, Version.parse());



-- 
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: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] Pochatkin commented on a diff in pull request #2121: IGNITE-19565 Use JobClassLoader for loading job classes

Posted by "Pochatkin (via GitHub)" <gi...@apache.org>.
Pochatkin commented on code in PR #2121:
URL: https://github.com/apache/ignite-3/pull/2121#discussion_r1218081042


##########
modules/compute/src/main/java/org/apache/ignite/internal/compute/JobClassLoaderFactory.java:
##########
@@ -71,22 +67,48 @@ public JobClassLoaderFactory(Function<String, Version> detectLastUnitVersion, Ig
      * @param units The units of the job.
      * @return The class loader.
      */
-    public JobClassLoader createClassLoader(List<DeploymentUnit> units) {
-        URL[] classPath = units.stream()
-                .map(this::constructPath)
-                .flatMap(JobClassLoaderFactory::collectClasspath)
-                .toArray(URL[]::new);
-
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("Classpath for job: " + Arrays.toString(classPath));
-        }
+    public CompletableFuture<JobClassLoader> createClassLoader(List<DeploymentUnit> units) {
+        Map<Integer, Stream<URL>> map = new TreeMap<>();
+
+        CompletableFuture[] futures = IntStream.range(0, units.size())
+                .mapToObj(id -> {
+                    return constructPath(units.get(id))
+                            .thenApply(JobClassLoaderFactory::collectClasspath)
+                            .thenApply(stream -> map.put(id, stream));
+                }).toArray(CompletableFuture[]::new);
+
+        return CompletableFuture.allOf(futures).thenApply(v -> {
+            return map.values().stream()
+                    .flatMap(Function.identity())
+                    .toArray(URL[]::new);
+        })
+                .thenApply(it -> new JobClassLoader(it, getClass().getClassLoader()))
+                .whenComplete((cl, err) -> {
+                    if (err != null) {
+                        LOG.error("Failed to create class loader", err);
+                    } else {
+                        System.out.println(map);
+                        System.out.println("Created class loader: " + cl);
+                        LOG.debug("Created class loader: {}", cl);
+                    }
+                });
+    }
 
-        return new JobClassLoader(classPath, getClass().getClassLoader());
+    private CompletableFuture<Path> constructPath(DeploymentUnit unit) {
+        return CompletableFuture.completedFuture(unit.version())
+                .thenCompose(version -> {
+                    if (version == Version.LATEST) {
+                        return lastVersion(unit.name());
+                    } else {
+                        return CompletableFuture.completedFuture(version);
+                    }
+                })
+                .thenCompose(version -> deployment.path(unit.name(), version));
     }
 
-    private Path constructPath(DeploymentUnit unit) {
-        Version version = unit.version() == Version.LATEST ? detectLastUnitVersion.apply(unit.name()) : unit.version();
-        return deployment.path(unit.name(), version);
+    private CompletableFuture<Version> lastVersion(String name) {
+        return deployment.versionsAsync(name)
+                .thenApply(versions -> versions.stream().max(Version::compareTo).orElseThrow());

Review Comment:
   Or else throw NoSuchElementException. Do you expect 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: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] PakhomovAlexander commented on a diff in pull request #2121: IGNITE-19565 Use JobClassLoader for loading job classes

Posted by "PakhomovAlexander (via GitHub)" <gi...@apache.org>.
PakhomovAlexander commented on code in PR #2121:
URL: https://github.com/apache/ignite-3/pull/2121#discussion_r1213344759


##########
modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeMessageTypes.java:
##########
@@ -35,4 +35,6 @@ public class ComputeMessageTypes {
      * Type for {@link ExecuteResponse}.
      */
     public static final short EXECUTE_RESPONSE = 1;
+
+    public static final short DEPLOYMENT_UNIT = 2;

Review Comment:
   add javadoc please



##########
modules/api/src/main/java/org/apache/ignite/compute/IgniteCompute.java:
##########
@@ -51,35 +41,7 @@ public interface IgniteCompute {
      * @param <R>      Job result type
      * @return CompletableFuture Job result.
      */
-    <R> CompletableFuture<R> execute(Set<ClusterNode> nodes, String jobClassName, Object... args);
-
-    /**
-     * Executes a job of the given class on the node where the given key is located. The node is a leader
-     * of the corresponding Raft group.
-     *
-     * @param tableName Name of the table whose key is used to determine the node to execute the job on.
-     * @param key Key that identifies the node to execute the job on.
-     * @param jobClass Class of the job to execute.
-     * @param args Arguments of the job.
-     * @param <R> Job result type.
-     * @return CompletableFuture Job result.
-     */
-    <R> CompletableFuture<R> executeColocated(String tableName, Tuple key, Class<? extends ComputeJob<R>> jobClass, Object... args);
-
-    /**
-     * Executes a job of the given class on the node where the given key is located. The node is a leader
-     * of the corresponding Raft group.
-     *
-     * @param tableName Name of the table whose key is used to determine the node to execute the job on.
-     * @param key Key that identifies the node to execute the job on.
-     * @param keyMapper Mapper used to map the key to a binary representation.
-     * @param jobClass Class of the job to execute.
-     * @param args Arguments of the job.
-     * @param <R> Job result type.
-     * @return CompletableFuture Job result.
-     */
-    <K, R> CompletableFuture<R> executeColocated(String tableName, K key, Mapper<K> keyMapper,
-                                                 Class<? extends ComputeJob<R>> jobClass, Object... args);
+    <R> CompletableFuture<R> execute(Set<ClusterNode> nodes, List<DeploymentUnit> units, String jobClassName, Object... args);

Review Comment:
   don't forget to update javadoc



-- 
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: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] Pochatkin commented on a diff in pull request #2121: IGNITE-19565 Use JobClassLoader for loading job classes

Posted by "Pochatkin (via GitHub)" <gi...@apache.org>.
Pochatkin commented on code in PR #2121:
URL: https://github.com/apache/ignite-3/pull/2121#discussion_r1214291372


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeTestStandalone.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.ignite.internal.compute;
+
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.compute.DeploymentUnit;
+import org.apache.ignite.compute.version.Version;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Integration tests for Compute functionality in standalone Ignite node.
+ */
+@SuppressWarnings("resource")
+class ItComputeTestStandalone extends ItComputeBaseTest {
+
+    private final URL jobsResource = ItComputeTestStandalone.class.getClassLoader().getResource("units/ignite-jobs-1.0-SNAPSHOT.jar");

Review Comment:
   Split by empty lines



##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeTestStandalone.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.ignite.internal.compute;
+
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.compute.DeploymentUnit;
+import org.apache.ignite.compute.version.Version;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Integration tests for Compute functionality in standalone Ignite node.
+ */
+@SuppressWarnings("resource")
+class ItComputeTestStandalone extends ItComputeBaseTest {
+

Review Comment:
   Remove empty line



##########
modules/code-deployment/src/main/java/org/apache/ignite/internal/deployunit/FileDeployerService.java:
##########
@@ -148,4 +148,17 @@ public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IO
             return new UnitContent(result);
         }, executor);
     }
+
+    /**
+     * Returns path to unit folder.
+     *
+     * @param id Deployment unit identifier.
+     * @param version Deployment unit version.
+     * @return Path to unit folder.
+     */
+    public Path unitPath(String id, Version version) {

Review Comment:
   Please use this method in this class also.



##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeTestStandalone.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.ignite.internal.compute;
+
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.compute.DeploymentUnit;
+import org.apache.ignite.compute.version.Version;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Integration tests for Compute functionality in standalone Ignite node.
+ */
+@SuppressWarnings("resource")
+class ItComputeTestStandalone extends ItComputeBaseTest {
+
+    private final URL jobsResource = ItComputeTestStandalone.class.getClassLoader().getResource("units/ignite-jobs-1.0-SNAPSHOT.jar");
+    private final String unitId = "jobs";
+    private final Version unitVersion = Version.parseVersion("1.0.0");
+    private final List<DeploymentUnit> units = List.of(new DeploymentUnit(unitId, unitVersion));
+
+    @BeforeEach
+    void setUp() throws IOException {
+
+        try (InputStream jarStream = jobsResource.openStream()) {
+            CompletableFuture<Boolean> deployAsync = node(0).deployment().deployAsync(
+                    unitId,
+                    unitVersion,
+                    () -> Map.of("ignite-jobs-1.0-SNAPSHOT.jar", jarStream)
+            );
+            assertThat(deployAsync, willCompleteSuccessfully());
+        }
+
+        cluster.runningNodes().forEach(node -> {
+            CompletableFuture<Boolean> deployAsync = node.deployment().onDemandDeploy(
+                    unitId,
+                    unitVersion
+            );
+            assertThat(deployAsync, willCompleteSuccessfully());
+        });
+    }
+
+    @Override
+    protected List<DeploymentUnit> units() {
+        return units;
+    }
+
+    @Override
+    protected String concatJobClassName() {
+        return "org.example.ConcatJob";
+    }
+
+    @Override
+    protected String getNodeNameJobClassName() {
+        return "org.example.GetNodeNameJob";
+    }
+
+    @Override
+    protected String failingJobClassName() {
+        return "org.example.FailingJob";
+    }
+
+    @Override
+    protected String jobExceptionClassName() {
+        return "org.example.JobException";
+    }
+
+    @Test
+    @Disabled("https://issues.apache.org/jira/browse/IGNITE-19623")
+    @Override
+    void executesFailingJobOnRemoteNodes() {
+        super.executesFailingJobOnRemoteNodes();
+    }
+

Review Comment:
   Remove empty line



##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeTestStandalone.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.ignite.internal.compute;
+
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.compute.DeploymentUnit;
+import org.apache.ignite.compute.version.Version;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Integration tests for Compute functionality in standalone Ignite node.
+ */
+@SuppressWarnings("resource")
+class ItComputeTestStandalone extends ItComputeBaseTest {
+
+    private final URL jobsResource = ItComputeTestStandalone.class.getClassLoader().getResource("units/ignite-jobs-1.0-SNAPSHOT.jar");
+    private final String unitId = "jobs";
+    private final Version unitVersion = Version.parseVersion("1.0.0");
+    private final List<DeploymentUnit> units = List.of(new DeploymentUnit(unitId, unitVersion));
+
+    @BeforeEach
+    void setUp() throws IOException {
+

Review Comment:
   Remove empty line



-- 
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: notifications-unsubscribe@ignite.apache.org

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