You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@mesos.apache.org by GitBox <gi...@apache.org> on 2021/01/30 20:04:41 UTC

[GitHub] [mesos] cf-natali opened a new pull request #380: Fixed a bug preventing agent recovery when executor GC is interrupted.

cf-natali opened a new pull request #380:
URL: https://github.com/apache/mesos/pull/380


   If the agent is interrupted after garbage collecting the executor's
   latest run meta directory but before garbage collecting the top-level
   executor meta directory, the "latest" symlink will dangle, which would
   cause the agent executor recovery to fail.
   Instead, we can simply ignore if the "latest" symlink dangles, since
   it's always created after the latest run directory it points to, and
   never deleted until the top-level executor meta directory is garbage
   collected.
   
   
   Example logs showing the problem:
   
   Agent GC'ing the directory:
   ```
   I0129 22:38:45.060012 28292 slave.cpp:7107] Executor 'task-72954d99-5719-414f-b7d9-5f35c5d70055' of framework 1f2209bb-43e4-4b8b-b36c-3fa1e855a0f1-0002 exited with status 0
   I0129 22:38:45.060871 28292 slave.cpp:7218] Cleaning up executor 'task-72954d99-5719-414f-b7d9-5f35c5d70055' of framework 1f2209bb-43e4-4b8b-b36c-3fa1e855a0f1-0002 at executor(1)@127.0.1.1:40075
   [...]
   I0129 22:38:45.061872 29250 gc.cpp:95] Scheduling '/tmp/tmp2y330b17mesos_agent_work_dir/meta/slaves/1f2209bb-43e4-4b8b-b36c-3fa1e855a0f1-S5/frameworks/1f2209bb-43e4-4b8b-b36c-3fa1e855a0f1-0002/executors/task-72954d99-5719-414f-b7d9-5f35c5d70055/runs/fa5986f6-777b-42fb-88b4-e4ce339c21ab' for gc 4.938180864secs in the future
   I0129 22:38:45.061939 29250 gc.cpp:95] Scheduling '/tmp/tmp2y330b17mesos_agent_work_dir/slaves/1f2209bb-43e4-4b8b-b36c-3fa1e855a0f1-S5/frameworks/1f2209bb-43e4-4b8b-b36c-3fa1e855a0f1-0002/executors/task-72954d99-5719-414f-b7d9-5f35c5d70055' for gc 4.93812992secs in the future
   [...]
   I0129 22:38:50.019327 29251 gc.cpp:272] Deleting /tmp/tmp2y330b17mesos_agent_work_dir/meta/slaves/1f2209bb-43e4-4b8b-b36c-3fa1e855a0f1-S5/frameworks/1f2209bb-43e4-4b8b-b36c-3fa1e855a0f1-0002/executors/task-72954d99-5719-414f-b7d9-5f35c5d70055/runs/fa5986f6-777b-42fb-88b4-e4ce339c21ab
   I0129 22:38:50.019573 29251 gc.cpp:288] Deleted '/tmp/tmp2y330b17mesos_agent_work_dir/meta/slaves/1f2209bb-43e4-4b8b-b36c-3fa1e855a0f1-S5/frameworks/1f2209bb-43e4-4b8b-b36c-3fa1e855a0f1-0002/executors/task-72954d99-5719-414f-b7d9-5f35c5d70055/runs/fa5986f6-777b-42fb-88b4-e4ce339c21ab'
   ```
   
   The agent got killed, and didn't get to GC `/tmp/tmp2y330b17mesos_agent_work_dir/slaves/1f2209bb-43e4-4b8b-b36c-3fa1e855a0f1-S5/frameworks/1f2209bb-43e4-4b8b-b36c-3fa1e855a0f1-0002/executors/task-72954d99-5719-414f-b7d9-5f35c5d70055`.
   
   Then the agent restarted:
   ```
   [...]
   E0129 22:38:54.942884 29402 slave.cpp:8355] EXIT with status 1: Failed to perform recovery: Failed to recover framework 1f2209bb-43e4-4b8b-b36c-3fa1e855a0f1-0002: Failed to recover executor 'task-72954d99-5719-414f-b7d9-5f35c5d70055': Failed to find latest run of executor 'task-72954d99-5719-414f-b7d9-5f35c5d70055': No such file or directory
   ```
   
   We can see that `latest` for executor `task-72954d99-5719-414f-b7d9-5f35c5d70055` points to run `fa5986f6-777b-42fb-88b4-e4ce339c21ab` which has already been GCed.
   ```
   cf@thinkpad:~/src/mesos$ ls -l /tmp/tmp2y330b17mesos_agent_work_dir/meta/slaves/1f2209bb-43e4-4b8b-b36c-3fa1e855a0f1-S5/frameworks/1f2209bb-43e4-4b8b-b36c-3fa1e855a0f1-0002/executors/task-72954d99-5719-414f-b7d9-5f35c5d70055/runs/latest 
   lrwxrwxrwx 1 cf cf 235 janv. 29 22:28 /tmp/tmp2y330b17mesos_agent_work_dir/meta/slaves/1f2209bb-43e4-4b8b-b36c-3fa1e855a0f1-S5/frameworks/1f2209bb-43e4-4b8b-b36c-3fa1e855a0f1-0002/executors/task-72954d99-5719-414f-b7d9-5f35c5d70055/runs/latest -> /tmp/tmp2y330b17mesos_agent_work_dir/meta/slaves/1f2209bb-43e4-4b8b-b36c-3fa1e855a0f1-S5/frameworks/1f2209bb-43e4-4b8b-b36c-3fa1e855a0f1-0002/executors/task-72954d99-5719-414f-b7d9-5f35c5d70055/runs/fa5986f6-777b-42fb-88b4-e4ce339c21ab
   cf@thinkpad:~/src/mesos$ ls -l /tmp/tmp2y330b17mesos_agent_work_dir/meta/slaves/1f2209bb-43e4-4b8b-b36c-3fa1e855a0f1-S5/frameworks/1f2209bb-43e4-4b8b-b36c-3fa1e855a0f1-0002/executors/task-72954d99-5719-414f-b7d9-5f35c5d70055/runs/
   total 4
   lrwxrwxrwx 1 cf cf 235 janv. 29 22:28 latest -> /tmp/tmp2y330b17mesos_agent_work_dir/meta/slaves/1f2209bb-43e4-4b8b-b36c-3fa1e855a0f1-S5/frameworks/1f2209bb-43e4-4b8b-b36c-3fa1e855a0f1-0002/executors/task-72954d99-5719-414f-b7d9-5f35c5d70055/runs/fa5986f6-777b-42fb-88b4-e4ce339c21ab
   cf@thinkpad:~/src/mesos$ 
   ```
   
   @bbannier  


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

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



