You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@knox.apache.org by GitBox <gi...@apache.org> on 2020/02/26 23:54:54 UTC

[GitHub] [knox] RogPodge opened a new pull request #275: KNOX-2253 Add support to handle Livy server redirects

RogPodge opened a new pull request #275: KNOX-2253 Add support to handle Livy server redirects
URL: https://github.com/apache/knox/pull/275
 
 
   (It is very **important** that you created an Apache Knox JIRA for this change and that the PR title/commit message includes the Apache Knox JIRA ID!)
   
   ## What changes were proposed in this pull request?
   
   Currently knox gateway does not seemlessly handle Livy redirect responses like it does with Yarn resource manager. We need a patch to knox gateway to handle these redirect responses on server side, and makes some adjustments to the service definition to address issues like KNOX-1609
   
   ## How was this patch tested?
   
   This patch was tested on a local deployment of Knox gateway against a Livy Instance that was using the following HA patch: https://github.com/RogPodge/incubator-livy/tree/LivyHARedirect
   

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


With regards,
Apache Git Services

[GitHub] [knox] risdenk commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects
URL: https://github.com/apache/knox/pull/275#discussion_r385179553
 
 

 ##########
 File path: gateway-service-livy/pom.xml
 ##########
 @@ -58,5 +65,10 @@
             <artifactId>gateway-test-utils</artifactId>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
 
 Review comment:
   This shouldn't be necessary - Should be handled automatically for all submodules.

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


With regards,
Apache Git Services

[GitHub] [knox] RogPodge commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects

Posted by GitBox <gi...@apache.org>.
RogPodge commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects
URL: https://github.com/apache/knox/pull/275#discussion_r394717991
 
 

 ##########
 File path: gateway-service-livy/src/main/java/org/apache/knox/gateway/livy/LivyHaDispatch.java
 ##########
 @@ -0,0 +1,156 @@
