You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zeppelin.apache.org by pd...@apache.org on 2021/04/16 07:31:48 UTC

[zeppelin] branch branch-0.9 updated: [ZEPPELIN-4983] Download local repo to interpreter nodes

This is an automated email from the ASF dual-hosted git repository.

pdallig pushed a commit to branch branch-0.9
in repository https://gitbox.apache.org/repos/asf/zeppelin.git


The following commit(s) were added to refs/heads/branch-0.9 by this push:
     new e597d14  [ZEPPELIN-4983] Download local repo to interpreter nodes
e597d14 is described below

commit e597d1476d8b3167a94ee96429ee63e5ddf9ce81
Author: Philipp Dallig <ph...@gmail.com>
AuthorDate: Tue Mar 30 12:08:52 2021 +0200

    [ZEPPELIN-4983] Download local repo to interpreter nodes
    
    ### What is this PR for?
    With this PR we synchronise required libraries to an interpreter node that is not running on the same machine as the Zeppelin server.
    The download is done via Thrift, which is already in use as communication between Zeppelin server and interpreter.
    
    ### What type of PR is it?
     - Feature
    
    ### Todos
    * [ ] - How this can be tested?
    
    ### What is the Jira issue?
    * https://issues.apache.org/jira/browse/ZEPPELIN-4983
    
    ### How should this be tested?
    * CI
    
    ### Questions:
    * Does the licenses files need update? No
    * Is there breaking changes for older versions? No
    * Does this needs documentation? No
    
    Author: Philipp Dallig <ph...@gmail.com>
    
    Closes #4072 from Reamer/local_repo and squashes the following commits:
    
    4a35d5b9c [Philipp Dallig] Adding an exit code not equal to 0 in case of an error
    2b8dbd841 [Philipp Dallig] Improve start
    5284082e6 [Philipp Dallig] Be more nullsafe and improve logging.
    9d8ebf012 [Philipp Dallig] Correct interpreter.sh
    52a3fed86 [Philipp Dallig] Download local repo to interpreter nodes
    
    (cherry picked from commit 32c6f76c02a0eedc196a57f41168887aff12a016)
    Signed-off-by: Philipp Dallig <ph...@gmail.com>
---
 bin/interpreter.sh                                 |   10 +
 zeppelin-interpreter/pom.xml                       |    5 +
 .../remote/RemoteInterpreterDownloader.java        |  148 ++
 .../remote/RemoteInterpreterEventClient.java       |    9 +
 .../interpreter/thrift/AngularObjectId.java        |    2 +-
 .../interpreter/thrift/AppOutputAppendEvent.java   |    2 +-
 .../interpreter/thrift/AppOutputUpdateEvent.java   |    2 +-
 .../interpreter/thrift/AppStatusUpdateEvent.java   |    2 +-
 .../interpreter/thrift/InterpreterCompletion.java  |    2 +-
 .../thrift/InterpreterRPCException.java            |    2 +-
 ...ApplicationResult.java => LibraryMetadata.java} |  304 ++-
 .../interpreter/thrift/OutputAppendEvent.java      |    2 +-
 .../interpreter/thrift/OutputUpdateAllEvent.java   |    2 +-
 .../interpreter/thrift/OutputUpdateEvent.java      |    2 +-
 .../zeppelin/interpreter/thrift/ParagraphInfo.java |    2 +-
 .../zeppelin/interpreter/thrift/RegisterInfo.java  |    2 +-
 .../thrift/RemoteApplicationResult.java            |    2 +-
 .../thrift/RemoteInterpreterContext.java           |    2 +-
 .../interpreter/thrift/RemoteInterpreterEvent.java |    2 +-
 .../thrift/RemoteInterpreterEventService.java      | 1941 +++++++++++++++++++-
 .../thrift/RemoteInterpreterEventType.java         |    2 +-
 .../thrift/RemoteInterpreterResult.java            |    2 +-
 .../thrift/RemoteInterpreterResultMessage.java     |    2 +-
 .../thrift/RemoteInterpreterService.java           |    2 +-
 .../interpreter/thrift/RunParagraphsEvent.java     |    2 +-
 .../interpreter/thrift/ServiceException.java       |    2 +-
 .../zeppelin/interpreter/thrift/WebUrlInfo.java    |    2 +-
 .../thrift/RemoteInterpreterEventService.thrift    |   11 +-
 .../interpreter/RemoteInterpreterEventServer.java  |   62 +-
 29 files changed, 2351 insertions(+), 181 deletions(-)

diff --git a/bin/interpreter.sh b/bin/interpreter.sh
index 1280ac0..16311d6 100755
--- a/bin/interpreter.sh
+++ b/bin/interpreter.sh
@@ -24,6 +24,15 @@ function usage() {
     echo "usage) $0 -p <port> -r <intp_port> -d <interpreter dir to load> -l <local interpreter repo dir to load> -g <interpreter group name>"
 }
 
+function downloadInterpreterLibraries() {
+    mkdir -p ${LOCAL_INTERPRETER_REPO}
+    IFS=' ' read -r -a JAVA_INTP_OPTS_ARRAY <<< "${JAVA_INTP_OPTS}"
+    ZEPPELIN_DOWNLOADER="org.apache.zeppelin.interpreter.remote.RemoteInterpreterDownloader"
+    INTERPRETER_DOWNLOAD_COMMAND+=("${ZEPPELIN_RUNNER}" "${JAVA_INTP_OPTS_ARRAY[@]}" "-cp" "${ZEPPELIN_INTP_CLASSPATH_OVERRIDES}:${ZEPPELIN_INTP_CLASSPATH}" "${ZEPPELIN_DOWNLOADER}" "${CALLBACK_HOST}" "${PORT}" "${INTERPRETER_SETTING_NAME}" "${LOCAL_INTERPRETER_REPO}")
+    echo "Interpreter download command: ${INTERPRETER_DOWNLOAD_COMMAND[@]}"
+    eval "${INTERPRETER_DOWNLOAD_COMMAND[@]}"
+}
+
 # pre-requisites for checking that we're running in container
 if [ -f /proc/self/cgroup ] && [ -n "$(command -v getent)" ]; then
     # checks if we're running in container...
@@ -267,6 +276,7 @@ elif [[ "${INTERPRETER_ID}" == "flink" ]]; then
 
 fi
 
+downloadInterpreterLibraries
 addJarInDirForIntp "${LOCAL_INTERPRETER_REPO}"
 
 if [[ -n "$ZEPPELIN_IMPERSONATE_USER" ]]; then
diff --git a/zeppelin-interpreter/pom.xml b/zeppelin-interpreter/pom.xml
index 303787f..9f85954 100644
--- a/zeppelin-interpreter/pom.xml
+++ b/zeppelin-interpreter/pom.xml
@@ -137,6 +137,11 @@
     </dependency>
 
     <dependency>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