[GitHub] [mesos] asekretenko commented on a change in pull request #380: Fixed a bug preventing agent recovery when executor GC is interrupted.

Posted by GitBox <gi...@apache.org>.
asekretenko commented on a change in pull request #380:
URL: https://github.com/apache/mesos/pull/380#discussion_r611048071



##########
File path: src/tests/slave_recovery_tests.cpp
##########
@@ -3301,6 +3301,130 @@ TYPED_TEST(SlaveRecoveryTest, GCExecutor)
 }
 
 
+// When the slave is down we remove the latest run directory
+// but not the "latest" symlink, to simulate a situation where the
+// slave died in the middle of gc'ing the run meta directory.
+TYPED_TEST(SlaveRecoveryTest, ExecutorDanglingLatestSymlink)
+{
+  Try<Owned<cluster::Master>> master = this->StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = this->CreateSlaveFlags();
+  flags.strict = true;
+
+  Fetcher fetcher(flags);
+
+  Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
+  ASSERT_SOME(_containerizer);
+  Owned<slave::Containerizer> containerizer(_containerizer.get());
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  Try<Owned<cluster::Slave>> slave =
+    this->StartSlave(detector.get(), containerizer.get(), flags);
+  ASSERT_SOME(slave);
+
+  // Enable checkpointing for the framework.
+  FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
+  frameworkInfo.set_checkpoint(true);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, frameworkInfo, master.get()->pid, DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(_, _, _));
+
+  Future<vector<Offer>> offers1;
+  EXPECT_CALL(sched, resourceOffers(_, _))
+    .WillOnce(FutureArg<1>(&offers1));