+/*
+ * 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.knox.gateway.livy;
+
+import org.apache.http.HttpResponse;
+import org.apache.http.client.methods.HttpRequestBase;
+import org.apache.http.client.methods.HttpUriRequest;
+import org.apache.http.entity.BufferedHttpEntity;
+import org.apache.knox.gateway.ha.dispatch.DefaultHaDispatch;
+import org.apache.knox.gateway.filter.AbstractGatewayFilter;
+import org.apache.knox.gateway.i18n.messages.MessagesFactory;
+import org.apache.knox.gateway.livy.i18n.RMMessages;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class  LivyHaDispatch extends DefaultHaDispatch {
+    private static final String FAILOVER_COUNTER_ATTRIBUTE = "dispatch.ha.failover.counter";
+    private static final String LOCATION = "Location";
+    private static final RMMessages LOG = MessagesFactory.get(RMMessages.class);
 
 Review comment:
   Changed

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


With regards,
Apache Git Services

[GitHub] [knox] risdenk commented on issue #275: KNOX-2253 Add support to handle Livy server redirects

Posted by GitBox <gi...@apache.org>.
risdenk commented on issue #275: KNOX-2253 Add support to handle Livy server redirects
URL: https://github.com/apache/knox/pull/275#issuecomment-592562916
 
 
   Also the travis failures are real:
   
   ```
   [INFO] --- maven-dependency-plugin:3.1.1:analyze-only (analyze-dependencies) @ gateway-service-livy ---
   [WARNING] Used undeclared dependencies found:
   [WARNING]    org.apache.httpcomponents:httpclient:jar:4.5.11:compile
   [WARNING]    org.apache.httpcomponents:httpcore:jar:4.4.13:compile
   ```

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


With regards,
Apache Git Services

[GitHub] [knox] RogPodge commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects

Posted by GitBox <gi...@apache.org>.
RogPodge commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects
URL: https://github.com/apache/knox/pull/275#discussion_r394718067
 
 

 ##########
 File path: gateway-service-livy/src/main/java/org/apache/knox/gateway/livy/i18n/RMMessages.java
 ##########
 @@ -0,0 +1,34 @@
+/*
+ * 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.knox.gateway.livy.i18n;
+
+import org.apache.knox.gateway.ha.dispatch.i18n.HaDispatchMessages;
+import org.apache.knox.gateway.i18n.messages.Message;
+import org.apache.knox.gateway.i18n.messages.MessageLevel;
+import org.apache.knox.gateway.i18n.messages.Messages;
+import org.apache.knox.gateway.i18n.messages.StackTrace;
+
+@Messages(logger = "org.apache.knox.gateway")
+public interface RMMessages extends HaDispatchMessages {
 
 Review comment:
   changed

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


With regards,
Apache Git Services

[GitHub] [knox] risdenk commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects
URL: https://github.com/apache/knox/pull/275#discussion_r385180102
 
 

 ##########
 File path: gateway-service-livy/src/main/java/org/apache/knox/gateway/livy/LivyHaBaseDispatcher.java
 ##########
 @@ -0,0 +1,177 @@
+/*
+ * 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.knox.gateway.livy;
+
+import org.apache.http.HttpResponse;
+import org.apache.http.client.methods.HttpRequestBase;
+import org.apache.http.client.methods.HttpUriRequest;
+import org.apache.http.entity.BufferedHttpEntity;
+import org.apache.knox.gateway.dispatch.DefaultDispatch;
+import org.apache.knox.gateway.filter.AbstractGatewayFilter;
+import org.apache.knox.gateway.ha.provider.HaProvider;
+import org.apache.knox.gateway.ha.provider.impl.HaServiceConfigConstants;
+import org.apache.knox.gateway.i18n.messages.MessagesFactory;
+import org.apache.knox.gateway.livy.i18n.RMMessages;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.atomic.AtomicInteger;
+
+class  LivyHaBaseDispatcher extends DefaultDispatch {
 
 Review comment:
   This whole class at first glance looks like a straight copy of `DefaultHaDispatch`? So why not just extend `DefaultHaDispatch`?

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


With regards,
Apache Git Services

[GitHub] [knox] risdenk commented on issue #275: KNOX-2253 Add support to handle Livy server redirects

Posted by GitBox <gi...@apache.org>.
risdenk commented on issue #275: KNOX-2253 Add support to handle Livy server redirects
URL: https://github.com/apache/knox/pull/275#issuecomment-612264129
 
 
   Thanks @RogPodge just noticed you pushed new changes. Github doesn't notify reviewers of new pushed commits to PRs :/ I'll take another look at this soon.

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


With regards,
Apache Git Services

[GitHub] [knox] risdenk commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects
URL: https://github.com/apache/knox/pull/275#discussion_r385180281
 
 

 ##########
 File path: gateway-service-livy/src/main/java/org/apache/knox/gateway/livy/LivyHaBaseDispatcher.java
 ##########
 @@ -0,0 +1,177 @@
+/*
+ * 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.knox.gateway.livy;
+
+import org.apache.http.HttpResponse;
+import org.apache.http.client.methods.HttpRequestBase;
+import org.apache.http.client.methods.HttpUriRequest;
+import org.apache.http.entity.BufferedHttpEntity;
+import org.apache.knox.gateway.dispatch.DefaultDispatch;
+import org.apache.knox.gateway.filter.AbstractGatewayFilter;
+import org.apache.knox.gateway.ha.provider.HaProvider;
+import org.apache.knox.gateway.ha.provider.impl.HaServiceConfigConstants;
+import org.apache.knox.gateway.i18n.messages.MessagesFactory;
+import org.apache.knox.gateway.livy.i18n.RMMessages;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.atomic.AtomicInteger;
+
+class  LivyHaBaseDispatcher extends DefaultDispatch {
+    private static final String FAILOVER_COUNTER_ATTRIBUTE = "dispatch.ha.failover.counter";
+    private static final String LOCATION = "Location";
+    private static final RMMessages LOG = MessagesFactory.get(RMMessages.class);
 
 Review comment:
   This has leftovers from copying elsewhere.

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


With regards,
Apache Git Services

[GitHub] [knox] risdenk commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects
URL: https://github.com/apache/knox/pull/275#discussion_r385182051
 
 

 ##########
 File path: gateway-service-definitions/src/main/resources/services/livy/0.4.0/service.xml
 ##########
 @@ -17,9 +17,9 @@
 -->
 <service role="LIVYSERVER" name="livy" version="0.4.0">
 
 Review comment:
   Most likely going to want a new version of the service definitions instead of just modifying what is there. I don't think 0.4.0 of Livy had HA redirects.

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


With regards,
Apache Git Services

[GitHub] [knox] risdenk commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects
URL: https://github.com/apache/knox/pull/275#discussion_r385179083
 
 

 ##########
 File path: gateway-service-definitions/src/main/resources/services/livy/0.4.0/service.xml
 ##########
 @@ -17,9 +17,9 @@
 -->
 <service role="LIVYSERVER" name="livy" version="0.4.0">
   <routes>
-    <route path="/livy/**?**"/>
-    <route path="/livy"/>
-    <route path="/livy/"/>
+    <route path="/livy/v1/**?**"/>
+    <route path="/livy/v1"/>
+    <route path="/livy/v1/"/>
 
 Review comment:
   Don't make these changes.

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


With regards,
Apache Git Services

[GitHub] [knox] RogPodge commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects

Posted by GitBox <gi...@apache.org>.
RogPodge commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects
URL: https://github.com/apache/knox/pull/275#discussion_r394718586
 
 

 ##########
 File path: gateway-service-livy/src/main/java/org/apache/knox/gateway/livy/LivyHaDispatch.java
 ##########
 @@ -0,0 +1,156 @@
+/*
+ * 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.knox.gateway.livy;
+
+import org.apache.http.HttpResponse;
+import org.apache.http.client.methods.HttpRequestBase;
+import org.apache.http.client.methods.HttpUriRequest;
+import org.apache.http.entity.BufferedHttpEntity;
+import org.apache.knox.gateway.ha.dispatch.DefaultHaDispatch;
+import org.apache.knox.gateway.filter.AbstractGatewayFilter;
+import org.apache.knox.gateway.i18n.messages.MessagesFactory;
+import org.apache.knox.gateway.livy.i18n.RMMessages;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class  LivyHaDispatch extends DefaultHaDispatch {
+    private static final String FAILOVER_COUNTER_ATTRIBUTE = "dispatch.ha.failover.counter";
+    private static final String LOCATION = "Location";
+    private static final RMMessages LOG = MessagesFactory.get(RMMessages.class);
+    private HttpResponse inboundResponse = null;
+
+
+    public LivyHaDispatch() {
+        setServiceRole("LIVYSERVER");
+    }
+
+    /**
+     *
+     * @return HttpReponse used for unit testing so we
+     * can inject inboundResponse before calling executeRequest method
+     */
+    private HttpResponse getInboundResponse() {
+        HttpResponse response = this.inboundResponse;
+        this.setInboundResponse(null);
+        return response;
+    }
+
+    void setInboundResponse(HttpResponse inboundResponse) {
+        this.inboundResponse = inboundResponse;
+    }
+
+    @Override
+    protected void executeRequest(HttpUriRequest outboundRequest, HttpServletRequest inboundRequest, HttpServletResponse outboundResponse) throws IOException {
+        HttpResponse inboundResponse = this.getInboundResponse();
+        try {
+           if( this.getInboundResponse() == null ) {
+             inboundResponse = executeOutboundRequest(outboundRequest);
+           }
+           writeOutboundResponse(outboundRequest, inboundRequest, outboundResponse, inboundResponse);
+        } catch (StandbyException e) {
+           LOG.errorReceivedFromStandbyNode(e);
+           failoverRequest(outboundRequest, inboundRequest, outboundResponse, inboundResponse, e);
+        } catch (IOException e) {
+           LOG.errorConnectingToServer(outboundRequest.getURI().toString(), e);
+           failoverRequest(outboundRequest, inboundRequest, outboundResponse, inboundResponse, e);
+        }
+     }
+
+    /**
+     * Checks for specific outbound response codes/content to trigger a retry or failover
+     */
+    @Override
+    protected void writeOutboundResponse(HttpUriRequest outboundRequest, HttpServletRequest inboundRequest, HttpServletResponse outboundResponse, HttpResponse inboundResponse) throws IOException {
+       int status = inboundResponse.getStatusLine().getStatusCode();
+       if ( status  == 403 || status == 307) {
+          BufferedHttpEntity entity = new BufferedHttpEntity(inboundResponse.getEntity());
+          inboundResponse.setEntity(entity);
+          ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+          inboundResponse.getEntity().writeTo(outputStream);
+          String body = new String(outputStream.toByteArray(), StandardCharsets.UTF_8);
+          if (body.contains("This is a standby Livy Instance")) {
+             throw new StandbyException();
+          }
+       }
+       super.writeOutboundResponse(outboundRequest, inboundRequest, outboundResponse, inboundResponse);
+    }
+
+    @Override
+    protected void failoverRequest(HttpUriRequest outboundRequest, HttpServletRequest inboundRequest, HttpServletResponse outboundResponse, HttpResponse inboundResponse, Exception exception) throws IOException {
+       LOG.failingOverRequest(outboundRequest.getURI().toString());
+       URI uri;
+       String outboundURIs;
+       AtomicInteger counter = (AtomicInteger) inboundRequest.getAttribute(FAILOVER_COUNTER_ATTRIBUTE);
+       if (counter == null) {
+          counter = new AtomicInteger(0);
+       }
+       inboundRequest.setAttribute(FAILOVER_COUNTER_ATTRIBUTE, counter);
+       outboundURIs = outboundRequest.getURI().toString();
+
+       if (counter.incrementAndGet() <= getMaxFailoverAttempts()) {
+          //null out target url so that rewriters run again
+          inboundRequest.setAttribute(AbstractGatewayFilter.TARGET_REQUEST_URL_ATTRIBUTE_NAME, null);
+
+           uri = getUriFromInbound(inboundRequest, inboundResponse, outboundURIs);
+           ((HttpRequestBase) outboundRequest).setURI(uri);
+          if (getFailoverSleep() > 0) {
+             try {
+                Thread.sleep(getFailoverSleep());
+             } catch (InterruptedException e) {
+                LOG.failoverSleepFailed(getServiceRole(), e);
+             }
+          }
+          executeRequest(outboundRequest, inboundRequest, outboundResponse);
+       } else {
+          LOG.maxFailoverAttemptsReached(getMaxFailoverAttempts(), getServiceRole());
+          if (inboundResponse != null) {
+             writeOutboundResponse(outboundRequest, inboundRequest, outboundResponse, inboundResponse);
+          } else {
+             throw new IOException(exception);
+          }
+       }
+    }
 
 Review comment:
   refactored to inherit from DefaultHADispatch

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


