You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by jc...@apache.org on 2016/02/03 02:40:46 UTC

aurora git commit: Add header to allow bypassing the LeaderRedirectFilter.

Repository: aurora
Updated Branches:
  refs/heads/master d033d5393 -> 2d91e18fb


Add header to allow bypassing the LeaderRedirectFilter.

Bugs closed: AURORA-1601

Reviewed at https://reviews.apache.org/r/42964/


Project: http://git-wip-us.apache.org/repos/asf/aurora/repo
Commit: http://git-wip-us.apache.org/repos/asf/aurora/commit/2d91e18f
Tree: http://git-wip-us.apache.org/repos/asf/aurora/tree/2d91e18f
Diff: http://git-wip-us.apache.org/repos/asf/aurora/diff/2d91e18f

Branch: refs/heads/master
Commit: 2d91e18fb9ee90ee0e11e0914c6fa65f029b4626
Parents: d033d53
Author: Joshua Cohen <jc...@apache.org>
Authored: Tue Feb 2 19:40:26 2016 -0600
Committer: Joshua Cohen <jc...@apache.org>
Committed: Tue Feb 2 19:40:26 2016 -0600

----------------------------------------------------------------------
 .../thrift/org/apache/aurora/gen/api.thrift     |  3 +
 examples/vagrant/clusters_direct.json           |  7 ++
 .../scheduler/http/LeaderRedirectFilter.java    |  8 ++
 src/main/python/apache/aurora/admin/admin.py    | 10 ++-
 .../python/apache/aurora/admin/aurora_admin.py  |  7 ++
 .../python/apache/aurora/client/api/__init__.py |  7 +-
 .../aurora/client/api/scheduler_client.py       | 23 +++++-
 .../scheduler/http/ServletFilterTest.java       | 14 ++++
 .../aurora/client/api/test_scheduler_client.py  | 81 +++++++++++++++++++-
 .../test_bypass_leader_redirect_end_to_end.sh   | 71 +++++++++++++++++
 .../sh/org/apache/aurora/e2e/test_end_to_end.sh |  1 +
 11 files changed, 222 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/2d91e18f/api/src/main/thrift/org/apache/aurora/gen/api.thrift
----------------------------------------------------------------------
diff --git a/api/src/main/thrift/org/apache/aurora/gen/api.thrift b/api/src/main/thrift/org/apache/aurora/gen/api.thrift
index 95313a0..c870bcf 100644
--- a/api/src/main/thrift/org/apache/aurora/gen/api.thrift
+++ b/api/src/main/thrift/org/apache/aurora/gen/api.thrift
@@ -1175,3 +1175,6 @@ service AuroraAdmin extends AuroraSchedulerManager {
    */
   Response rewriteConfigs(1: RewriteConfigsRequest request)
 }