+    </dependency>
+
+    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-log4j12</artifactId>
     </dependency>
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterDownloader.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterDownloader.java
new file mode 100644
index 0000000..f385478
--- /dev/null
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterDownloader.java
@@ -0,0 +1,148 @@
+/*
+ * 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.zeppelin.interpreter.remote;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+
+import org.apache.commons.io.FileUtils;
+import org.apache.zeppelin.interpreter.thrift.LibraryMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RemoteInterpreterDownloader {
+  private static final Logger LOGGER = LoggerFactory.getLogger(RemoteInterpreterDownloader.class);
+
+  private static final int MAX_LIBRARY_DOWNLOAD_ATTEMPTS = 3;
+
+  private final RemoteInterpreterEventClient client;
+  private final String interpreter;
+  private final File localRepoDir;
+
+  public RemoteInterpreterDownloader(
+      String interpreter,
+      RemoteInterpreterEventClient client,
+      File localRepoDir) {
+    this.client = client;
+    this.interpreter = interpreter;
+    this.localRepoDir = localRepoDir;
+  }
+
+  public static void main(String[] args) {
+    if (args.length == 4) {
+      String zeppelinServerHost = args[0];
+      int port = Integer.parseInt(args[1]);
+      String interpreter = args[2];
+      String localRepoPath = args[3];
+      RemoteInterpreterEventClient intpEventClient = new RemoteInterpreterEventClient(
+          zeppelinServerHost, port, 3);
+
+      RemoteInterpreterDownloader downloader = new RemoteInterpreterDownloader(interpreter,
+          intpEventClient, new File(localRepoPath));
+      downloader.syncAllLibraries();
+    } else {
+      LOGGER.error(
+          "Wrong amount of Arguments. Expected: [ZeppelinHostname] [ZeppelinPort] [InterpreterName] [LocalRepoPath]");
+      // based on sysexits.h, 64 indicated a command line usage error
+      System.exit(64);
+    }
+  }
+
+  private void syncAllLibraries() {
+    LOGGER.info("Loading all libraries for interpreter {} to {}", interpreter, localRepoDir);
+    List<LibraryMetadata> metadatas = client.getAllLibraryMetadatas(interpreter);
+    if (!localRepoDir.isDirectory()) {
+      LOGGER.error("{} is no directory", localRepoDir);
+      return;
+    }
+    Set<String> syncedLibraries = new HashSet<>();
+    // Add or update new libraries
+    for (LibraryMetadata metadata : metadatas) {
+      File library = new File(localRepoDir, metadata.getName());
+      addOrUpdateLibrary(library, metadata);
+      syncedLibraries.add(metadata.getName());
+    }
+    // Delete old Jar files
+    for (File file : FileUtils.listFiles(localRepoDir, new String[] { "jar" }, false)) {
+      if (!syncedLibraries.contains(file.getName())) {
+        try {
+          LOGGER.info("Delete {}, because it's not present on the server side", file.toPath());
+          Files.delete(file.toPath());
+        } catch (IOException e) {
+          LOGGER.error("Unable to delete old library {} during sync.", file, e);
+        }
+      }
+    }
+  }
+
+  private void addOrUpdateLibrary(File library, LibraryMetadata metadata) {
+    try {
+      if (library.exists() && library.canRead()) {
+        long localChecksum = FileUtils.checksumCRC32(library);
+        if (localChecksum == metadata.getChecksum()) {
+          // nothing to do if checksum is matching
+          LOGGER.info("Library {} is present ", library.getName());
+        } else {
+          // checksum didn't match
+          Files.delete(library.toPath());
+          downloadLibrary(library, metadata);
+        }
+      } else {
+        downloadLibrary(library, metadata);
+      }
+    } catch (IOException e) {
+      LOGGER.error("Can not add or update library {}", library, e);
+    }
+  }
+
+  private void downloadLibrary(File library, LibraryMetadata metadata) {
+    LOGGER.debug("Trying to download library {} to {}", metadata.getName(), library);
+    try {
+      if (library.createNewFile()) {
+        int attempt = 0;
+        while (attempt < MAX_LIBRARY_DOWNLOAD_ATTEMPTS) {
+          ByteBuffer bytes = client.getLibrary(interpreter, metadata.getName());
+          if (bytes == null) {
+            LOGGER.error("ByteBuffer of library {} is null."
+                + " For a detailed message take a look into Zeppelin-Server log. Attempt {} of {}",
+                metadata.getName(), ++attempt, MAX_LIBRARY_DOWNLOAD_ATTEMPTS);
+          } else {
+            FileUtils.writeByteArrayToFile(library, bytes.array());
+            if (FileUtils.checksumCRC32(library) == metadata.getChecksum()) {
+              LOGGER.info("Library {} successfully transfered", library.getName());
+              break;
+            } else {
+              LOGGER.error("Library Checksum didn't match. Attempt {} of {}", ++attempt,
+                  MAX_LIBRARY_DOWNLOAD_ATTEMPTS);
+            }
+          }
+        }
+      } else {
+        LOGGER.error("Unable to create a new library file {}", library);
+      }
+    } catch (IOException e) {
+      LOGGER.error("Unable to download Library {}", metadata.getName(), e);
+    }
+  }
+}
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterEventClient.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterEventClient.java
index eb0decf..8090123 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterEventClient.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterEventClient.java
@@ -28,6 +28,7 @@ import org.apache.zeppelin.interpreter.InterpreterResultMessage;
 import org.apache.zeppelin.interpreter.thrift.AppOutputAppendEvent;
 import org.apache.zeppelin.interpreter.thrift.AppOutputUpdateEvent;
 import org.apache.zeppelin.interpreter.thrift.AppStatusUpdateEvent;
+import org.apache.zeppelin.interpreter.thrift.LibraryMetadata;
 import org.apache.zeppelin.interpreter.thrift.OutputAppendEvent;
 import org.apache.zeppelin.interpreter.thrift.OutputUpdateAllEvent;
 import org.apache.zeppelin.interpreter.thrift.OutputUpdateEvent;
@@ -130,6 +131,14 @@ public class RemoteInterpreterEventClient implements ResourcePoolConnector,
     return callRemoteFunction(client -> client.getParagraphList(user, noteId));
   }
 
+  public List<LibraryMetadata> getAllLibraryMetadatas(String interpreter) {
+    return callRemoteFunction(client -> client.getAllLibraryMetadatas(interpreter));
+  }
+
+  public ByteBuffer getLibrary(String interpreter, String libraryName) {
+    return callRemoteFunction(client -> client.getLibrary(interpreter, libraryName));
+  }
+
   @Override
   public Object readResource(ResourceId resourceId) {
     try {
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/AngularObjectId.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/AngularObjectId.java
index b2239bd..8f93f2e 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/AngularObjectId.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/AngularObjectId.java
@@ -24,7 +24,7 @@
 package org.apache.zeppelin.interpreter.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-01-22")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-03-09")
 public class AngularObjectId implements org.apache.thrift.TBase<AngularObjectId, AngularObjectId._Fields>, java.io.Serializable, Cloneable, Comparable<AngularObjectId> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AngularObjectId");
 
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/AppOutputAppendEvent.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/AppOutputAppendEvent.java
index 9933931..676e469 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/AppOutputAppendEvent.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/AppOutputAppendEvent.java
@@ -24,7 +24,7 @@
 package org.apache.zeppelin.interpreter.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-01-22")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-03-09")
 public class AppOutputAppendEvent implements org.apache.thrift.TBase<AppOutputAppendEvent, AppOutputAppendEvent._Fields>, java.io.Serializable, Cloneable, Comparable<AppOutputAppendEvent> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AppOutputAppendEvent");
 
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/AppOutputUpdateEvent.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/AppOutputUpdateEvent.java
index 2179356..46b0ac6 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/AppOutputUpdateEvent.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/AppOutputUpdateEvent.java
@@ -24,7 +24,7 @@
 package org.apache.zeppelin.interpreter.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-01-22")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-03-09")
 public class AppOutputUpdateEvent implements org.apache.thrift.TBase<AppOutputUpdateEvent, AppOutputUpdateEvent._Fields>, java.io.Serializable, Cloneable, Comparable<AppOutputUpdateEvent> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AppOutputUpdateEvent");
 
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/AppStatusUpdateEvent.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/AppStatusUpdateEvent.java
index ea4df90..57a18c6 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/AppStatusUpdateEvent.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/AppStatusUpdateEvent.java
@@ -24,7 +24,7 @@
 package org.apache.zeppelin.interpreter.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-01-22")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-03-09")
 public class AppStatusUpdateEvent implements org.apache.thrift.TBase<AppStatusUpdateEvent, AppStatusUpdateEvent._Fields>, java.io.Serializable, Cloneable, Comparable<AppStatusUpdateEvent> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AppStatusUpdateEvent");
 
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/InterpreterCompletion.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/InterpreterCompletion.java
index 2a27646..7a92c07 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/InterpreterCompletion.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/InterpreterCompletion.java
@@ -24,7 +24,7 @@
 package org.apache.zeppelin.interpreter.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-01-22")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-03-09")
 public class InterpreterCompletion implements org.apache.thrift.TBase<InterpreterCompletion, InterpreterCompletion._Fields>, java.io.Serializable, Cloneable, Comparable<InterpreterCompletion> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("InterpreterCompletion");
 
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/InterpreterRPCException.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/InterpreterRPCException.java
index 4ef03bc..f86ac4d 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/InterpreterRPCException.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/InterpreterRPCException.java
@@ -24,7 +24,7 @@
 package org.apache.zeppelin.interpreter.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-01-22")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-03-09")
 public class InterpreterRPCException extends org.apache.thrift.TException implements org.apache.thrift.TBase<InterpreterRPCException, InterpreterRPCException._Fields>, java.io.Serializable, Cloneable, Comparable<InterpreterRPCException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("InterpreterRPCException");
 
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteApplicationResult.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/LibraryMetadata.java
similarity index 59%
copy from zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteApplicationResult.java
copy to zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/LibraryMetadata.java
index 8620fc4..4299eb6 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteApplicationResult.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/LibraryMetadata.java
@@ -24,23 +24,23 @@
 package org.apache.zeppelin.interpreter.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-01-22")
-public class RemoteApplicationResult implements org.apache.thrift.TBase<RemoteApplicationResult, RemoteApplicationResult._Fields>, java.io.Serializable, Cloneable, Comparable<RemoteApplicationResult> {
-  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RemoteApplicationResult");
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-03-09")
+public class LibraryMetadata implements org.apache.thrift.TBase<LibraryMetadata, LibraryMetadata._Fields>, java.io.Serializable, Cloneable, Comparable<LibraryMetadata> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LibraryMetadata");
 
-  private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)1);
-  private static final org.apache.thrift.protocol.TField MSG_FIELD_DESC = new org.apache.thrift.protocol.TField("msg", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField CHECKSUM_FIELD_DESC = new org.apache.thrift.protocol.TField("checksum", org.apache.thrift.protocol.TType.I64, (short)2);
 
-  private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new RemoteApplicationResultStandardSchemeFactory();
-  private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new RemoteApplicationResultTupleSchemeFactory();
+  private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new LibraryMetadataStandardSchemeFactory();
+  private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new LibraryMetadataTupleSchemeFactory();
 
-  public boolean success; // required
-  public @org.apache.thrift.annotation.Nullable java.lang.String msg; // required
+  public @org.apache.thrift.annotation.Nullable java.lang.String name; // required
+  public long checksum; // required
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-    SUCCESS((short)1, "success"),
-    MSG((short)2, "msg");
+    NAME((short)1, "name"),
+    CHECKSUM((short)2, "checksum");
 
     private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -56,10 +56,10 @@ public class RemoteApplicationResult implements org.apache.thrift.TBase<RemoteAp
     @org.apache.thrift.annotation.Nullable
     public static _Fields findByThriftId(int fieldId) {
       switch(fieldId) {
-        case 1: // SUCCESS
-          return SUCCESS;
-        case 2: // MSG
-          return MSG;
+        case 1: // NAME
+          return NAME;
+        case 2: // CHECKSUM
+          return CHECKSUM;
         default:
           return null;
       }
@@ -101,117 +101,117 @@ public class RemoteApplicationResult implements org.apache.thrift.TBase<RemoteAp
   }
 
   // isset id assignments
-  private static final int __SUCCESS_ISSET_ID = 0;
+  private static final int __CHECKSUM_ISSET_ID = 0;
   private byte __isset_bitfield = 0;
   public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-    tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
-    tmpMap.put(_Fields.MSG, new org.apache.thrift.meta_data.FieldMetaData("msg", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+    tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.CHECKSUM, new org.apache.thrift.meta_data.FieldMetaData("checksum", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(RemoteApplicationResult.class, metaDataMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(LibraryMetadata.class, metaDataMap);
   }
 
-  public RemoteApplicationResult() {
+  public LibraryMetadata() {
   }
 
-  public RemoteApplicationResult(
-    boolean success,
-    java.lang.String msg)
+  public LibraryMetadata(
+    java.lang.String name,
+    long checksum)
   {
     this();
-    this.success = success;
-    setSuccessIsSet(true);
-    this.msg = msg;
+    this.name = name;
+    this.checksum = checksum;
+    setChecksumIsSet(true);
   }
 
   /**
    * Performs a deep copy on <i>other</i>.
    */