With regards,
Apache Git Services

[GitHub] [knox] risdenk commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects
URL: https://github.com/apache/knox/pull/275#discussion_r385756253
 
 

 ##########
 File path: gateway-service-livy/src/main/java/org/apache/knox/gateway/livy/LivyHaDispatch.java
 ##########
 @@ -0,0 +1,156 @@
+/*
+ * 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.knox.gateway.livy;
+
+import org.apache.http.HttpResponse;
+import org.apache.http.client.methods.HttpRequestBase;
+import org.apache.http.client.methods.HttpUriRequest;
+import org.apache.http.entity.BufferedHttpEntity;
+import org.apache.knox.gateway.ha.dispatch.DefaultHaDispatch;
+import org.apache.knox.gateway.filter.AbstractGatewayFilter;
+import org.apache.knox.gateway.i18n.messages.MessagesFactory;
+import org.apache.knox.gateway.livy.i18n.RMMessages;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class  LivyHaDispatch extends DefaultHaDispatch {
+    private static final String FAILOVER_COUNTER_ATTRIBUTE = "dispatch.ha.failover.counter";
+    private static final String LOCATION = "Location";
+    private static final RMMessages LOG = MessagesFactory.get(RMMessages.class);
+    private HttpResponse inboundResponse = null;
+
+
+    public LivyHaDispatch() {
+        setServiceRole("LIVYSERVER");
+    }
+
+    /**
+     *
+     * @return HttpReponse used for unit testing so we
+     * can inject inboundResponse before calling executeRequest method
+     */
+    private HttpResponse getInboundResponse() {
+        HttpResponse response = this.inboundResponse;
+        this.setInboundResponse(null);
+        return response;
+    }
+
+    void setInboundResponse(HttpResponse inboundResponse) {
+        this.inboundResponse = inboundResponse;
+    }
+
+    @Override
+    protected void executeRequest(HttpUriRequest outboundRequest, HttpServletRequest inboundRequest, HttpServletResponse outboundResponse) throws IOException {
+        HttpResponse inboundResponse = this.getInboundResponse();
+        try {
+           if( this.getInboundResponse() == null ) {
+             inboundResponse = executeOutboundRequest(outboundRequest);
+           }
+           writeOutboundResponse(outboundRequest, inboundRequest, outboundResponse, inboundResponse);
+        } catch (StandbyException e) {
+           LOG.errorReceivedFromStandbyNode(e);
+           failoverRequest(outboundRequest, inboundRequest, outboundResponse, inboundResponse, e);
+        } catch (IOException e) {
+           LOG.errorConnectingToServer(outboundRequest.getURI().toString(), e);
+           failoverRequest(outboundRequest, inboundRequest, outboundResponse, inboundResponse, e);
+        }
+     }
+
+    /**
+     * Checks for specific outbound response codes/content to trigger a retry or failover
+     */
+    @Override
+    protected void writeOutboundResponse(HttpUriRequest outboundRequest, HttpServletRequest inboundRequest, HttpServletResponse outboundResponse, HttpResponse inboundResponse) throws IOException {
+       int status = inboundResponse.getStatusLine().getStatusCode();
+       if ( status  == 403 || status == 307) {
+          BufferedHttpEntity entity = new BufferedHttpEntity(inboundResponse.getEntity());
+          inboundResponse.setEntity(entity);
+          ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+          inboundResponse.getEntity().writeTo(outputStream);
+          String body = new String(outputStream.toByteArray(), StandardCharsets.UTF_8);
+          if (body.contains("This is a standby Livy Instance")) {
+             throw new StandbyException();
+          }
+       }
+       super.writeOutboundResponse(outboundRequest, inboundRequest, outboundResponse, inboundResponse);
+    }
+
+    @Override
+    protected void failoverRequest(HttpUriRequest outboundRequest, HttpServletRequest inboundRequest, HttpServletResponse outboundResponse, HttpResponse inboundResponse, Exception exception) throws IOException {
+       LOG.failingOverRequest(outboundRequest.getURI().toString());
+       URI uri;
+       String outboundURIs;
+       AtomicInteger counter = (AtomicInteger) inboundRequest.getAttribute(FAILOVER_COUNTER_ATTRIBUTE);
+       if (counter == null) {
+          counter = new AtomicInteger(0);
+       }
+       inboundRequest.setAttribute(FAILOVER_COUNTER_ATTRIBUTE, counter);
+       outboundURIs = outboundRequest.getURI().toString();
+
+       if (counter.incrementAndGet() <= getMaxFailoverAttempts()) {
+          //null out target url so that rewriters run again
+          inboundRequest.setAttribute(AbstractGatewayFilter.TARGET_REQUEST_URL_ATTRIBUTE_NAME, null);
+
+           uri = getUriFromInbound(inboundRequest, inboundResponse, outboundURIs);
+           ((HttpRequestBase) outboundRequest).setURI(uri);
+          if (getFailoverSleep() > 0) {
+             try {
+                Thread.sleep(getFailoverSleep());
+             } catch (InterruptedException e) {
+                LOG.failoverSleepFailed(getServiceRole(), e);
+             }
+          }
+          executeRequest(outboundRequest, inboundRequest, outboundResponse);
+       } else {
+          LOG.maxFailoverAttemptsReached(getMaxFailoverAttempts(), getServiceRole());
+          if (inboundResponse != null) {
+             writeOutboundResponse(outboundRequest, inboundRequest, outboundResponse, inboundResponse);
+          } else {
+             throw new IOException(exception);
+          }
+       }
+    }
 
 Review comment:
   This whole method is a copy from `DefaultHaDispatch` so don't need to copy it here?
   
   I guess the only differnce is `getUriFromInbound` but that instead could be an overridden `getDispatchUrl` and add the custom logic there?

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