+
+// The name of the header that should be sent to bypass leader redirection in the Scheduler.
+const string BYPASS_LEADER_REDIRECT_HEADER_NAME = 'Bypass-Leader-Redirect'
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/aurora/blob/2d91e18f/examples/vagrant/clusters_direct.json
----------------------------------------------------------------------
diff --git a/examples/vagrant/clusters_direct.json b/examples/vagrant/clusters_direct.json
new file mode 100644
index 0000000..5416c37
--- /dev/null
+++ b/examples/vagrant/clusters_direct.json
@@ -0,0 +1,7 @@
+[{
+  "name": "devcluster",
+  "scheduler_uri": "http://localhost:8081",
+  "auth_mechanism": "UNAUTHENTICATED",
+  "slave_run_directory": "latest",
+  "slave_root": "/var/lib/mesos"
+}]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/aurora/blob/2d91e18f/src/main/java/org/apache/aurora/scheduler/http/LeaderRedirectFilter.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/http/LeaderRedirectFilter.java b/src/main/java/org/apache/aurora/scheduler/http/LeaderRedirectFilter.java
index 41b9984..8b8b305 100644
--- a/src/main/java/org/apache/aurora/scheduler/http/LeaderRedirectFilter.java
+++ b/src/main/java/org/apache/aurora/scheduler/http/LeaderRedirectFilter.java
@@ -30,6 +30,7 @@ import com.google.common.io.Resources;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.aurora.gen.apiConstants.BYPASS_LEADER_REDIRECT_HEADER_NAME;
 import static org.apache.aurora.scheduler.http.LeaderRedirect.LeaderStatus;
 
 /**
@@ -58,6 +59,13 @@ public class LeaderRedirectFilter extends AbstractFilter {
   public void doFilter(HttpServletRequest request, HttpServletResponse response, FilterChain chain)
       throws IOException, ServletException {
 
+    if (request.getHeader(BYPASS_LEADER_REDIRECT_HEADER_NAME) != null) {
+      LOG.info(BYPASS_LEADER_REDIRECT_HEADER_NAME + " header was present on the request, bypassing "
+          + "leader redirection.");
+      chain.doFilter(request, response);
+      return;
+    }
+
     LeaderStatus leaderStatus = redirector.getLeaderStatus();
     switch (leaderStatus) {
       case LEADING:

http://git-wip-us.apache.org/repos/asf/aurora/blob/2d91e18f/src/main/python/apache/aurora/admin/admin.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/admin/admin.py b/src/main/python/apache/aurora/admin/admin.py
index b68f546..6ecb5cb 100644
--- a/src/main/python/apache/aurora/admin/admin.py
+++ b/src/main/python/apache/aurora/admin/admin.py
@@ -67,8 +67,14 @@ def make_admin_client(cluster):
   if cluster not in CLUSTERS:
     die('Unknown cluster: %s. Known clusters: %s' % (cluster, ", ".join(CLUSTERS.keys())))
 
-  verbose = getattr(app.get_options(), 'verbosity', 'normal') == 'verbose'
-  return AuroraClientAPI(CLUSTERS[cluster], AURORA_ADMIN_USER_AGENT_NAME, verbose=verbose)
+  options = app.get_options()
+  verbose = getattr(options, 'verbosity', 'normal') == 'verbose'
+
+  return AuroraClientAPI(
+      CLUSTERS[cluster],
+      AURORA_ADMIN_USER_AGENT_NAME,
+      verbose=verbose,
+      bypass_leader_redirect=options.bypass_leader_redirect)
 
 
 @app.command

http://git-wip-us.apache.org/repos/asf/aurora/blob/2d91e18f/src/main/python/apache/aurora/admin/aurora_admin.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/admin/aurora_admin.py b/src/main/python/apache/aurora/admin/aurora_admin.py
index 470b2d2..fbebbab 100644
--- a/src/main/python/apache/aurora/admin/aurora_admin.py
+++ b/src/main/python/apache/aurora/admin/aurora_admin.py
@@ -40,6 +40,13 @@ LogOptions.disable_disk_logging()
 app.set_name('aurora-admin')
 app.set_usage(generate_terse_usage())
 
+app.add_option(
+    '--bypass-leader-redirect',
+    action='store_true',
+    default=False,
+    dest='bypass_leader_redirect',
+    help='Bypass the scheduler\'s leader redirect filter')
+
 
 def proxy_main():
   app.main()

http://git-wip-us.apache.org/repos/asf/aurora/blob/2d91e18f/src/main/python/apache/aurora/client/api/__init__.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/api/__init__.py b/src/main/python/apache/aurora/client/api/__init__.py
index 1b2ce4d..63bd649 100644
--- a/src/main/python/apache/aurora/client/api/__init__.py
+++ b/src/main/python/apache/aurora/client/api/__init__.py
@@ -48,15 +48,18 @@ class AuroraClientAPI(object):
       self,
       cluster,
       user_agent,
-      verbose=False):
+      verbose=False,
+      bypass_leader_redirect=False):
 
     if not isinstance(cluster, Cluster):
       raise TypeError('AuroraClientAPI expects instance of Cluster for "cluster", got %s' %
           type(cluster))
+
     self._scheduler_proxy = SchedulerProxy(
         cluster,
         verbose=verbose,
-        user_agent=user_agent)
+        user_agent=user_agent,
+        bypass_leader_redirect=bypass_leader_redirect)
     self._cluster = cluster
 
   @property

http://git-wip-us.apache.org/repos/asf/aurora/blob/2d91e18f/src/main/python/apache/aurora/client/api/scheduler_client.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/api/scheduler_client.py b/src/main/python/apache/aurora/client/api/scheduler_client.py
index 8636624..cbdb50a 100644
--- a/src/main/python/apache/aurora/client/api/scheduler_client.py
+++ b/src/main/python/apache/aurora/client/api/scheduler_client.py
@@ -17,6 +17,7 @@ import threading
 import time
 import traceback
 
+import requests
 from pystachio import Default, Integer, String
 from thrift.protocol import TJSONProtocol
 from thrift.transport import TTransport
@@ -31,6 +32,7 @@ from apache.aurora.common.cluster import Cluster
 from apache.aurora.common.transport import TRequestsTransport
 
 from gen.apache.aurora.api import AuroraAdmin
+from gen.apache.aurora.api.constants import BYPASS_LEADER_REDIRECT_HEADER_NAME
 from gen.apache.aurora.api.ttypes import ResponseCode
 
 try:
@@ -48,6 +50,15 @@ class SchedulerClientTrait(Cluster.Trait):
   auth_mechanism    = Default(String, 'UNAUTHENTICATED')  # noqa
 
 
+def _bypass_leader_redirect_session_factory(should_bypass=False):
+  session = requests.session()
+
+  if should_bypass:
+    session.headers[BYPASS_LEADER_REDIRECT_HEADER_NAME] = 'true'
+
+  return session
+
+
 class SchedulerClient(object):
   THRIFT_RETRIES = 5
   RETRY_TIMEOUT = Amount(1, Time.SECONDS)
@@ -71,11 +82,12 @@ class SchedulerClient(object):
     else:
       raise ValueError('"cluster" does not specify zk or scheduler_uri')
 
-  def __init__(self, auth, user_agent, verbose=False):
+  def __init__(self, auth, user_agent, verbose=False, bypass_leader_redirect=False):
     self._client = None
     self._auth_handler = auth
     self._user_agent = user_agent
     self._verbose = verbose
+    self._bypass_leader_redirect = bypass_leader_redirect
 
   def get_thrift_client(self):
     if self._client is None:
@@ -91,7 +103,14 @@ class SchedulerClient(object):
     return None
 
   def _connect_scheduler(self, uri, clock=time):
-    transport = TRequestsTransport(uri, auth=self._auth_handler.auth(), user_agent=self._user_agent)
+    transport = TRequestsTransport(
+        uri,
+        auth=self._auth_handler.auth(),
+        user_agent=self._user_agent,
+        session_factory=functools.partial(
+            _bypass_leader_redirect_session_factory,
+            should_bypass=self._bypass_leader_redirect))
+
     protocol = TJSONProtocol.TJSONProtocol(transport)
     schedulerClient = AuroraAdmin.Client(protocol)
     for _ in range(self.THRIFT_RETRIES):

http://git-wip-us.apache.org/repos/asf/aurora/blob/2d91e18f/src/test/java/org/apache/aurora/scheduler/http/ServletFilterTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/http/ServletFilterTest.java b/src/test/java/org/apache/aurora/scheduler/http/ServletFilterTest.java
index 21371d9..1a5cd01 100644
--- a/src/test/java/org/apache/aurora/scheduler/http/ServletFilterTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/http/ServletFilterTest.java
@@ -27,6 +27,7 @@ import com.sun.jersey.api.client.ClientResponse.Status;
 
 import org.junit.Test;
 
+import static org.apache.aurora.gen.apiConstants.BYPASS_LEADER_REDIRECT_HEADER_NAME;
 import static org.junit.Assert.assertEquals;
 
 public class ServletFilterTest extends AbstractJettyTest {
@@ -123,4 +124,17 @@ public class ServletFilterTest extends AbstractJettyTest {
     leaderRedirectSmokeTest(Status.TEMPORARY_REDIRECT, Optional.absent());
     assertResponseStatus("/", Status.OK, Optional.absent());
   }
+
+  @Test
+  public void testHeaderOverridesLeaderRedirect() throws Exception {
+    replayAndStart();
+
+    unsetLeadingSchduler();
+
+    ClientResponse response = getRequestBuilder("/scheduler")
+        .header(BYPASS_LEADER_REDIRECT_HEADER_NAME, "true")
+        .get(ClientResponse.class);
+
+    assertEquals(Status.OK.getStatusCode(), response.getStatus());
+  }
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/2d91e18f/src/test/python/apache/aurora/client/api/test_scheduler_client.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/api/test_scheduler_client.py b/src/test/python/apache/aurora/client/api/test_scheduler_client.py
index 4c4caaf..bed800d 100644
--- a/src/test/python/apache/aurora/client/api/test_scheduler_client.py
+++ b/src/test/python/apache/aurora/client/api/test_scheduler_client.py
@@ -32,6 +32,7 @@ from apache.aurora.common.transport import TRequestsTransport
 
 import gen.apache.aurora.api.AuroraAdmin as AuroraAdmin
 import gen.apache.aurora.api.AuroraSchedulerManager as AuroraSchedulerManager
+from gen.apache.aurora.api.constants import BYPASS_LEADER_REDIRECT_HEADER_NAME
 from gen.apache.aurora.api.ttypes import (
     Hosts,
     JobConfiguration,
@@ -394,7 +395,67 @@ class TestSchedulerClient(unittest.TestCase):
     uri = 'https://scheduler.example.com:1337'
     client._connect_scheduler(uri, mock_time)
 
-    mock_transport.assert_called_once_with(uri, auth=auth.auth(), user_agent=user_agent)
+    mock_transport.assert_called_once_with(
+        uri,
+        auth=auth.auth(),
+        user_agent=user_agent,
+        session_factory=mock.ANY)
+
+  @mock.patch('apache.aurora.client.api.scheduler_client.TRequestsTransport',
+              spec=TRequestsTransport)
+  def test_connect_scheduler_without_bypass_leader_redirect(self, mock_transport):
+    mock_transport.return_value.open.side_effect = [TTransport.TTransportException, True]
+    mock_time = mock.create_autospec(spec=time, instance=True)
+
+    auth = mock_auth()
+    user_agent = 'Some-User-Agent'
+
+    client = scheduler_client.SchedulerClient(
+        auth,
+        user_agent,
+        verbose=True,
+        bypass_leader_redirect=False)
+
+    uri = 'https://scheduler.example.com:1337'
+    client._connect_scheduler(uri, mock_time)
+
+    mock_transport.assert_called_once_with(
+        uri,
+        auth=auth.auth(),
+        user_agent=user_agent,
+        session_factory=mock.ANY)
+
+    _, _, kwargs = mock_transport.mock_calls[0]
+    session = kwargs['session_factory']()
+    assert session.headers.get(BYPASS_LEADER_REDIRECT_HEADER_NAME) is None
+
+  @mock.patch('apache.aurora.client.api.scheduler_client.TRequestsTransport',
+              spec=TRequestsTransport)
+  def test_connect_scheduler_with_bypass_leader_redirect(self, mock_transport):
+    mock_transport.return_value.open.side_effect = [TTransport.TTransportException, True]
+    mock_time = mock.create_autospec(spec=time, instance=True)
+
+    auth = mock_auth()
+    user_agent = 'Some-User-Agent'
+
+    client = scheduler_client.SchedulerClient(
+        auth,
+        user_agent,
+        verbose=True,
+        bypass_leader_redirect=True)
+
+    uri = 'https://scheduler.example.com:1337'
+    client._connect_scheduler(uri, mock_time)
+
+    mock_transport.assert_called_once_with(
+        uri,
+        auth=auth.auth(),
+        user_agent=user_agent,
+        session_factory=mock.ANY)
+
+    _, _, kwargs = mock_transport.mock_calls[0]
+    session = kwargs['session_factory']()
+    assert session.headers[BYPASS_LEADER_REDIRECT_HEADER_NAME] == 'true'
 
   @mock.patch('apache.aurora.client.api.scheduler_client.SchedulerClient',
               spec=scheduler_client.SchedulerClient)
@@ -454,7 +515,11 @@ class TestSchedulerClient(unittest.TestCase):
 
     client._connect_scheduler(uri, mock_time)
 
-    mock_transport.assert_called_once_with(uri, auth=auth.auth(), user_agent=user_agent)
+    mock_transport.assert_called_once_with(
+        uri,
+        auth=auth.auth(),
+        user_agent=user_agent,
+        session_factory=mock.ANY)
 
   @mock.patch('apache.aurora.client.api.scheduler_client.TRequestsTransport',
               spec=TRequestsTransport)
@@ -477,7 +542,11 @@ class TestSchedulerClient(unittest.TestCase):
 
     client._connect_scheduler(uri, mock_time)
 
-    mock_transport.assert_called_once_with(uri, auth=auth.auth(), user_agent=user_agent)
+    mock_transport.assert_called_once_with(
+        uri,
+        auth=auth.auth(),
+        user_agent=user_agent,
+        session_factory=mock.ANY)
 
   @mock.patch('apache.aurora.client.api.scheduler_client.TRequestsTransport',
               spec=TRequestsTransport)
@@ -500,7 +569,11 @@ class TestSchedulerClient(unittest.TestCase):
 
     client._connect_scheduler(uri, mock_time)
 
-    mock_transport.assert_called_once_with(uri, auth=auth.auth(), user_agent=user_agent)
+    mock_transport.assert_called_once_with(
+        uri,
+        auth=auth.auth(),
+        user_agent=user_agent,
+        session_factory=mock.ANY)
 
   def test_no_zk_or_scheduler_uri(self):
     cluster = None

http://git-wip-us.apache.org/repos/asf/aurora/blob/2d91e18f/src/test/sh/org/apache/aurora/e2e/test_bypass_leader_redirect_end_to_end.sh
----------------------------------------------------------------------
diff --git a/src/test/sh/org/apache/aurora/e2e/test_bypass_leader_redirect_end_to_end.sh b/src/test/sh/org/apache/aurora/e2e/test_bypass_leader_redirect_end_to_end.sh
new file mode 100755
index 0000000..5c0f12b
--- /dev/null
+++ b/src/test/sh/org/apache/aurora/e2e/test_bypass_leader_redirect_end_to_end.sh
@@ -0,0 +1,71 @@
+#!/bin/bash
+#
+# Licensed 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.
+#
+#
+# An integration test for the bypassing the leader redirect filter, using the vagrant environment as
+# a testbed.
+set -eux
+
+function enter_vagrant {
+  exec vagrant ssh -- \
+    /vagrant/src/test/sh/org/apache/aurora/e2e/test_bypass_leader_redirect_end_to_end.sh "$@"
+}
+
+function await_scheduler_ready {
+  while ! curl -s localhost:8081/vars | grep "scheduler_lifecycle_LEADER_AWAITING_REGISTRATION 1"; do
+    sleep 3
+  done
+}
+
+function setup {
+  aurorabuild all
+  sudo cp /vagrant/examples/vagrant/clusters_direct.json /etc/aurora/clusters.json
+  sudo stop mesos-master || true
+  sudo restart aurora-scheduler
+  await_scheduler_ready
+}
+
+function test_bypass_leader_redirect {
+  aurora_admin query --bypass-leader-redirect devcluster vagrant http_example
+}
+
+function tear_down {
+  local retcode=$1
+  sudo cp /vagrant/examples/vagrant/clusters.json /etc/aurora/clusters.json
+  sudo start mesos-master || true
+  if [[ $retcode -ne 0 ]]; then
+    echo
+    echo '!!! FAILED'
+    echo
+  fi
+  exit $retcode
+}
+
+function main {
+  if [[ "$USER" != "vagrant" ]]; then
+    enter_vagrant "$@"
+  else
+    trap 'tear_down 1' EXIT
+    setup
+    test_bypass_leader_redirect
+    set +x
+    echo
+    echo '*** OK (All tests passed) ***'
+    echo
+    trap '' EXIT
+    tear_down 0
+  fi
+}
+
+main "$@"

http://git-wip-us.apache.org/repos/asf/aurora/blob/2d91e18f/src/test/sh/org/apache/aurora/e2e/test_end_to_end.sh
----------------------------------------------------------------------
diff --git a/src/test/sh/org/apache/aurora/e2e/test_end_to_end.sh b/src/test/sh/org/apache/aurora/e2e/test_end_to_end.sh
index 1171dca..75130a3 100755
--- a/src/test/sh/org/apache/aurora/e2e/test_end_to_end.sh
+++ b/src/test/sh/org/apache/aurora/e2e/test_end_to_end.sh
@@ -357,4 +357,5 @@ test_admin "${TEST_ADMIN_ARGS[@]}"
 test_basic_auth_unauthenticated  "${TEST_JOB_ARGS[@]}"
 
 /vagrant/src/test/sh/org/apache/aurora/e2e/test_kerberos_end_to_end.sh
+/vagrant/src/test/sh/org/apache/aurora/e2e/test_bypass_leader_redirect_end_to_end.sh
 RETCODE=0