Review comment:
       ```
   .WillRepeatedly(Return())
   ```
   is now also necessary.
   
   Otherwise, the test will crach if the scheduler receives the offer upon agent re-registartion before the test ends.
   
   I don't know if you have seen this failure; on my system this happens once in ~30 runs, found by running  ```src/mesos-tests --gtest_filter="*ExecutorDanglingLatestSymlink*" --gtest_repeat=1000 --gtest_break_on_failure```.




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

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



[GitHub] [mesos] cf-natali commented on pull request #380: Fixed a bug preventing agent recovery when executor GC is interrupted.

Posted by GitBox <gi...@apache.org>.
cf-natali commented on pull request #380:
URL: https://github.com/apache/mesos/pull/380#issuecomment-773595212


   @bmahler 


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

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



[GitHub] [mesos] cf-natali commented on pull request #380: Fixed a bug preventing agent recovery when executor GC is interrupted.

Posted by GitBox <gi...@apache.org>.
cf-natali commented on pull request #380:
URL: https://github.com/apache/mesos/pull/380#issuecomment-773595212


   @bmahler 


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

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



[GitHub] [mesos] asekretenko commented on a change in pull request #380: Fixed a bug preventing agent recovery when executor GC is interrupted.

Posted by GitBox <gi...@apache.org>.
asekretenko commented on a change in pull request #380:
URL: https://github.com/apache/mesos/pull/380#discussion_r585615415



##########
File path: src/tests/slave_recovery_tests.cpp
##########
@@ -3301,6 +3301,139 @@ TYPED_TEST(SlaveRecoveryTest, GCExecutor)
 }
 
 
+// When the slave is down we remove the latest run directory
+// but not the "latest" symlink, to simulate a situation where the
+// slave died in the middle of gc'ing the run meta directory.
+TYPED_TEST(SlaveRecoveryTest, ExecutorDanglingLatestSymlink)
+{
+  Try<Owned<cluster::Master>> master = this->StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = this->CreateSlaveFlags();
+  flags.strict = true;
+
+  Fetcher fetcher(flags);
+
+  Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
+  ASSERT_SOME(_containerizer);
+  Owned<slave::Containerizer> containerizer(_containerizer.get());
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  Try<Owned<cluster::Slave>> slave =
+    this->StartSlave(detector.get(), containerizer.get(), flags);
+  ASSERT_SOME(slave);
+
+  // Enable checkpointing for the framework.
+  FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
+  frameworkInfo.set_checkpoint(true);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, frameworkInfo, master.get()->pid, DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(_, _, _));
+
+  Future<vector<Offer>> offers1;
+  EXPECT_CALL(sched, resourceOffers(_, _))
+    .WillOnce(FutureArg<1>(&offers1));
+
+  driver.start();
+
+  AWAIT_READY(offers1);
+  ASSERT_FALSE(offers1->empty());
+
+  TaskInfo task = createTask(offers1.get()[0], SLEEP_COMMAND(1000));
+
+  // Capture the slave and framework ids.
+  SlaveID slaveId = offers1.get()[0].slave_id();
+  FrameworkID frameworkId = offers1.get()[0].framework_id();
+
+  Future<RegisterExecutorMessage> registerExecutor =
+    FUTURE_PROTOBUF(RegisterExecutorMessage(), _, _);
+
+  Future<Nothing> status;
+  EXPECT_CALL(sched, statusUpdate(_, _))
+    .WillOnce(FutureSatisfy(&status))
+    .WillRepeatedly(Return()); // Ignore subsequent updates.
+
+  driver.launchTasks(offers1.get()[0].id(), {task});
+
+  // Capture the executor id.
+  AWAIT_READY(registerExecutor);
+  ExecutorID executorId = registerExecutor->executor_id();
+
+  // Wait for TASK_RUNNING update.
+  AWAIT_READY(status);
+
+  // Terminate the slave.
+  slave.get()->terminate();
+
+  // The "latest" symlink should exist.
+  const string latestPath = paths::getExecutorLatestRunPath(
+      paths::getMetaRootDir(flags.work_dir),
+      slaveId,
+      frameworkId,
+      executorId);
+  ASSERT_TRUE(os::exists(latestPath));
+  // And should point to the latest run.
+  const Result<string> path = os::realpath(latestPath);
+  ASSERT_SOME(path);
+  // Delete it - "latest" will now dangle.
+  ASSERT_SOME(os::rmdir(path.get(), true));
+
+  // Recover the state.
+  Result<slave::state::State> recoverState =
+    slave::state::recover(paths::getMetaRootDir(flags.work_dir), true);
+
+  ASSERT_SOME(recoverState);
+  ASSERT_SOME(recoverState->slave);
+
+  // The executor should be recovered without any run.
+  slave::state::FrameworkState frameworkState =
+    recoverState->slave->frameworks.at(frameworkId);
+  ASSERT_EQ(1u, frameworkState.executors.size());
+  slave::state::ExecutorState& executorState =
+    frameworkState.executors.at(executorId);
+  ASSERT_NONE(executorState.latest);
+  ASSERT_TRUE(executorState.runs.empty());
+
+  Future<ReregisterSlaveMessage> reregisterSlaveMessage =
+    FUTURE_PROTOBUF(ReregisterSlaveMessage(), _, _);
+
+  Future<vector<Offer>> offers2;
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers2))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  slave = this->StartSlave(detector.get(), containerizer.get(), flags);
+  ASSERT_SOME(slave);
+
+  AWAIT_READY(reregisterSlaveMessage);
+
+  // Make sure all slave resources are reoffered.
+  AWAIT_READY(offers2);
+  EXPECT_EQ(Resources(offers1.get()[0].resources()),
+            Resources(offers2.get()[0].resources()));