-  public RemoteApplicationResult(RemoteApplicationResult other) {
+  public LibraryMetadata(LibraryMetadata other) {
     __isset_bitfield = other.__isset_bitfield;
-    this.success = other.success;
-    if (other.isSetMsg()) {
-      this.msg = other.msg;
+    if (other.isSetName()) {
+      this.name = other.name;
     }
+    this.checksum = other.checksum;
   }
 
-  public RemoteApplicationResult deepCopy() {
-    return new RemoteApplicationResult(this);
+  public LibraryMetadata deepCopy() {
+    return new LibraryMetadata(this);
   }
 
   @Override
   public void clear() {
-    setSuccessIsSet(false);
-    this.success = false;
-    this.msg = null;
+    this.name = null;
+    setChecksumIsSet(false);
+    this.checksum = 0;
   }
 
-  public boolean isSuccess() {
-    return this.success;
+  @org.apache.thrift.annotation.Nullable
+  public java.lang.String getName() {
+    return this.name;
   }
 
-  public RemoteApplicationResult setSuccess(boolean success) {
-    this.success = success;
-    setSuccessIsSet(true);
+  public LibraryMetadata setName(@org.apache.thrift.annotation.Nullable java.lang.String name) {
+    this.name = name;
     return this;
   }
 
-  public void unsetSuccess() {
-    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+  public void unsetName() {
+    this.name = null;
   }
 
-  /** Returns true if field success is set (has been assigned a value) and false otherwise */
-  public boolean isSetSuccess() {
-    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+  /** Returns true if field name is set (has been assigned a value) and false otherwise */
+  public boolean isSetName() {
+    return this.name != null;
   }
 
-  public void setSuccessIsSet(boolean value) {
-    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value);
+  public void setNameIsSet(boolean value) {
+    if (!value) {
+      this.name = null;
+    }
   }
 
-  @org.apache.thrift.annotation.Nullable
-  public java.lang.String getMsg() {
-    return this.msg;
+  public long getChecksum() {
+    return this.checksum;
   }
 
-  public RemoteApplicationResult setMsg(@org.apache.thrift.annotation.Nullable java.lang.String msg) {
-    this.msg = msg;
+  public LibraryMetadata setChecksum(long checksum) {
+    this.checksum = checksum;
+    setChecksumIsSet(true);
     return this;
   }
 
-  public void unsetMsg() {
-    this.msg = null;
+  public void unsetChecksum() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __CHECKSUM_ISSET_ID);
   }
 
-  /** Returns true if field msg is set (has been assigned a value) and false otherwise */
-  public boolean isSetMsg() {
-    return this.msg != null;
+  /** Returns true if field checksum is set (has been assigned a value) and false otherwise */
+  public boolean isSetChecksum() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __CHECKSUM_ISSET_ID);
   }
 
-  public void setMsgIsSet(boolean value) {
-    if (!value) {
-      this.msg = null;
-    }
+  public void setChecksumIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __CHECKSUM_ISSET_ID, value);
   }
 
   public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
     switch (field) {
-    case SUCCESS:
+    case NAME:
       if (value == null) {
-        unsetSuccess();
+        unsetName();
       } else {
-        setSuccess((java.lang.Boolean)value);
+        setName((java.lang.String)value);
       }
       break;
 
-    case MSG:
+    case CHECKSUM:
       if (value == null) {
-        unsetMsg();
+        unsetChecksum();
       } else {
-        setMsg((java.lang.String)value);
+        setChecksum((java.lang.Long)value);
       }
       break;
 
@@ -221,11 +221,11 @@ public class RemoteApplicationResult implements org.apache.thrift.TBase<RemoteAp
   @org.apache.thrift.annotation.Nullable
   public java.lang.Object getFieldValue(_Fields field) {
     switch (field) {
-    case SUCCESS:
-      return isSuccess();
+    case NAME:
+      return getName();
 
-    case MSG:
-      return getMsg();
+    case CHECKSUM:
+      return getChecksum();
 
     }
     throw new java.lang.IllegalStateException();
@@ -238,10 +238,10 @@ public class RemoteApplicationResult implements org.apache.thrift.TBase<RemoteAp
     }
 
     switch (field) {
-    case SUCCESS:
-      return isSetSuccess();
-    case MSG:
-      return isSetMsg();
+    case NAME:
+      return isSetName();
+    case CHECKSUM:
+      return isSetChecksum();
     }
     throw new java.lang.IllegalStateException();
   }
@@ -250,32 +250,32 @@ public class RemoteApplicationResult implements org.apache.thrift.TBase<RemoteAp
   public boolean equals(java.lang.Object that) {
     if (that == null)
       return false;
-    if (that instanceof RemoteApplicationResult)
-      return this.equals((RemoteApplicationResult)that);
+    if (that instanceof LibraryMetadata)
+      return this.equals((LibraryMetadata)that);
     return false;
   }
 
-  public boolean equals(RemoteApplicationResult that) {
+  public boolean equals(LibraryMetadata that) {
     if (that == null)
       return false;
     if (this == that)
       return true;
 
-    boolean this_present_success = true;
-    boolean that_present_success = true;
-    if (this_present_success || that_present_success) {
-      if (!(this_present_success && that_present_success))
+    boolean this_present_name = true && this.isSetName();
+    boolean that_present_name = true && that.isSetName();
+    if (this_present_name || that_present_name) {
+      if (!(this_present_name && that_present_name))
         return false;
-      if (this.success != that.success)
+      if (!this.name.equals(that.name))
         return false;
     }
 
-    boolean this_present_msg = true && this.isSetMsg();
-    boolean that_present_msg = true && that.isSetMsg();
-    if (this_present_msg || that_present_msg) {
-      if (!(this_present_msg && that_present_msg))
+    boolean this_present_checksum = true;
+    boolean that_present_checksum = true;
+    if (this_present_checksum || that_present_checksum) {
+      if (!(this_present_checksum && that_present_checksum))
         return false;
-      if (!this.msg.equals(that.msg))
+      if (this.checksum != that.checksum)
         return false;
     }
 
@@ -286,39 +286,39 @@ public class RemoteApplicationResult implements org.apache.thrift.TBase<RemoteAp
   public int hashCode() {
     int hashCode = 1;
 
-    hashCode = hashCode * 8191 + ((success) ? 131071 : 524287);
+    hashCode = hashCode * 8191 + ((isSetName()) ? 131071 : 524287);
+    if (isSetName())
+      hashCode = hashCode * 8191 + name.hashCode();
 
-    hashCode = hashCode * 8191 + ((isSetMsg()) ? 131071 : 524287);
-    if (isSetMsg())
-      hashCode = hashCode * 8191 + msg.hashCode();
+    hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(checksum);
 
     return hashCode;
   }
 
   @Override
-  public int compareTo(RemoteApplicationResult other) {
+  public int compareTo(LibraryMetadata other) {
     if (!getClass().equals(other.getClass())) {
       return getClass().getName().compareTo(other.getClass().getName());
     }
 
     int lastComparison = 0;
 
-    lastComparison = java.lang.Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+    lastComparison = java.lang.Boolean.valueOf(isSetName()).compareTo(other.isSetName());
     if (lastComparison != 0) {
       return lastComparison;
     }
-    if (isSetSuccess()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+    if (isSetName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = java.lang.Boolean.valueOf(isSetMsg()).compareTo(other.isSetMsg());
+    lastComparison = java.lang.Boolean.valueOf(isSetChecksum()).compareTo(other.isSetChecksum());
     if (lastComparison != 0) {
       return lastComparison;
     }
-    if (isSetMsg()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.msg, other.msg);
+    if (isSetChecksum()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.checksum, other.checksum);
       if (lastComparison != 0) {
         return lastComparison;
       }
@@ -341,26 +341,30 @@ public class RemoteApplicationResult implements org.apache.thrift.TBase<RemoteAp
 
   @Override
   public java.lang.String toString() {
-    java.lang.StringBuilder sb = new java.lang.StringBuilder("RemoteApplicationResult(");
+    java.lang.StringBuilder sb = new java.lang.StringBuilder("LibraryMetadata(");
     boolean first = true;
 
-    sb.append("success:");
-    sb.append(this.success);
-    first = false;
-    if (!first) sb.append(", ");
-    sb.append("msg:");
-    if (this.msg == null) {
+    sb.append("name:");
+    if (this.name == null) {
       sb.append("null");
     } else {
-      sb.append(this.msg);
+      sb.append(this.name);
     }
     first = false;
+    if (!first) sb.append(", ");
+    sb.append("checksum:");
+    sb.append(this.checksum);
+    first = false;
     sb.append(")");
     return sb.toString();
   }
 
   public void validate() throws org.apache.thrift.TException {
     // check for required fields
+    if (name == null) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'name' was not present! Struct: " + toString());
+    }
+    // alas, we cannot check 'checksum' because it's a primitive and you chose the non-beans generator.
     // check for sub-struct validity
   }
 
@@ -382,15 +386,15 @@ public class RemoteApplicationResult implements org.apache.thrift.TBase<RemoteAp
     }
   }
 
-  private static class RemoteApplicationResultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-    public RemoteApplicationResultStandardScheme getScheme() {
-      return new RemoteApplicationResultStandardScheme();
+  private static class LibraryMetadataStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+    public LibraryMetadataStandardScheme getScheme() {
+      return new LibraryMetadataStandardScheme();
     }
   }
 
-  private static class RemoteApplicationResultStandardScheme extends org.apache.thrift.scheme.StandardScheme<RemoteApplicationResult> {
+  private static class LibraryMetadataStandardScheme extends org.apache.thrift.scheme.StandardScheme<LibraryMetadata> {
 
-    public void read(org.apache.thrift.protocol.TProtocol iprot, RemoteApplicationResult struct) throws org.apache.thrift.TException {
+    public void read(org.apache.thrift.protocol.TProtocol iprot, LibraryMetadata struct) throws org.apache.thrift.TException {
       org.apache.thrift.protocol.TField schemeField;
       iprot.readStructBegin();
       while (true)
@@ -400,18 +404,18 @@ public class RemoteApplicationResult implements org.apache.thrift.TBase<RemoteAp
           break;
         }
         switch (schemeField.id) {
-          case 1: // SUCCESS
-            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
-              struct.success = iprot.readBool();
-              struct.setSuccessIsSet(true);
+          case 1: // NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.name = iprot.readString();
+              struct.setNameIsSet(true);
             } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 2: // MSG
-            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-              struct.msg = iprot.readString();
-              struct.setMsgIsSet(true);
+          case 2: // CHECKSUM
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.checksum = iprot.readI64();
+              struct.setChecksumIsSet(true);
             } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
@@ -424,66 +428,52 @@ public class RemoteApplicationResult implements org.apache.thrift.TBase<RemoteAp
       iprot.readStructEnd();
 
       // check for required fields of primitive type, which can't be checked in the validate method
+      if (!struct.isSetChecksum()) {
+        throw new org.apache.thrift.protocol.TProtocolException("Required field 'checksum' was not found in serialized data! Struct: " + toString());
+      }
       struct.validate();
     }
 
-    public void write(org.apache.thrift.protocol.TProtocol oprot, RemoteApplicationResult struct) throws org.apache.thrift.TException {
+    public void write(org.apache.thrift.protocol.TProtocol oprot, LibraryMetadata struct) throws org.apache.thrift.TException {
       struct.validate();
 
       oprot.writeStructBegin(STRUCT_DESC);
-      oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
-      oprot.writeBool(struct.success);
-      oprot.writeFieldEnd();
-      if (struct.msg != null) {
-        oprot.writeFieldBegin(MSG_FIELD_DESC);
-        oprot.writeString(struct.msg);
+      if (struct.name != null) {
+        oprot.writeFieldBegin(NAME_FIELD_DESC);
+        oprot.writeString(struct.name);
         oprot.writeFieldEnd();
       }
+      oprot.writeFieldBegin(CHECKSUM_FIELD_DESC);
+      oprot.writeI64(struct.checksum);
+      oprot.writeFieldEnd();
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
 
   }
 
-  private static class RemoteApplicationResultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-    public RemoteApplicationResultTupleScheme getScheme() {
-      return new RemoteApplicationResultTupleScheme();
+  private static class LibraryMetadataTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+    public LibraryMetadataTupleScheme getScheme() {
+      return new LibraryMetadataTupleScheme();
     }
   }
 
-  private static class RemoteApplicationResultTupleScheme extends org.apache.thrift.scheme.TupleScheme<RemoteApplicationResult> {
+  private static class LibraryMetadataTupleScheme extends org.apache.thrift.scheme.TupleScheme<LibraryMetadata> {
 
     @Override
-    public void write(org.apache.thrift.protocol.TProtocol prot, RemoteApplicationResult struct) throws org.apache.thrift.TException {
+    public void write(org.apache.thrift.protocol.TProtocol prot, LibraryMetadata struct) throws org.apache.thrift.TException {
       org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-      java.util.BitSet optionals = new java.util.BitSet();
-      if (struct.isSetSuccess()) {
-        optionals.set(0);
-      }
-      if (struct.isSetMsg()) {
-        optionals.set(1);
-      }
-      oprot.writeBitSet(optionals, 2);
-      if (struct.isSetSuccess()) {
-        oprot.writeBool(struct.success);
-      }
-      if (struct.isSetMsg()) {
-        oprot.writeString(struct.msg);
-      }
+      oprot.writeString(struct.name);
+      oprot.writeI64(struct.checksum);
     }
 
     @Override
-    public void read(org.apache.thrift.protocol.TProtocol prot, RemoteApplicationResult struct) throws org.apache.thrift.TException {
+    public void read(org.apache.thrift.protocol.TProtocol prot, LibraryMetadata struct) throws org.apache.thrift.TException {
       org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-      java.util.BitSet incoming = iprot.readBitSet(2);
-      if (incoming.get(0)) {
-        struct.success = iprot.readBool();
-        struct.setSuccessIsSet(true);
-      }
-      if (incoming.get(1)) {
-        struct.msg = iprot.readString();
-        struct.setMsgIsSet(true);
-      }
+      struct.name = iprot.readString();
+      struct.setNameIsSet(true);
+      struct.checksum = iprot.readI64();
+      struct.setChecksumIsSet(true);
     }
   }
 
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/OutputAppendEvent.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/OutputAppendEvent.java
index e7cfefe..67a3a8f 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/OutputAppendEvent.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/OutputAppendEvent.java
@@ -24,7 +24,7 @@
 package org.apache.zeppelin.interpreter.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-01-22")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-03-09")
 public class OutputAppendEvent implements org.apache.thrift.TBase<OutputAppendEvent, OutputAppendEvent._Fields>, java.io.Serializable, Cloneable, Comparable<OutputAppendEvent> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("OutputAppendEvent");
 
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/OutputUpdateAllEvent.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/OutputUpdateAllEvent.java
index b2d9de1..587359f 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/OutputUpdateAllEvent.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/OutputUpdateAllEvent.java
@@ -24,7 +24,7 @@
 package org.apache.zeppelin.interpreter.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-01-22")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-03-09")
 public class OutputUpdateAllEvent implements org.apache.thrift.TBase<OutputUpdateAllEvent, OutputUpdateAllEvent._Fields>, java.io.Serializable, Cloneable, Comparable<OutputUpdateAllEvent> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("OutputUpdateAllEvent");
 
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/OutputUpdateEvent.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/OutputUpdateEvent.java
index 938cfa3..ef5d2f0 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/OutputUpdateEvent.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/OutputUpdateEvent.java
@@ -24,7 +24,7 @@
 package org.apache.zeppelin.interpreter.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-01-22")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-03-09")
 public class OutputUpdateEvent implements org.apache.thrift.TBase<OutputUpdateEvent, OutputUpdateEvent._Fields>, java.io.Serializable, Cloneable, Comparable<OutputUpdateEvent> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("OutputUpdateEvent");
 
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/ParagraphInfo.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/ParagraphInfo.java
index 37b9f94..083d800 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/ParagraphInfo.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/ParagraphInfo.java
@@ -24,7 +24,7 @@
 package org.apache.zeppelin.interpreter.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-01-22")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-03-09")
 public class ParagraphInfo implements org.apache.thrift.TBase<ParagraphInfo, ParagraphInfo._Fields>, java.io.Serializable, Cloneable, Comparable<ParagraphInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ParagraphInfo");
 
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RegisterInfo.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RegisterInfo.java
index 10d4e12..2c3667e 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RegisterInfo.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RegisterInfo.java
@@ -24,7 +24,7 @@
 package org.apache.zeppelin.interpreter.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-01-22")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-03-09")
 public class RegisterInfo implements org.apache.thrift.TBase<RegisterInfo, RegisterInfo._Fields>, java.io.Serializable, Cloneable, Comparable<RegisterInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RegisterInfo");
 
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteApplicationResult.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteApplicationResult.java
index 8620fc4..8aa5bc2 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteApplicationResult.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteApplicationResult.java
@@ -24,7 +24,7 @@
 package org.apache.zeppelin.interpreter.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-01-22")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-03-09")
 public class RemoteApplicationResult implements org.apache.thrift.TBase<RemoteApplicationResult, RemoteApplicationResult._Fields>, java.io.Serializable, Cloneable, Comparable<RemoteApplicationResult> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RemoteApplicationResult");
 
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterContext.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterContext.java
index faf6079..a6243c8 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterContext.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterContext.java
@@ -24,7 +24,7 @@
 package org.apache.zeppelin.interpreter.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-01-22")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-03-09")
 public class RemoteInterpreterContext implements org.apache.thrift.TBase<RemoteInterpreterContext, RemoteInterpreterContext._Fields>, java.io.Serializable, Cloneable, Comparable<RemoteInterpreterContext> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RemoteInterpreterContext");
 
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterEvent.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterEvent.java
index daa4599..eb22200 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterEvent.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterEvent.java
@@ -24,7 +24,7 @@
 package org.apache.zeppelin.interpreter.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-01-22")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-03-09")
 public class RemoteInterpreterEvent implements org.apache.thrift.TBase<RemoteInterpreterEvent, RemoteInterpreterEvent._Fields>, java.io.Serializable, Cloneable, Comparable<RemoteInterpreterEvent> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RemoteInterpreterEvent");
 
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterEventService.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterEventService.java
index cc21ac0..9a91dbd 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterEventService.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterEventService.java
@@ -24,7 +24,7 @@
 package org.apache.zeppelin.interpreter.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-01-22")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-03-09")
 public class RemoteInterpreterEventService {
 
   public interface Iface {
@@ -69,6 +69,10 @@ public class RemoteInterpreterEventService {
 
     public java.util.List<ParagraphInfo> getParagraphList(java.lang.String user, java.lang.String noteId) throws org.apache.zeppelin.interpreter.thrift.InterpreterRPCException, org.apache.thrift.TException;
 
+    public java.util.List<LibraryMetadata> getAllLibraryMetadatas(java.lang.String intpSettingName) throws org.apache.thrift.TException;
+
+    public java.nio.ByteBuffer getLibrary(java.lang.String intpSettingName, java.lang.String libraryName) throws org.apache.thrift.TException;
+
   }
 
   public interface AsyncIface {
@@ -113,6 +117,10 @@ public class RemoteInterpreterEventService {
 
     public void getParagraphList(java.lang.String user, java.lang.String noteId, org.apache.thrift.async.AsyncMethodCallback<java.util.List<ParagraphInfo>> resultHandler) throws org.apache.thrift.TException;
 
+    public void getAllLibraryMetadatas(java.lang.String intpSettingName, org.apache.thrift.async.AsyncMethodCallback<java.util.List<LibraryMetadata>> resultHandler) throws org.apache.thrift.TException;
+
+    public void getLibrary(java.lang.String intpSettingName, java.lang.String libraryName, org.apache.thrift.async.AsyncMethodCallback<java.nio.ByteBuffer> resultHandler) throws org.apache.thrift.TException;
+
   }
 
   public static class Client extends org.apache.thrift.TServiceClient implements Iface {
@@ -618,6 +626,53 @@ public class RemoteInterpreterEventService {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getParagraphList failed: unknown result");
     }
 
+    public java.util.List<LibraryMetadata> getAllLibraryMetadatas(java.lang.String intpSettingName) throws org.apache.thrift.TException
+    {
+      send_getAllLibraryMetadatas(intpSettingName);
+      return recv_getAllLibraryMetadatas();
+    }
+
+    public void send_getAllLibraryMetadatas(java.lang.String intpSettingName) throws org.apache.thrift.TException
+    {
+      getAllLibraryMetadatas_args args = new getAllLibraryMetadatas_args();
+      args.setIntpSettingName(intpSettingName);
+      sendBase("getAllLibraryMetadatas", args);
+    }
+
+    public java.util.List<LibraryMetadata> recv_getAllLibraryMetadatas() throws org.apache.thrift.TException
+    {
+      getAllLibraryMetadatas_result result = new getAllLibraryMetadatas_result();
+      receiveBase(result, "getAllLibraryMetadatas");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getAllLibraryMetadatas failed: unknown result");
+    }
+
+    public java.nio.ByteBuffer getLibrary(java.lang.String intpSettingName, java.lang.String libraryName) throws org.apache.thrift.TException
+    {
+      send_getLibrary(intpSettingName, libraryName);
+      return recv_getLibrary();
+    }
+
+    public void send_getLibrary(java.lang.String intpSettingName, java.lang.String libraryName) throws org.apache.thrift.TException
+    {
+      getLibrary_args args = new getLibrary_args();
+      args.setIntpSettingName(intpSettingName);
+      args.setLibraryName(libraryName);
+      sendBase("getLibrary", args);
+    }
+
+    public java.nio.ByteBuffer recv_getLibrary() throws org.apache.thrift.TException
+    {
+      getLibrary_result result = new getLibrary_result();
+      receiveBase(result, "getLibrary");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getLibrary failed: unknown result");
+    }
+
   }
   public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
     public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
@@ -1309,6 +1364,73 @@ public class RemoteInterpreterEventService {
       }
     }
 
+    public void getAllLibraryMetadatas(java.lang.String intpSettingName, org.apache.thrift.async.AsyncMethodCallback<java.util.List<LibraryMetadata>> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getAllLibraryMetadatas_call method_call = new getAllLibraryMetadatas_call(intpSettingName, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getAllLibraryMetadatas_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<LibraryMetadata>> {
+      private java.lang.String intpSettingName;
+      public getAllLibraryMetadatas_call(java.lang.String intpSettingName, org.apache.thrift.async.AsyncMethodCallback<java.util.List<LibraryMetadata>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.intpSettingName = intpSettingName;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getAllLibraryMetadatas", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getAllLibraryMetadatas_args args = new getAllLibraryMetadatas_args();
+        args.setIntpSettingName(intpSettingName);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public java.util.List<LibraryMetadata> getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getAllLibraryMetadatas();
+      }
+    }
+
+    public void getLibrary(java.lang.String intpSettingName, java.lang.String libraryName, org.apache.thrift.async.AsyncMethodCallback<java.nio.ByteBuffer> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getLibrary_call method_call = new getLibrary_call(intpSettingName, libraryName, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getLibrary_call extends org.apache.thrift.async.TAsyncMethodCall<java.nio.ByteBuffer> {
+      private java.lang.String intpSettingName;
+      private java.lang.String libraryName;
+      public getLibrary_call(java.lang.String intpSettingName, java.lang.String libraryName, org.apache.thrift.async.AsyncMethodCallback<java.nio.ByteBuffer> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.intpSettingName = intpSettingName;
+        this.libraryName = libraryName;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getLibrary", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getLibrary_args args = new getLibrary_args();
+        args.setIntpSettingName(intpSettingName);
+        args.setLibraryName(libraryName);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public java.nio.ByteBuffer getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getLibrary();
+      }
+    }
+
   }
 
   public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
@@ -1342,6 +1464,8 @@ public class RemoteInterpreterEventService {
       processMap.put("getResource", new getResource());
       processMap.put("invokeMethod", new invokeMethod());
       processMap.put("getParagraphList", new getParagraphList());
+      processMap.put("getAllLibraryMetadatas", new getAllLibraryMetadatas());
+      processMap.put("getLibrary", new getLibrary());
       return processMap;
     }
 
@@ -1925,6 +2049,56 @@ public class RemoteInterpreterEventService {
       }
     }
 
+    public static class getAllLibraryMetadatas<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getAllLibraryMetadatas_args> {
+      public getAllLibraryMetadatas() {
+        super("getAllLibraryMetadatas");
+      }
+
+      public getAllLibraryMetadatas_args getEmptyArgsInstance() {
+        return new getAllLibraryMetadatas_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public getAllLibraryMetadatas_result getResult(I iface, getAllLibraryMetadatas_args args) throws org.apache.thrift.TException {
+        getAllLibraryMetadatas_result result = new getAllLibraryMetadatas_result();
+        result.success = iface.getAllLibraryMetadatas(args.intpSettingName);
+        return result;
+      }
+    }
+
+    public static class getLibrary<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getLibrary_args> {
+      public getLibrary() {
+        super("getLibrary");
+      }
+
+      public getLibrary_args getEmptyArgsInstance() {
+        return new getLibrary_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public getLibrary_result getResult(I iface, getLibrary_args args) throws org.apache.thrift.TException {
+        getLibrary_result result = new getLibrary_result();
+        result.success = iface.getLibrary(args.intpSettingName, args.libraryName);
+        return result;
+      }
+    }
+
   }
 
   public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
@@ -1958,6 +2132,8 @@ public class RemoteInterpreterEventService {
       processMap.put("getResource", new getResource());
       processMap.put("invokeMethod", new invokeMethod());
       processMap.put("getParagraphList", new getParagraphList());
+      processMap.put("getAllLibraryMetadatas", new getAllLibraryMetadatas());
+      processMap.put("getLibrary", new getLibrary());
       return processMap;
     }
 
@@ -3245,6 +3421,128 @@ public class RemoteInterpreterEventService {
       }
     }
 
+    public static class getAllLibraryMetadatas<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getAllLibraryMetadatas_args, java.util.List<LibraryMetadata>> {
+      public getAllLibraryMetadatas() {
+        super("getAllLibraryMetadatas");
+      }
+
+      public getAllLibraryMetadatas_args getEmptyArgsInstance() {
+        return new getAllLibraryMetadatas_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<LibraryMetadata>> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<java.util.List<LibraryMetadata>>() { 
+          public void onComplete(java.util.List<LibraryMetadata> o) {
+            getAllLibraryMetadatas_result result = new getAllLibraryMetadatas_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            getAllLibraryMetadatas_result result = new getAllLibraryMetadatas_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, getAllLibraryMetadatas_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<LibraryMetadata>> resultHandler) throws org.apache.thrift.TException {
+        iface.getAllLibraryMetadatas(args.intpSettingName,resultHandler);
+      }
+    }
+
+    public static class getLibrary<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getLibrary_args, java.nio.ByteBuffer> {
+      public getLibrary() {
+        super("getLibrary");
+      }
+
+      public getLibrary_args getEmptyArgsInstance() {
+        return new getLibrary_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<java.nio.ByteBuffer> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<java.nio.ByteBuffer>() { 
+          public void onComplete(java.nio.ByteBuffer o) {
+            getLibrary_result result = new getLibrary_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            getLibrary_result result = new getLibrary_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, getLibrary_args args, org.apache.thrift.async.AsyncMethodCallback<java.nio.ByteBuffer> resultHandler) throws org.apache.thrift.TException {
+        iface.getLibrary(args.intpSettingName, args.libraryName,resultHandler);
+      }
+    }
+
   }
 
   public static class registerInterpreterProcess_args implements org.apache.thrift.TBase<registerInterpreterProcess_args, registerInterpreterProcess_args._Fields>, java.io.Serializable, Cloneable, Comparable<registerInterpreterProcess_args>   {
@@ -19770,4 +20068,1645 @@ public class RemoteInterpreterEventService {
     }
   }
 
+  public static class getAllLibraryMetadatas_args implements org.apache.thrift.TBase<getAllLibraryMetadatas_args, getAllLibraryMetadatas_args._Fields>, java.io.Serializable, Cloneable, Comparable<getAllLibraryMetadatas_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getAllLibraryMetadatas_args");
+
+    private static final org.apache.thrift.protocol.TField INTP_SETTING_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("intpSettingName", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getAllLibraryMetadatas_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getAllLibraryMetadatas_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable java.lang.String intpSettingName; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      INTP_SETTING_NAME((short)1, "intpSettingName");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // INTP_SETTING_NAME
+            return INTP_SETTING_NAME;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.INTP_SETTING_NAME, new org.apache.thrift.meta_data.FieldMetaData("intpSettingName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getAllLibraryMetadatas_args.class, metaDataMap);
+    }
+
+    public getAllLibraryMetadatas_args() {
+    }
+
+    public getAllLibraryMetadatas_args(
+      java.lang.String intpSettingName)
+    {
+      this();
+      this.intpSettingName = intpSettingName;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getAllLibraryMetadatas_args(getAllLibraryMetadatas_args other) {
+      if (other.isSetIntpSettingName()) {
+        this.intpSettingName = other.intpSettingName;
+      }
+    }
+
+    public getAllLibraryMetadatas_args deepCopy() {
+      return new getAllLibraryMetadatas_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.intpSettingName = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.String getIntpSettingName() {
+      return this.intpSettingName;
+    }
+
+    public getAllLibraryMetadatas_args setIntpSettingName(@org.apache.thrift.annotation.Nullable java.lang.String intpSettingName) {
+      this.intpSettingName = intpSettingName;
+      return this;
+    }
+
+    public void unsetIntpSettingName() {
+      this.intpSettingName = null;
+    }
+
+    /** Returns true if field intpSettingName is set (has been assigned a value) and false otherwise */
+    public boolean isSetIntpSettingName() {
+      return this.intpSettingName != null;
+    }
+
+    public void setIntpSettingNameIsSet(boolean value) {
+      if (!value) {
+        this.intpSettingName = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case INTP_SETTING_NAME:
+        if (value == null) {
+          unsetIntpSettingName();
+        } else {
+          setIntpSettingName((java.lang.String)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case INTP_SETTING_NAME:
+        return getIntpSettingName();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case INTP_SETTING_NAME:
+        return isSetIntpSettingName();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getAllLibraryMetadatas_args)
+        return this.equals((getAllLibraryMetadatas_args)that);
+      return false;
+    }
+
+    public boolean equals(getAllLibraryMetadatas_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_intpSettingName = true && this.isSetIntpSettingName();
+      boolean that_present_intpSettingName = true && that.isSetIntpSettingName();
+      if (this_present_intpSettingName || that_present_intpSettingName) {
+        if (!(this_present_intpSettingName && that_present_intpSettingName))
+          return false;
+        if (!this.intpSettingName.equals(that.intpSettingName))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetIntpSettingName()) ? 131071 : 524287);
+      if (isSetIntpSettingName())
+        hashCode = hashCode * 8191 + intpSettingName.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getAllLibraryMetadatas_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetIntpSettingName()).compareTo(other.isSetIntpSettingName());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetIntpSettingName()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.intpSettingName, other.intpSettingName);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+    }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("getAllLibraryMetadatas_args(");
+      boolean first = true;
+
+      sb.append("intpSettingName:");
+      if (this.intpSettingName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.intpSettingName);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getAllLibraryMetadatas_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getAllLibraryMetadatas_argsStandardScheme getScheme() {
+        return new getAllLibraryMetadatas_argsStandardScheme();
+      }
+    }
+
+    private static class getAllLibraryMetadatas_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getAllLibraryMetadatas_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getAllLibraryMetadatas_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // INTP_SETTING_NAME
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.intpSettingName = iprot.readString();
+                struct.setIntpSettingNameIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getAllLibraryMetadatas_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.intpSettingName != null) {
+          oprot.writeFieldBegin(INTP_SETTING_NAME_FIELD_DESC);
+          oprot.writeString(struct.intpSettingName);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getAllLibraryMetadatas_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getAllLibraryMetadatas_argsTupleScheme getScheme() {
+        return new getAllLibraryMetadatas_argsTupleScheme();
+      }
+    }
+
+    private static class getAllLibraryMetadatas_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getAllLibraryMetadatas_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getAllLibraryMetadatas_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetIntpSettingName()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetIntpSettingName()) {
+          oprot.writeString(struct.intpSettingName);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getAllLibraryMetadatas_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.intpSettingName = iprot.readString();
+          struct.setIntpSettingNameIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
+  public static class getAllLibraryMetadatas_result implements org.apache.thrift.TBase<getAllLibraryMetadatas_result, getAllLibraryMetadatas_result._Fields>, java.io.Serializable, Cloneable, Comparable<getAllLibraryMetadatas_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getAllLibraryMetadatas_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getAllLibraryMetadatas_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getAllLibraryMetadatas_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable java.util.List<LibraryMetadata> success; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+              new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, LibraryMetadata.class))));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getAllLibraryMetadatas_result.class, metaDataMap);
+    }
+
+    public getAllLibraryMetadatas_result() {
+    }
+
+    public getAllLibraryMetadatas_result(
+      java.util.List<LibraryMetadata> success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getAllLibraryMetadatas_result(getAllLibraryMetadatas_result other) {
+      if (other.isSetSuccess()) {
+        java.util.List<LibraryMetadata> __this__success = new java.util.ArrayList<LibraryMetadata>(other.success.size());
+        for (LibraryMetadata other_element : other.success) {
+          __this__success.add(new LibraryMetadata(other_element));
+        }
+        this.success = __this__success;
+      }
+    }
+
+    public getAllLibraryMetadatas_result deepCopy() {
+      return new getAllLibraryMetadatas_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+    }
+
+    public int getSuccessSize() {
+      return (this.success == null) ? 0 : this.success.size();
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.Iterator<LibraryMetadata> getSuccessIterator() {
+      return (this.success == null) ? null : this.success.iterator();
+    }
+
+    public void addToSuccess(LibraryMetadata elem) {
+      if (this.success == null) {
+        this.success = new java.util.ArrayList<LibraryMetadata>();
+      }
+      this.success.add(elem);
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.List<LibraryMetadata> getSuccess() {
+      return this.success;
+    }
+
+    public getAllLibraryMetadatas_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.List<LibraryMetadata> success) {
+      this.success = success;
+      return this;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((java.util.List<LibraryMetadata>)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getAllLibraryMetadatas_result)
+        return this.equals((getAllLibraryMetadatas_result)that);
+      return false;
+    }
+
+    public boolean equals(getAllLibraryMetadatas_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
+      if (isSetSuccess())
+        hashCode = hashCode * 8191 + success.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getAllLibraryMetadatas_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+      }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("getAllLibraryMetadatas_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getAllLibraryMetadatas_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getAllLibraryMetadatas_resultStandardScheme getScheme() {
+        return new getAllLibraryMetadatas_resultStandardScheme();
+      }
+    }
+
+    private static class getAllLibraryMetadatas_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getAllLibraryMetadatas_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getAllLibraryMetadatas_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list50 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<LibraryMetadata>(_list50.size);
+                  @org.apache.thrift.annotation.Nullable LibraryMetadata _elem51;
+                  for (int _i52 = 0; _i52 < _list50.size; ++_i52)
+                  {
+                    _elem51 = new LibraryMetadata();
+                    _elem51.read(iprot);
+                    struct.success.add(_elem51);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getAllLibraryMetadatas_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
+            for (LibraryMetadata _iter53 : struct.success)
+            {
+              _iter53.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getAllLibraryMetadatas_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getAllLibraryMetadatas_resultTupleScheme getScheme() {
+        return new getAllLibraryMetadatas_resultTupleScheme();
+      }
+    }
+
+    private static class getAllLibraryMetadatas_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getAllLibraryMetadatas_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getAllLibraryMetadatas_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          {
+            oprot.writeI32(struct.success.size());
+            for (LibraryMetadata _iter54 : struct.success)
+            {
+              _iter54.write(oprot);
+            }
+          }
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getAllLibraryMetadatas_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          {
+            org.apache.thrift.protocol.TList _list55 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new java.util.ArrayList<LibraryMetadata>(_list55.size);
+            @org.apache.thrift.annotation.Nullable LibraryMetadata _elem56;
+            for (int _i57 = 0; _i57 < _list55.size; ++_i57)
+            {
+              _elem56 = new LibraryMetadata();
+              _elem56.read(iprot);
+              struct.success.add(_elem56);
+            }
+          }
+          struct.setSuccessIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
+  public static class getLibrary_args implements org.apache.thrift.TBase<getLibrary_args, getLibrary_args._Fields>, java.io.Serializable, Cloneable, Comparable<getLibrary_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getLibrary_args");
+
+    private static final org.apache.thrift.protocol.TField INTP_SETTING_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("intpSettingName", org.apache.thrift.protocol.TType.STRING, (short)1);
+    private static final org.apache.thrift.protocol.TField LIBRARY_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("libraryName", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getLibrary_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getLibrary_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable java.lang.String intpSettingName; // required
+    public @org.apache.thrift.annotation.Nullable java.lang.String libraryName; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      INTP_SETTING_NAME((short)1, "intpSettingName"),
+      LIBRARY_NAME((short)2, "libraryName");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // INTP_SETTING_NAME
+            return INTP_SETTING_NAME;
+          case 2: // LIBRARY_NAME
+            return LIBRARY_NAME;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.INTP_SETTING_NAME, new org.apache.thrift.meta_data.FieldMetaData("intpSettingName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.LIBRARY_NAME, new org.apache.thrift.meta_data.FieldMetaData("libraryName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getLibrary_args.class, metaDataMap);
+    }
+
+    public getLibrary_args() {
+    }
+
+    public getLibrary_args(
+      java.lang.String intpSettingName,
+      java.lang.String libraryName)
+    {
+      this();
+      this.intpSettingName = intpSettingName;
+      this.libraryName = libraryName;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getLibrary_args(getLibrary_args other) {
+      if (other.isSetIntpSettingName()) {
+        this.intpSettingName = other.intpSettingName;
+      }
+      if (other.isSetLibraryName()) {
+        this.libraryName = other.libraryName;
+      }
+    }
+
+    public getLibrary_args deepCopy() {
+      return new getLibrary_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.intpSettingName = null;
+      this.libraryName = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.String getIntpSettingName() {
+      return this.intpSettingName;
+    }
+
+    public getLibrary_args setIntpSettingName(@org.apache.thrift.annotation.Nullable java.lang.String intpSettingName) {
+      this.intpSettingName = intpSettingName;
+      return this;
+    }
+
+    public void unsetIntpSettingName() {
+      this.intpSettingName = null;
+    }
+
+    /** Returns true if field intpSettingName is set (has been assigned a value) and false otherwise */
+    public boolean isSetIntpSettingName() {
+      return this.intpSettingName != null;
+    }
+
+    public void setIntpSettingNameIsSet(boolean value) {
+      if (!value) {
+        this.intpSettingName = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.String getLibraryName() {
+      return this.libraryName;
+    }
+
+    public getLibrary_args setLibraryName(@org.apache.thrift.annotation.Nullable java.lang.String libraryName) {
+      this.libraryName = libraryName;
+      return this;
+    }
+
+    public void unsetLibraryName() {
+      this.libraryName = null;
+    }
+
+    /** Returns true if field libraryName is set (has been assigned a value) and false otherwise */
+    public boolean isSetLibraryName() {
+      return this.libraryName != null;
+    }
+
+    public void setLibraryNameIsSet(boolean value) {
+      if (!value) {
+        this.libraryName = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case INTP_SETTING_NAME:
+        if (value == null) {
+          unsetIntpSettingName();
+        } else {
+          setIntpSettingName((java.lang.String)value);
+        }
+        break;
+
+      case LIBRARY_NAME:
+        if (value == null) {
+          unsetLibraryName();
+        } else {
+          setLibraryName((java.lang.String)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case INTP_SETTING_NAME:
+        return getIntpSettingName();
+
+      case LIBRARY_NAME:
+        return getLibraryName();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case INTP_SETTING_NAME:
+        return isSetIntpSettingName();
+      case LIBRARY_NAME:
+        return isSetLibraryName();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getLibrary_args)
+        return this.equals((getLibrary_args)that);
+      return false;
+    }
+
+    public boolean equals(getLibrary_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_intpSettingName = true && this.isSetIntpSettingName();
+      boolean that_present_intpSettingName = true && that.isSetIntpSettingName();
+      if (this_present_intpSettingName || that_present_intpSettingName) {
+        if (!(this_present_intpSettingName && that_present_intpSettingName))
+          return false;
+        if (!this.intpSettingName.equals(that.intpSettingName))
+          return false;
+      }
+
+      boolean this_present_libraryName = true && this.isSetLibraryName();
+      boolean that_present_libraryName = true && that.isSetLibraryName();
+      if (this_present_libraryName || that_present_libraryName) {
+        if (!(this_present_libraryName && that_present_libraryName))
+          return false;
+        if (!this.libraryName.equals(that.libraryName))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetIntpSettingName()) ? 131071 : 524287);
+      if (isSetIntpSettingName())
+        hashCode = hashCode * 8191 + intpSettingName.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetLibraryName()) ? 131071 : 524287);
+      if (isSetLibraryName())
+        hashCode = hashCode * 8191 + libraryName.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getLibrary_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetIntpSettingName()).compareTo(other.isSetIntpSettingName());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetIntpSettingName()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.intpSettingName, other.intpSettingName);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetLibraryName()).compareTo(other.isSetLibraryName());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetLibraryName()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.libraryName, other.libraryName);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+    }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("getLibrary_args(");
+      boolean first = true;
+
+      sb.append("intpSettingName:");
+      if (this.intpSettingName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.intpSettingName);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("libraryName:");
+      if (this.libraryName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.libraryName);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getLibrary_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getLibrary_argsStandardScheme getScheme() {
+        return new getLibrary_argsStandardScheme();
+      }
+    }
+
+    private static class getLibrary_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getLibrary_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getLibrary_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // INTP_SETTING_NAME
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.intpSettingName = iprot.readString();
+                struct.setIntpSettingNameIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // LIBRARY_NAME
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.libraryName = iprot.readString();
+                struct.setLibraryNameIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getLibrary_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.intpSettingName != null) {
+          oprot.writeFieldBegin(INTP_SETTING_NAME_FIELD_DESC);
+          oprot.writeString(struct.intpSettingName);
+          oprot.writeFieldEnd();
+        }
+        if (struct.libraryName != null) {
+          oprot.writeFieldBegin(LIBRARY_NAME_FIELD_DESC);
+          oprot.writeString(struct.libraryName);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getLibrary_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getLibrary_argsTupleScheme getScheme() {
+        return new getLibrary_argsTupleScheme();
+      }
+    }
+
+    private static class getLibrary_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getLibrary_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getLibrary_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetIntpSettingName()) {
+          optionals.set(0);
+        }
+        if (struct.isSetLibraryName()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetIntpSettingName()) {
+          oprot.writeString(struct.intpSettingName);
+        }
+        if (struct.isSetLibraryName()) {
+          oprot.writeString(struct.libraryName);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getLibrary_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.intpSettingName = iprot.readString();
+          struct.setIntpSettingNameIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.libraryName = iprot.readString();
+          struct.setLibraryNameIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
+  public static class getLibrary_result implements org.apache.thrift.TBase<getLibrary_result, getLibrary_result._Fields>, java.io.Serializable, Cloneable, Comparable<getLibrary_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getLibrary_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getLibrary_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getLibrary_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer success; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getLibrary_result.class, metaDataMap);
+    }
+
+    public getLibrary_result() {
+    }
+
+    public getLibrary_result(
+      java.nio.ByteBuffer success)
+    {
+      this();
+      this.success = org.apache.thrift.TBaseHelper.copyBinary(success);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getLibrary_result(getLibrary_result other) {
+      if (other.isSetSuccess()) {
+        this.success = org.apache.thrift.TBaseHelper.copyBinary(other.success);
+      }
+    }
+
+    public getLibrary_result deepCopy() {
+      return new getLibrary_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+    }
+
+    public byte[] getSuccess() {
+      setSuccess(org.apache.thrift.TBaseHelper.rightSize(success));
+      return success == null ? null : success.array();
+    }
+
+    public java.nio.ByteBuffer bufferForSuccess() {
+      return org.apache.thrift.TBaseHelper.copyBinary(success);
+    }
+
+    public getLibrary_result setSuccess(byte[] success) {
+      this.success = success == null ? (java.nio.ByteBuffer)null     : java.nio.ByteBuffer.wrap(success.clone());
+      return this;
+    }
+
+    public getLibrary_result setSuccess(@org.apache.thrift.annotation.Nullable java.nio.ByteBuffer success) {
+      this.success = org.apache.thrift.TBaseHelper.copyBinary(success);
+      return this;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          if (value instanceof byte[]) {
+            setSuccess((byte[])value);
+          } else {
+            setSuccess((java.nio.ByteBuffer)value);
+          }
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getLibrary_result)
+        return this.equals((getLibrary_result)that);
+      return false;
+    }
+
+    public boolean equals(getLibrary_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
+      if (isSetSuccess())
+        hashCode = hashCode * 8191 + success.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getLibrary_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+      }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("getLibrary_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        org.apache.thrift.TBaseHelper.toString(this.success, sb);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getLibrary_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getLibrary_resultStandardScheme getScheme() {
+        return new getLibrary_resultStandardScheme();
+      }
+    }
+
+    private static class getLibrary_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getLibrary_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getLibrary_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.success = iprot.readBinary();
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getLibrary_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          oprot.writeBinary(struct.success);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getLibrary_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getLibrary_resultTupleScheme getScheme() {
+        return new getLibrary_resultTupleScheme();
+      }
+    }
+
+    private static class getLibrary_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getLibrary_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getLibrary_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          oprot.writeBinary(struct.success);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getLibrary_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.success = iprot.readBinary();
+          struct.setSuccessIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
 }
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterEventType.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterEventType.java
index 49961d2..292c519 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterEventType.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterEventType.java
@@ -24,7 +24,7 @@
 package org.apache.zeppelin.interpreter.thrift;
 
 
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-01-22")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-03-09")
 public enum RemoteInterpreterEventType implements org.apache.thrift.TEnum {
   NO_OP(1),
   ANGULAR_OBJECT_ADD(2),
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterResult.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterResult.java
index 098f8f2..4fce5b0 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterResult.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterResult.java
@@ -24,7 +24,7 @@
 package org.apache.zeppelin.interpreter.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-01-22")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-03-09")
 public class RemoteInterpreterResult implements org.apache.thrift.TBase<RemoteInterpreterResult, RemoteInterpreterResult._Fields>, java.io.Serializable, Cloneable, Comparable<RemoteInterpreterResult> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RemoteInterpreterResult");
 
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterResultMessage.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterResultMessage.java
index 6ca2f1e..6ea6fa6 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterResultMessage.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterResultMessage.java
@@ -24,7 +24,7 @@
 package org.apache.zeppelin.interpreter.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-01-22")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-03-09")
 public class RemoteInterpreterResultMessage implements org.apache.thrift.TBase<RemoteInterpreterResultMessage, RemoteInterpreterResultMessage._Fields>, java.io.Serializable, Cloneable, Comparable<RemoteInterpreterResultMessage> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RemoteInterpreterResultMessage");
 
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterService.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterService.java
index d206b1b..4cde441 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterService.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterService.java
@@ -24,7 +24,7 @@
 package org.apache.zeppelin.interpreter.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-01-22")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-03-09")
 public class RemoteInterpreterService {
 
   public interface Iface {
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RunParagraphsEvent.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RunParagraphsEvent.java
index b128cf3..47edc59 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RunParagraphsEvent.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RunParagraphsEvent.java
@@ -24,7 +24,7 @@
 package org.apache.zeppelin.interpreter.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-01-22")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-03-09")
 public class RunParagraphsEvent implements org.apache.thrift.TBase<RunParagraphsEvent, RunParagraphsEvent._Fields>, java.io.Serializable, Cloneable, Comparable<RunParagraphsEvent> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RunParagraphsEvent");
 
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/ServiceException.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/ServiceException.java
index 5fc6739..1882f45 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/ServiceException.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/ServiceException.java
@@ -24,7 +24,7 @@
 package org.apache.zeppelin.interpreter.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-01-22")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-03-09")
 public class ServiceException extends org.apache.thrift.TException implements org.apache.thrift.TBase<ServiceException, ServiceException._Fields>, java.io.Serializable, Cloneable, Comparable<ServiceException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ServiceException");
 
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/WebUrlInfo.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/WebUrlInfo.java
index 2877f4e..a622c0d 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/WebUrlInfo.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/WebUrlInfo.java
@@ -24,7 +24,7 @@
 package org.apache.zeppelin.interpreter.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-01-22")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2021-03-09")
 public class WebUrlInfo implements org.apache.thrift.TBase<WebUrlInfo, WebUrlInfo._Fields>, java.io.Serializable, Cloneable, Comparable<WebUrlInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("WebUrlInfo");
 