With regards,
Apache Git Services

[GitHub] [knox] risdenk commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects
URL: https://github.com/apache/knox/pull/275#discussion_r385753189
 
 

 ##########
 File path: gateway-service-livy/src/main/java/org/apache/knox/gateway/livy/LivyHaDispatch.java
 ##########
 @@ -0,0 +1,156 @@
+/*
+ * 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.knox.gateway.livy;
+
+import org.apache.http.HttpResponse;
+import org.apache.http.client.methods.HttpRequestBase;
+import org.apache.http.client.methods.HttpUriRequest;
+import org.apache.http.entity.BufferedHttpEntity;
+import org.apache.knox.gateway.ha.dispatch.DefaultHaDispatch;
+import org.apache.knox.gateway.filter.AbstractGatewayFilter;
+import org.apache.knox.gateway.i18n.messages.MessagesFactory;
+import org.apache.knox.gateway.livy.i18n.RMMessages;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class  LivyHaDispatch extends DefaultHaDispatch {
+    private static final String FAILOVER_COUNTER_ATTRIBUTE = "dispatch.ha.failover.counter";
+    private static final String LOCATION = "Location";
+    private static final RMMessages LOG = MessagesFactory.get(RMMessages.class);
 
 Review comment:
   Still labeled as `RMMessages`.

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


With regards,
Apache Git Services

[GitHub] [knox] risdenk commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects
URL: https://github.com/apache/knox/pull/275#discussion_r385180528
 
 

 ##########
 File path: gateway-service-livy/src/main/java/org/apache/knox/gateway/livy/LivyHaDispatch.java
 ##########
 @@ -0,0 +1,128 @@
+/*
+ * 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.knox.gateway.livy;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.knox.gateway.filter.rewrite.impl.UrlRewriteRequestStream;
+import org.apache.knox.gateway.security.SubjectUtils;
+import org.apache.knox.gateway.i18n.messages.MessagesFactory;
+import org.apache.knox.gateway.livy.i18n.RMMessages;
+import org.apache.knox.gateway.config.Configure;
+import org.apache.knox.gateway.ha.provider.HaProvider;
+import org.apache.knox.gateway.ha.provider.HaServiceConfig;
+
+import javax.servlet.ServletInputStream;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletRequestWrapper;
+import javax.servlet.http.HttpServletResponse;
+import java.io.BufferedReader;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * This specialized dispatch provides Livy specific features to the
+ * default dispatch.
+ */
+public class LivyHaDispatch extends LivyHaBaseDispatcher {
+  private static final String RESOURCE_ROLE = "RESOURCEMANAGER";
+  private HaProvider haProvider;  
+
+  private static final RMMessages LOG = MessagesFactory.get(RMMessages.class);
+
 
 Review comment:
   This has leftovers from copying elsewhere.

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


With regards,
Apache Git Services

[GitHub] [knox] risdenk commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects
URL: https://github.com/apache/knox/pull/275#discussion_r385753973
 
 

 ##########
 File path: gateway-service-definitions/src/main/resources/services/livy/0.4.0/rewrite.xml
 ##########
 @@ -6,9 +6,7 @@
    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
-
 
 Review comment:
   Still changes to this file that aren't needed.

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


With regards,
Apache Git Services

[GitHub] [knox] risdenk commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects
URL: https://github.com/apache/knox/pull/275#discussion_r385753383
 
 

 ##########
 File path: gateway-service-livy/src/main/java/org/apache/knox/gateway/livy/i18n/RMMessages.java
 ##########
 @@ -0,0 +1,34 @@
+/*
+ * 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.knox.gateway.livy.i18n;
+
+import org.apache.knox.gateway.ha.dispatch.i18n.HaDispatchMessages;
+import org.apache.knox.gateway.i18n.messages.Message;
+import org.apache.knox.gateway.i18n.messages.MessageLevel;
+import org.apache.knox.gateway.i18n.messages.Messages;
+import org.apache.knox.gateway.i18n.messages.StackTrace;
+
+@Messages(logger = "org.apache.knox.gateway")
+public interface RMMessages extends HaDispatchMessages {
 
 Review comment:
   Still labeled as `RMMessages`.

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


With regards,
Apache Git Services

[GitHub] [knox] risdenk commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #275: KNOX-2253 Add support to handle Livy server redirects
URL: https://github.com/apache/knox/pull/275#discussion_r385178929
 
 

 ##########
 File path: gateway-service-definitions/src/main/resources/services/livy/0.4.0/rewrite.xml
 ##########
 @@ -16,33 +16,10 @@
    limitations under the License.
 
 Review comment:
   Don't make these unrelated changes? You look to be trying to remove `/v1` from the url handling which isn't necessary.

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


With regards,
Apache Git Services