Review comment:
       Shouldn't detecting agent re-registration via `ReregisterSlaveMessage` be sufficient for the purpose of this test? If this check is necessary, a comment explaining why would be beneficial.




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

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



[GitHub] [mesos] cf-natali commented on pull request #380: Fixed a bug preventing agent recovery when executor GC is interrupted.

Posted by GitBox <gi...@apache.org>.
cf-natali commented on pull request #380:
URL: https://github.com/apache/mesos/pull/380#issuecomment-781592423


   @qianzhangxa @asekretenko 


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

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



[GitHub] [mesos] cf-natali commented on a change in pull request #380: Fixed a bug preventing agent recovery when executor GC is interrupted.

Posted by GitBox <gi...@apache.org>.
cf-natali commented on a change in pull request #380:
URL: https://github.com/apache/mesos/pull/380#discussion_r611069471



##########
File path: src/tests/slave_recovery_tests.cpp
##########
@@ -3301,6 +3301,130 @@ TYPED_TEST(SlaveRecoveryTest, GCExecutor)
 }
 
 
+// When the slave is down we remove the latest run directory
+// but not the "latest" symlink, to simulate a situation where the
+// slave died in the middle of gc'ing the run meta directory.
+TYPED_TEST(SlaveRecoveryTest, ExecutorDanglingLatestSymlink)
+{
+  Try<Owned<cluster::Master>> master = this->StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = this->CreateSlaveFlags();
+  flags.strict = true;
+
+  Fetcher fetcher(flags);
+
+  Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
+  ASSERT_SOME(_containerizer);
+  Owned<slave::Containerizer> containerizer(_containerizer.get());
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  Try<Owned<cluster::Slave>> slave =
+    this->StartSlave(detector.get(), containerizer.get(), flags);
+  ASSERT_SOME(slave);
+
+  // Enable checkpointing for the framework.
+  FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
+  frameworkInfo.set_checkpoint(true);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, frameworkInfo, master.get()->pid, DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(_, _, _));
+
+  Future<vector<Offer>> offers1;
+  EXPECT_CALL(sched, resourceOffers(_, _))
+    .WillOnce(FutureArg<1>(&offers1));

Review comment:
       Thanks - I ran the test repeatedly before but not after the update.
   Should be fixed now.




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

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



[GitHub] [mesos] cf-natali commented on a change in pull request #380: Fixed a bug preventing agent recovery when executor GC is interrupted.

Posted by GitBox <gi...@apache.org>.
cf-natali commented on a change in pull request #380:
URL: https://github.com/apache/mesos/pull/380#discussion_r585966235



##########
File path: src/tests/slave_recovery_tests.cpp
##########
@@ -3301,6 +3301,139 @@ TYPED_TEST(SlaveRecoveryTest, GCExecutor)
 }
 
 