diff --git a/zeppelin-interpreter/src/main/thrift/RemoteInterpreterEventService.thrift b/zeppelin-interpreter/src/main/thrift/RemoteInterpreterEventService.thrift
index 3be799a..ad08821 100644
--- a/zeppelin-interpreter/src/main/thrift/RemoteInterpreterEventService.thrift
+++ b/zeppelin-interpreter/src/main/thrift/RemoteInterpreterEventService.thrift
@@ -98,6 +98,12 @@ struct ParagraphInfo {
   4: string paragraphText
 }
 
+// The metadata of a file
+struct LibraryMetadata {
+   1: required string name;
+   2: required i64 checksum;
+}
+
 exception ServiceException{
   1: required string message;
 }
@@ -131,4 +137,7 @@ service RemoteInterpreterEventService {
   binary invokeMethod(1: string intpGroupId, 2: string invokeMethodJson) throws (1: RemoteInterpreterService.InterpreterRPCException ex);
 
   list<ParagraphInfo> getParagraphList(1: string user, 2: string noteId) throws (1: RemoteInterpreterService.InterpreterRPCException ex);
-}
\ No newline at end of file
+
+  list<LibraryMetadata> getAllLibraryMetadatas(1: string intpSettingName);
+  binary getLibrary(1: string intpSettingName, 2: string libraryName);
+}
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/RemoteInterpreterEventServer.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/RemoteInterpreterEventServer.java
index e2ccdca..7aa6d6c 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/RemoteInterpreterEventServer.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/RemoteInterpreterEventServer.java
@@ -19,6 +19,9 @@ package org.apache.zeppelin.interpreter;
 
 import com.google.gson.Gson;
 import com.google.gson.reflect.TypeToken;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.thrift.TException;
 import org.apache.thrift.server.TThreadPoolServer;
 import org.apache.thrift.transport.TServerSocket;
@@ -36,6 +39,7 @@ import org.apache.zeppelin.interpreter.thrift.AppOutputAppendEvent;
 import org.apache.zeppelin.interpreter.thrift.AppOutputUpdateEvent;
 import org.apache.zeppelin.interpreter.thrift.AppStatusUpdateEvent;
 import org.apache.zeppelin.interpreter.thrift.InterpreterRPCException;
+import org.apache.zeppelin.interpreter.thrift.LibraryMetadata;
 import org.apache.zeppelin.interpreter.thrift.ParagraphInfo;
 import org.apache.zeppelin.interpreter.thrift.RegisterInfo;
 import org.apache.zeppelin.interpreter.thrift.OutputAppendEvent;
@@ -44,7 +48,6 @@ import org.apache.zeppelin.interpreter.thrift.OutputUpdateEvent;
 import org.apache.zeppelin.interpreter.thrift.RemoteInterpreterEventService;
 import org.apache.zeppelin.interpreter.thrift.RemoteInterpreterResultMessage;
 import org.apache.zeppelin.interpreter.thrift.RunParagraphsEvent;
-import org.apache.zeppelin.interpreter.thrift.ServiceException;
 import org.apache.zeppelin.interpreter.thrift.WebUrlInfo;
 import org.apache.zeppelin.notebook.Note;
 import org.apache.zeppelin.resource.RemoteResource;