+// When the slave is down we remove the latest run directory
+// but not the "latest" symlink, to simulate a situation where the
+// slave died in the middle of gc'ing the run meta directory.
+TYPED_TEST(SlaveRecoveryTest, ExecutorDanglingLatestSymlink)
+{
+  Try<Owned<cluster::Master>> master = this->StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = this->CreateSlaveFlags();
+  flags.strict = true;
+
+  Fetcher fetcher(flags);
+
+  Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
+  ASSERT_SOME(_containerizer);
+  Owned<slave::Containerizer> containerizer(_containerizer.get());
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  Try<Owned<cluster::Slave>> slave =
+    this->StartSlave(detector.get(), containerizer.get(), flags);
+  ASSERT_SOME(slave);
+
+  // Enable checkpointing for the framework.
+  FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
+  frameworkInfo.set_checkpoint(true);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, frameworkInfo, master.get()->pid, DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(_, _, _));
+
+  Future<vector<Offer>> offers1;
+  EXPECT_CALL(sched, resourceOffers(_, _))
+    .WillOnce(FutureArg<1>(&offers1));
+
+  driver.start();
+
+  AWAIT_READY(offers1);
+  ASSERT_FALSE(offers1->empty());
+
+  TaskInfo task = createTask(offers1.get()[0], SLEEP_COMMAND(1000));
+
+  // Capture the slave and framework ids.
+  SlaveID slaveId = offers1.get()[0].slave_id();
+  FrameworkID frameworkId = offers1.get()[0].framework_id();
+
+  Future<RegisterExecutorMessage> registerExecutor =
+    FUTURE_PROTOBUF(RegisterExecutorMessage(), _, _);
+
+  Future<Nothing> status;
+  EXPECT_CALL(sched, statusUpdate(_, _))
+    .WillOnce(FutureSatisfy(&status))
+    .WillRepeatedly(Return()); // Ignore subsequent updates.
+
+  driver.launchTasks(offers1.get()[0].id(), {task});
+
+  // Capture the executor id.
+  AWAIT_READY(registerExecutor);
+  ExecutorID executorId = registerExecutor->executor_id();
+
+  // Wait for TASK_RUNNING update.
+  AWAIT_READY(status);
+
+  // Terminate the slave.
+  slave.get()->terminate();
+
+  // The "latest" symlink should exist.
+  const string latestPath = paths::getExecutorLatestRunPath(
+      paths::getMetaRootDir(flags.work_dir),
+      slaveId,
+      frameworkId,
+      executorId);
+  ASSERT_TRUE(os::exists(latestPath));
+  // And should point to the latest run.
+  const Result<string> path = os::realpath(latestPath);
+  ASSERT_SOME(path);
+  // Delete it - "latest" will now dangle.
+  ASSERT_SOME(os::rmdir(path.get(), true));
+
+  // Recover the state.
+  Result<slave::state::State> recoverState =
+    slave::state::recover(paths::getMetaRootDir(flags.work_dir), true);
+
+  ASSERT_SOME(recoverState);
+  ASSERT_SOME(recoverState->slave);
+
+  // The executor should be recovered without any run.
+  slave::state::FrameworkState frameworkState =
+    recoverState->slave->frameworks.at(frameworkId);
+  ASSERT_EQ(1u, frameworkState.executors.size());
+  slave::state::ExecutorState& executorState =
+    frameworkState.executors.at(executorId);
+  ASSERT_NONE(executorState.latest);
+  ASSERT_TRUE(executorState.runs.empty());
+
+  Future<ReregisterSlaveMessage> reregisterSlaveMessage =
+    FUTURE_PROTOBUF(ReregisterSlaveMessage(), _, _);
+
+  Future<vector<Offer>> offers2;
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers2))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  slave = this->StartSlave(detector.get(), containerizer.get(), flags);
+  ASSERT_SOME(slave);
+
+  AWAIT_READY(reregisterSlaveMessage);
+
+  // Make sure all slave resources are reoffered.
+  AWAIT_READY(offers2);
+  EXPECT_EQ(Resources(offers1.get()[0].resources()),
+            Resources(offers2.get()[0].resources()));

Review comment:
       Yes it should be enough - I've removed this.
   
   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.

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



[GitHub] [mesos] asekretenko merged pull request #380: Fixed a bug preventing agent recovery when executor GC is interrupted.

Posted by GitBox <gi...@apache.org>.
asekretenko merged pull request #380:
URL: https://github.com/apache/mesos/pull/380


   


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

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