@@ -56,8 +59,11 @@ import org.apache.zeppelin.user.AuthenticationInfo;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.LinkedList;
 import java.util.List;
@@ -557,4 +563,58 @@ public class RemoteInterpreterEventServer implements RemoteInterpreterEventServi
       LOGGER.error("Fail to updateParagraphConfig", e);
     }
   }
+
+  @Override
+  public List<LibraryMetadata> getAllLibraryMetadatas(String interpreter) throws TException {
+    if (StringUtils.isBlank(interpreter)) {
+      LOGGER.warn("Interpreter is blank");
+      return Collections.emptyList();
+    }
+    File interpreterLocalRepo = new File(
+        zConf.getAbsoluteDir(ZeppelinConfiguration.ConfVars.ZEPPELIN_DEP_LOCALREPO)
+            + File.separator
+            + interpreter);
+    if (!interpreterLocalRepo.exists()) {
+      LOGGER.warn("Local interpreter repository {} for interpreter {} doesn't exists", interpreterLocalRepo,
+          interpreter);
+      return Collections.emptyList();
+    }
+    if (!interpreterLocalRepo.isDirectory()) {
+      LOGGER.warn("Local interpreter repository {} is no folder", interpreterLocalRepo);
+      return Collections.emptyList();
+    }
+    Collection<File> files = FileUtils.listFiles(interpreterLocalRepo, new String[] { "jar" }, false);
+    List<LibraryMetadata> metaDatas = new ArrayList<>(files.size());
+    for (File file : files) {
+      try {
+        metaDatas.add(new LibraryMetadata(file.getName(), FileUtils.checksumCRC32(file)));
+      } catch (IOException e) {
+        LOGGER.warn(e.getMessage(), e);
+      }
+    }
+    return metaDatas;
+  }
+
+
+  @Override
+  public ByteBuffer getLibrary(String interpreter, String libraryName) throws TException {
+    if (StringUtils.isAnyBlank(interpreter, libraryName)) {
+      LOGGER.warn("Interpreter \"{}\" or libraryName \"{}\" is blank", interpreter, libraryName);
+      return null;
+    }
+    File library = new File(zConf.getAbsoluteDir(ZeppelinConfiguration.ConfVars.ZEPPELIN_DEP_LOCALREPO)
+        + File.separator + interpreter + File.separator + libraryName);
+    if (!library.exists()) {
+      LOGGER.warn("Library {} doesn't exists", library);
+      return null;
+    }
+
+    try {
+      return ByteBuffer.wrap(FileUtils.readFileToByteArray(library));
+    } catch (IOException e) {
+      LOGGER.error("Unable to read library {}", library, e);
+    }
+    return null;
+  }
+
 }