You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@knox.apache.org by GitBox <gi...@apache.org> on 2020/03/19 17:52:36 UTC

[GitHub] [knox] lmccay opened a new pull request #296: Knox 2240

lmccay opened a new pull request #296: Knox 2240
URL: https://github.com/apache/knox/pull/296
 
 
   (It is very **important** that you created an Apache Knox JIRA for this change and that the PR title/commit message includes the Apache Knox JIRA ID!)
   
   ## What changes were proposed in this pull request?
   
   This PR introduces a new custom groovy command for KnoxShell for interaction with hadoop filesystems through the WebHDFS API. This includes the ability to mount filesystems, unmount them and list mounts. Mounts are persisted in the user's home directory for knoxshell so that they live across KnoxShell sessions.
   
   It is limited to the hdfs client classes that are provided in KnoxShell today but with follow up patches to both the client classes and the command will cover more of the Hadoop filesystem capabilities - such as chown and chmod.
   
   ## How was this patch tested?
   
   Primarily tested through manual use of the KnoxShell environment. Minor improvements in CSV builder have added unit tests as well.
   
   Please review [Knox Contributing Process](https://cwiki.apache.org/confluence/display/KNOX/Contribution+Process#ContributionProcess-GithubWorkflow) before opening a pull request.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [knox] smolnar82 commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use
URL: https://github.com/apache/knox/pull/296#discussion_r396264134
 
 

 ##########
 File path: gateway-shell/src/main/java/org/apache/knox/gateway/shell/commands/WebHDFSCommand.java
 ##########
 @@ -0,0 +1,376 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.knox.gateway.shell.commands;
+
+import java.io.Console;
+import java.io.File;
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.TimeZone;
+
+import org.apache.knox.gateway.shell.CredentialCollectionException;
+import org.apache.knox.gateway.shell.CredentialCollector;
+import org.apache.knox.gateway.shell.KnoxSession;
+import org.apache.knox.gateway.shell.KnoxShellException;
+import org.apache.knox.gateway.shell.hdfs.Hdfs;
+import org.apache.knox.gateway.shell.hdfs.Status.Response;
+import org.apache.knox.gateway.shell.table.KnoxShellTable;
+import org.apache.knox.gateway.util.JsonUtils;
+import org.codehaus.groovy.tools.shell.Groovysh;
+
+public class WebHDFSCommand extends AbstractKnoxShellCommand {
+  private static final String KNOXMOUNTPOINTS = "__knoxmountpoints";
+  private Map<String, KnoxSession> sessions = new HashMap<>();
 
 Review comment:
   nit: make it final?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [knox] smolnar82 commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use
URL: https://github.com/apache/knox/pull/296#discussion_r396269512
 
 

 ##########
 File path: gateway-shell/src/main/java/org/apache/knox/gateway/shell/table/KnoxShellTable.java
 ##########
 @@ -137,6 +137,10 @@ else if (colIndex instanceof String) {
     double[] colArray = new double[col.size()];
     Conversions conversionMethod = null;
     for (int i = 0; i < col.size(); i++) {
+      Object v = col.get(i);
+      if (v instanceof String && ((String) v).trim().isEmpty()) {
+        col.set(i, "0");
 
 Review comment:
   Why `0` is the default in case the column has no value? Reading the PR description I know you are expecting numbers in these columns, but it could be any kind of data in other scenarios `DATE`, `TEXT`,...
   What if - to make it more similar to how relational DBs look like - you use `NULL`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [knox] smolnar82 commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use
URL: https://github.com/apache/knox/pull/296#discussion_r396267407
 
 

 ##########
 File path: gateway-shell/src/main/java/org/apache/knox/gateway/shell/commands/WebHDFSCommand.java
 ##########
 @@ -0,0 +1,376 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.knox.gateway.shell.commands;
+
+import java.io.Console;
+import java.io.File;
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.TimeZone;
+
+import org.apache.knox.gateway.shell.CredentialCollectionException;
+import org.apache.knox.gateway.shell.CredentialCollector;
+import org.apache.knox.gateway.shell.KnoxSession;
+import org.apache.knox.gateway.shell.KnoxShellException;
+import org.apache.knox.gateway.shell.hdfs.Hdfs;
+import org.apache.knox.gateway.shell.hdfs.Status.Response;
+import org.apache.knox.gateway.shell.table.KnoxShellTable;
+import org.apache.knox.gateway.util.JsonUtils;
+import org.codehaus.groovy.tools.shell.Groovysh;
+
+public class WebHDFSCommand extends AbstractKnoxShellCommand {
+  private static final String KNOXMOUNTPOINTS = "__knoxmountpoints";
+  private Map<String, KnoxSession> sessions = new HashMap<>();
+
+  public WebHDFSCommand(Groovysh shell) {
+    super(shell, ":filesystem", ":fs");
+  }
+
+  @Override
+  public String getUsage() {
+    String usage = "Usage: \n" +
+                   "  :fs ls {target-path} \n" +
+                   "  :fs cat {target-path} \n" +
+                   "  :fs get {from-path} {to-path} \n" +
+                   "  :fs put {from-path} {tp-path} \n" +
+                   "  :fs rm {target-path} \n" +
+                   "  :fs mkdir {dir-path} \n";
+    return usage;
+  }
+
+  @Override
+  public Object execute(List<String> args) {
+    Map<String, String> mounts = getMountPoints();
+    if (args.isEmpty()) {
+      args.add("ls");
+    }
+    if (args.get(0).equalsIgnoreCase("mount")) {
+      String url = args.get(1);
+      String mountPoint = args.get(2);
+      KnoxSession session = establishSession(mountPoint, url);
+      if (session != null) {
+        mounts.put(mountPoint, url);
+        KnoxSession.persistMountPoints(mounts);
+        return url + " mounted as " + mountPoint;
+      }
+
+      return "Failed to mount " + url + " as " + mountPoint;
+    }
+    else if (args.get(0).equalsIgnoreCase("unmount")) {
+      String mountPoint = args.get(1);
+      sessions.remove(mountPoint);
+      mounts.remove(mountPoint);
+      KnoxSession.persistMountPoints(mounts);
+    }
+    else if (args.get(0).equalsIgnoreCase("mounts")) {
+      KnoxShellTable table = new KnoxShellTable();
+      table.header("Mount Point").header("Topology URL");
+      for (String mountPoint : mounts.keySet()) {
+        table.row().value(mountPoint).value(mounts.get(mountPoint));
+      }
+      return table;
+    }
+    else if (args.get(0).equalsIgnoreCase("ls")) {
+      String path = args.get(1);
+      try {
+        String directory;
+        String mountPoint = determineMountPoint(path);
+        if (mountPoint != null) {
+          KnoxSession session = getSessionForMountPoint(mounts, mountPoint);
+          if (session != null) {
+            directory = determineTargetPath(path, mountPoint);
+            String json = Hdfs.ls(session).dir(directory).now().getString();
+            Map<String,HashMap<String, ArrayList<HashMap<String, String>>>> map =
+                JsonUtils.getFileStatusesAsMap(json);
+            if (map != null) {
+              ArrayList<HashMap<String, String>> list = map.get("FileStatuses").get("FileStatus");
+              KnoxShellTable table = buildTableFromListStatus(directory, list);
+              return table;
+            }
+          }
+          else {
+            return "No session established for mountPoint: " + mountPoint + " Use :fs mount {topology-url} {mountpoint-name}";
+          }
+        }
+        else {
+          System.out.println("No mountpoint found. Use ':fs mount {topologyURL} {mountpoint}'.");
+        }
+      } catch (KnoxShellException | IOException e) {
+        e.printStackTrace();
+      }
+    }
+    else if (args.get(0).equalsIgnoreCase("put")) {
+      // Hdfs.put( session ).file( dataFile ).to( dataDir + "/" + dataFile ).now()
+      // :fs put from-path to-path
+      String localFile = args.get(1);
+      String path = args.get(2);
+
+      String mountPoint = determineMountPoint(path);
+      KnoxSession session = getSessionForMountPoint(mounts, mountPoint);
+      if (session != null) {
+        String targetPath = determineTargetPath(path, mountPoint);
+        try {
+          boolean overwrite = false;
+          try {
+            Response response = Hdfs.status(session).file(targetPath).now();
+            if (response.exists()) {
+              if (collectClearInput(targetPath + " already exists would you like to overwrite (Y/n)").equalsIgnoreCase("y")) {
+                overwrite = true;
+              }
+            }
+          } catch (KnoxShellException e) {
+            // NOP
+          }
+          int permission = 755;
+          if (args.size() >= 4) {
+            permission = Integer.parseInt(args.get(3));
+          }
+          Hdfs.put(session).file(localFile).to(targetPath).overwrite(overwrite).permission(permission).now().getString();
+        } catch (IOException e) {
+          e.printStackTrace();
+        }
+      }
+      else {
+        return "No session established for mountPoint: " + mountPoint + " Use :fs mount {topology-url} {mountpoint-name}";
+      }
+    }
+    else if (args.get(0).equalsIgnoreCase("rm")) {
+      // Hdfs.rm( session ).file( dataFile ).now()
+      // :fs rm target-path
+      String path = args.get(1);
+
+      String mountPoint = determineMountPoint(path);
+      KnoxSession session = getSessionForMountPoint(mounts, mountPoint);
+      if (session != null) {
+        String targetPath = determineTargetPath(path, mountPoint);
+        try {
+          Hdfs.rm(session).file(targetPath).now().getString();
+        } catch (KnoxShellException | IOException e) {
+          e.printStackTrace();
+        }
+      }
+      else {
+        return "No session established for mountPoint: " + mountPoint + " Use :fs mount {topology-url} {mountpoint-name}";
+      }
+    }
+    else if (args.get(0).equalsIgnoreCase("cat")) {
+      // println Hdfs.get( session ).from( dataDir + "/" + dataFile ).now().string
+      // :fs cat target-path
+      String path = args.get(1);
+
+      String mountPoint = determineMountPoint(path);
+      KnoxSession session = getSessionForMountPoint(mounts, mountPoint);
+      if (session != null) {
+        String targetPath = determineTargetPath(path, mountPoint);
+        try {
+          String contents = Hdfs.get(session).from(targetPath).now().getString();
+          return contents;
+        } catch (KnoxShellException | IOException e) {
+          e.printStackTrace();
+        }
+      }
+      else {
+        return "No session established for mountPoint: " + mountPoint + " Use :fs mount {topology-url} {mountpoint-name}";
+      }
+    }
+    else if (args.get(0).equalsIgnoreCase("mkdir")) {
+      // println Hdfs.mkdir( session ).dir( directoryPath ).perm( "777" ).now().string
+      // :fs mkdir target-path [perms]
+      String path = args.get(1);
+      String perms = null;
+      if (args.size() == 3) {
+        perms = args.get(2);
+      }
+
+      String mountPoint = determineMountPoint(path);
+      KnoxSession session = getSessionForMountPoint(mounts, mountPoint);
+      if (session != null) {
+        String targetPath = determineTargetPath(path, mountPoint);
+        try {
+          if (perms != null) {
+            Hdfs.mkdir(sessions.get(mountPoint)).dir(targetPath).now().getString();
+          }
+          else {
+            Hdfs.mkdir(session).dir(targetPath).perm(perms).now().getString();
+          }
+          return "Successfully created directory: " + targetPath;
+        } catch (KnoxShellException | IOException e) {
+          e.printStackTrace();
+        }
+      }
+      else {
+        return "No session established for mountPoint: " + mountPoint + " Use :fs mount {topology-url} {mountpoint-name}";
+      }
+    }
+    else if (args.get(0).equalsIgnoreCase("get")) {
+      // println Hdfs.get( session ).from( dataDir + "/" + dataFile ).now().string
+      // :fs get from-path [to-path]
+      String path = args.get(1);
+
+      String mountPoint = determineMountPoint(path);
+      KnoxSession session = getSessionForMountPoint(mounts, mountPoint);
+      if (session != null) {
+        String from = determineTargetPath(path, mountPoint);
+        String to = null;
+        if (args.size() > 2) {
+          to = args.get(2);
+        }
+        else {
+          to = System.getProperty("user.home") + File.separator +
+              path.substring(path.lastIndexOf(File.separator));
+        }
+        try {
+          Hdfs.get(sessions.get(mountPoint)).from(from).file(to).now().getString();
+        } catch (KnoxShellException | IOException e) {
+          e.printStackTrace();
+        }
+      }
+      else {
+        return "No session established for mountPoint: " + mountPoint + " Use :fs mount {topology-url} {mountpoint-name}";
+      }
+    }
+    else {
+      System.out.println("Unknown filesystem command");
+      System.out.println(getUsage());
+    }
+    return "";
+  }
+
+  private KnoxSession getSessionForMountPoint(Map<String, String> mounts, String mountPoint) {
+    KnoxSession session = sessions.get(mountPoint);
+    if (session == null) {
+      String url = mounts.get(mountPoint);
+      if (url != null) {
+        session = establishSession(mountPoint, url);
+      }
+    }
+    return session;
+  }
+
+  private KnoxSession establishSession(String mountPoint, String url) {
+    CredentialCollector dlg;
+    try {
+      dlg = login();
+    } catch (CredentialCollectionException e) {
+      e.printStackTrace();
+      return null;
+    }
+    String username = dlg.name();
+    String password = new String(dlg.chars());
+    KnoxSession session = null;
+    try {
+      session = KnoxSession.login(url, username, password);
+      sessions.put(mountPoint, session);
+    } catch (URISyntaxException e) {
+      e.printStackTrace();
+    }
+    return session;
+  }
+
+  private String collectClearInput(String prompt) {
+    Console c = System.console();
+    if (c == null) {
+      System.err.println("No console.");
+      System.exit(1);
+    }
+
+    String value = c.readLine(prompt);
+
+    return value;
+  }
+
+  private String determineTargetPath(String path, String mountPoint) {
+    String directory = null;
+    if (path.startsWith("/")) {
+      directory = stripMountPoint(path, mountPoint);
+    }
+    return directory;
+  }
+
+  private String stripMountPoint(String path, String mountPoint) {
+    String newPath = path.replace("/" + mountPoint, "");
+    return newPath;
+  }
+
+  private String determineMountPoint(String path) {
+    String mountPoint = null;
+    if (path.startsWith("/")) {
+      // does the user supplied path starts at a root
+      // if so check for a mountPoint based on the first element of the path
+      String[] pathElements = path.split("/");
+      mountPoint = pathElements[1];
+    }
+    return mountPoint;
+  }
+
+  private KnoxShellTable buildTableFromListStatus(String directory, List<HashMap<String, String>> list) {
+    Calendar cal = Calendar.getInstance(TimeZone.getDefault(), Locale.getDefault());
+    KnoxShellTable table = new KnoxShellTable();
+    table.title(directory);
+    table.header("permission")
+      .header("owner")
+      .header("group")
+      .header("length")
+      .header("modtime")
+      .header("name");
+
+    for (Map<String, String> map : list) {
+      cal.setTimeInMillis(Long.parseLong(map.get("modificationTime")));
+      table.row()
+        .value(map.get("permission"))
+        .value(map.get("owner"))
+        .value(map.get("group"))
+        .value(map.get("length"))
+        .value(cal.getTime())
+        .value(map.get("pathSuffix"));
+    }
+
+    return table;
+  }
+
+  @SuppressWarnings("unchecked")
+  protected Map<String, String> getMountPoints() {
+    Map<String, String> mounts = (Map<String, String>) getVariables().get(KNOXMOUNTPOINTS);
+    if (mounts == null) {
+      try {
+        mounts = KnoxSession.loadMountPoints();
+      } catch (IOException e) {
+        e.printStackTrace();
+      }
+      if (mounts != null) {
+        getVariables().put(KNOXMOUNTPOINTS, mounts);
+      }
+      else {
+        mounts = new HashMap<>();
+      }
+    }
+    return mounts;
+  }
+
+  public static void main(String[] args) {
+    WebHDFSCommand cmd = new WebHDFSCommand(new Groovysh());
+    List<String> args2 = new ArrayList<>();
+    cmd.execute(args2);
 
 Review comment:
   `args` is not passed. You may want to change it to:
   ```
   cmd.execute(new ArrayList<>(Arrays.asList(args)));
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [knox] smolnar82 commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use
URL: https://github.com/apache/knox/pull/296#discussion_r396255497
 
 

 ##########
 File path: gateway-shell/src/main/java/org/apache/knox/gateway/shell/KnoxSession.java
 ##########
 @@ -661,6 +676,20 @@ public static void persistDataSources(Map<String, KnoxDataSource> datasources) {
     return sqlHistories;
   }
 
+  public static Map<String, String> loadMountPoints() throws IOException {
+    Map<String, String> mounts = null;
 
 Review comment:
   To avoid NPEs I'd initialize it as an empty map (no null check is needed when this is invoked)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [knox] lmccay commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use

Posted by GitBox <gi...@apache.org>.
lmccay commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use
URL: https://github.com/apache/knox/pull/296#discussion_r396627487
 
 

 ##########
 File path: gateway-shell/src/main/java/org/apache/knox/gateway/shell/table/KnoxShellTable.java
 ##########
 @@ -137,6 +137,10 @@ else if (colIndex instanceof String) {
     double[] colArray = new double[col.size()];
     Conversions conversionMethod = null;
     for (int i = 0; i < col.size(); i++) {
+      Object v = col.get(i);
+      if (v instanceof String && ((String) v).trim().isEmpty()) {
+        col.set(i, "0");
 
 Review comment:
   It is explicitly expecting numeric data based on the use of this method. Null and empty string need to be considered 0 in these cases for the conversion to make sense.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [knox] smolnar82 commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use
URL: https://github.com/apache/knox/pull/296#discussion_r396263931
 
 

 ##########
 File path: gateway-shell/src/main/java/org/apache/knox/gateway/shell/commands/CSVCommand.java
 ##########
 @@ -49,10 +49,24 @@ public Object execute(List<String> args) {
 
     try {
       if (withHeaders) {
-        table = KnoxShellTable.builder().csv().withHeaders().url(url);
+        if (url.startsWith("$")) {
+          // a knoxshell variable is a csv file as a string
+          String csvString = (String) getVariables().get(url.substring(1));
+          table = KnoxShellTable.builder().csv().withHeaders().string(csvString);
+        }
+        else {
+          table = KnoxShellTable.builder().csv().withHeaders().url(url);
+        }
       }
       else {
-        table = KnoxShellTable.builder().csv().url(url);
+        if (url.startsWith("$")) {
+          // a knoxshell variable is a csv file as a string
+          String csvString = (String) getVariables().get(url.substring(1));
+          table = KnoxShellTable.builder().csv().string(csvString);
+        }
+        else {
+          table = KnoxShellTable.builder().csv().url(url);
+        }
 
 Review comment:
   Thes line can be replaced like:
   ```
       CSVKnoxShellTableBuilder csvTableBuilder = withHeaders ?  KnoxShellTable.builder().csv() :  KnoxShellTable.builder().csv().withHeaders();
       table = url.startsWith("$") ?  csvTableBuilder.env(url) ?  csvTableBuilder.url(url);
   ```
   Please note that `env(String variableName)` should be added in `CSVKnoxShellTableBuilder`.
   
   IMO the implementation is cleaner this way.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [knox] smolnar82 commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use
URL: https://github.com/apache/knox/pull/296#discussion_r396268208
 
 

 ##########
 File path: gateway-shell/src/main/java/org/apache/knox/gateway/shell/table/CSVKnoxShellTableBuilder.java
 ##########
 @@ -39,33 +41,47 @@ public CSVKnoxShellTableBuilder withHeaders() {
   }
 
   public KnoxShellTable url(String url) throws IOException {
-    int rowIndex = 0;
     URL urlToCsv = new URL(url);
     URLConnection connection = urlToCsv.openConnection();
     try (Reader urlConnectionStreamReader = new InputStreamReader(connection.getInputStream(), StandardCharsets.UTF_8);
         BufferedReader csvReader = new BufferedReader(urlConnectionStreamReader);) {
-      if (title != null) {
-        this.table.title(title);
+      buildTableFromCSVReader(csvReader);
+    }
+    return this.table;
+  }
+
+  public KnoxShellTable string(String csvString) throws IOException {
+    InputStream is = new ByteArrayInputStream(csvString.getBytes(StandardCharsets.UTF_8));
 
 Review comment:
   Why not move this into the `try` block so that `is` will be closed too.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [knox] lmccay commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use

Posted by GitBox <gi...@apache.org>.
lmccay commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use
URL: https://github.com/apache/knox/pull/296#discussion_r396623893
 
 

 ##########
 File path: gateway-shell/src/main/java/org/apache/knox/gateway/shell/KnoxSession.java
 ##########
 @@ -705,6 +734,15 @@ private static String readFileToString(File file) throws IOException {
     return obj;
   }
 
+  public static <T> Map<String, T> getMapFromJsonString(String json) throws IOException {
+    Map<String, T> obj;
+    JsonFactory factory = new JsonFactory();
+    ObjectMapper mapper = new ObjectMapper(factory);
+    TypeReference<Map<String, T>> typeRef = new TypeReference<Map<String, T>>() {};
+    obj = mapper.readValue(json, typeRef);
+    return obj;
+  }
+
 
 Review comment:
   I'll check why I didn't use that - there was a reason. :)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [knox] smolnar82 commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use
URL: https://github.com/apache/knox/pull/296#discussion_r396255177
 
 

 ##########
 File path: gateway-shell/src/main/java/org/apache/knox/gateway/shell/KnoxSession.java
 ##########
 @@ -661,6 +676,20 @@ public static void persistDataSources(Map<String, KnoxDataSource> datasources) {
     return sqlHistories;
   }
 
+  public static Map<String, String> loadMountPoints() throws IOException {
+    Map<String, String> mounts = null;
+    String home = System.getProperty("user.home");
+
+    File mountFile = new File(
+        home + File.separator +
+        ".knoxshell" + File.separator + KNOXMOUNTPOINTS_JSON);
 
 Review comment:
   I'd recommend using `https://docs.oracle.com/javase/8/docs/api/java/nio/file/Paths.html#get-java.lang.String-java.lang.String...-`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [knox] lmccay commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use

Posted by GitBox <gi...@apache.org>.
lmccay commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use
URL: https://github.com/apache/knox/pull/296#discussion_r396625035
 
 

 ##########
 File path: gateway-shell/src/main/java/org/apache/knox/gateway/shell/commands/CSVCommand.java
 ##########
 @@ -49,10 +49,24 @@ public Object execute(List<String> args) {
 
     try {
       if (withHeaders) {
-        table = KnoxShellTable.builder().csv().withHeaders().url(url);
+        if (url.startsWith("$")) {
+          // a knoxshell variable is a csv file as a string
+          String csvString = (String) getVariables().get(url.substring(1));
+          table = KnoxShellTable.builder().csv().withHeaders().string(csvString);
+        }
+        else {
+          table = KnoxShellTable.builder().csv().withHeaders().url(url);
+        }
       }
       else {
-        table = KnoxShellTable.builder().csv().url(url);
+        if (url.startsWith("$")) {
+          // a knoxshell variable is a csv file as a string
+          String csvString = (String) getVariables().get(url.substring(1));
+          table = KnoxShellTable.builder().csv().string(csvString);
+        }
+        else {
+          table = KnoxShellTable.builder().csv().url(url);
+        }
 
 Review comment:
   I don't want details of the groovysh leaking into the generic CSV builder code. The custom commands provide an abstraction of that layer and isolates it from the lower level classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [knox] lmccay commented on issue #296: Knox 2240

Posted by GitBox <gi...@apache.org>.
lmccay commented on issue #296: Knox 2240
URL: https://github.com/apache/knox/pull/296#issuecomment-601389278
 
 
   Broke some unit tests - looking into it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [knox] smolnar82 commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use
URL: https://github.com/apache/knox/pull/296#discussion_r396257126
 
 

 ##########
 File path: gateway-shell/src/main/java/org/apache/knox/gateway/shell/KnoxSession.java
 ##########
 @@ -705,6 +734,15 @@ private static String readFileToString(File file) throws IOException {
     return obj;
   }
 
+  public static <T> Map<String, T> getMapFromJsonString(String json) throws IOException {
+    Map<String, T> obj;
+    JsonFactory factory = new JsonFactory();
+    ObjectMapper mapper = new ObjectMapper(factory);
+    TypeReference<Map<String, T>> typeRef = new TypeReference<Map<String, T>>() {};
+    obj = mapper.readValue(json, typeRef);
+    return obj;
+  }
+
 
 Review comment:
   Knox already provides this utility function: `org.apache.knox.gateway.util.JsonUtils.getMapFromJsonString(String)`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [knox] lmccay merged pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use

Posted by GitBox <gi...@apache.org>.
lmccay merged pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use
URL: https://github.com/apache/knox/pull/296
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [knox] lmccay commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use

Posted by GitBox <gi...@apache.org>.
lmccay commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use
URL: https://github.com/apache/knox/pull/296#discussion_r396623599
 
 

 ##########
 File path: gateway-shell/src/main/java/org/apache/knox/gateway/shell/KnoxSession.java
 ##########
 @@ -661,6 +676,20 @@ public static void persistDataSources(Map<String, KnoxDataSource> datasources) {
     return sqlHistories;
   }
 
+  public static Map<String, String> loadMountPoints() throws IOException {
+    Map<String, String> mounts = null;
+    String home = System.getProperty("user.home");
+
+    File mountFile = new File(
+        home + File.separator +
+        ".knoxshell" + File.separator + KNOXMOUNTPOINTS_JSON);
 
 Review comment:
   This works fine as is.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [knox] lmccay commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use

Posted by GitBox <gi...@apache.org>.
lmccay commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use
URL: https://github.com/apache/knox/pull/296#discussion_r396696175
 
 

 ##########
 File path: gateway-shell/src/main/java/org/apache/knox/gateway/shell/KnoxSession.java
 ##########
 @@ -705,6 +734,15 @@ private static String readFileToString(File file) throws IOException {
     return obj;
   }
 
+  public static <T> Map<String, T> getMapFromJsonString(String json) throws IOException {
+    Map<String, T> obj;
+    JsonFactory factory = new JsonFactory();
+    ObjectMapper mapper = new ObjectMapper(factory);
+    TypeReference<Map<String, T>> typeRef = new TypeReference<Map<String, T>>() {};
+    obj = mapper.readValue(json, typeRef);
+    return obj;
+  }
+
 
 Review comment:
   Okay - the reason that I did that was to bubble up the exception in the groovysh and scripting environments. Rather than swallowing it and logging we allow it to be handled directly or at least communicated via stacktrace in the console. This is not acceptable for other uses of that util method in the gateway. At somepoint, a gateway-shell specific JSONUtils may be required. We can visit that later.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [knox] smolnar82 commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use
URL: https://github.com/apache/knox/pull/296#discussion_r396274308
 
 

 ##########
 File path: gateway-util-common/src/test/java/org/apache/knox/gateway/util/JsonUtilsTest.java
 ##########
 @@ -54,4 +55,46 @@ public void testGetMapFromString() {
     assertEquals("Bearer", map.get("token_type"));
     assertEquals("1364487943100", map.get("expires_in"));
   }
+
+  @Test
+  public void testFileStatusesAsMap() {
+    String json = "{\n" +
+      "   \"FileStatuses\":{\n" +
+      "      \"FileStatus\":[\n" +
+      "         {\n" +
+      "            \"accessTime\":0,\n" +
+      "            \"blockSize\":0,\n" +
+      "            \"childrenNum\":3,\n" +
+      "            \"fileId\":16389,\n" +
+      "            \"group\":\"supergroup\",\n" +
+      "            \"length\":0,\n" +
+      "            \"modificationTime\":1581578495905,\n" +
+      "            \"owner\":\"hdfs\",\n" +
+      "            \"pathSuffix\":\"tmp\",\n" +
+      "            \"permission\":\"1777\",\n" +
+      "            \"replication\":0,\n" +
+      "            \"storagePolicy\":0,\n" +
+      "            \"type\":\"DIRECTORY\"\n" +
+      "         },\n" +
+      "         {\n" +
+      "            \"accessTime\":0,\n" +
+      "            \"blockSize\":0,\n" +
+      "            \"childrenNum\":658,\n" +
+      "            \"fileId\":16386,\n" +
+      "            \"group\":\"supergroup\",\n" +
+      "            \"length\":0,\n" +
+      "            \"modificationTime\":1581578527580,\n" +
+      "            \"owner\":\"hdfs\",\n" +
+      "            \"pathSuffix\":\"user\",\n" +
+      "            \"permission\":\"755\",\n" +
+      "            \"replication\":0,\n" +
+      "            \"storagePolicy\":0,\n" +
+      "            \"type\":\"DIRECTORY\"\n" +
+      "          } \n" +
+      "       ]\n" +
+      "   }\n" +
+      "}";
+    Map<String,HashMap<String, ArrayList<HashMap<String, String>>>> map = JsonUtils.getFileStatusesAsMap(json);
+    assertNotNull(map);
+  }
 
 Review comment:
   nit: some more asserts to place here to see if the generated map has the proper content?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [knox] smolnar82 commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use
URL: https://github.com/apache/knox/pull/296#discussion_r396263931
 
 

 ##########
 File path: gateway-shell/src/main/java/org/apache/knox/gateway/shell/commands/CSVCommand.java
 ##########
 @@ -49,10 +49,24 @@ public Object execute(List<String> args) {
 
     try {
       if (withHeaders) {
-        table = KnoxShellTable.builder().csv().withHeaders().url(url);
+        if (url.startsWith("$")) {
+          // a knoxshell variable is a csv file as a string
+          String csvString = (String) getVariables().get(url.substring(1));
+          table = KnoxShellTable.builder().csv().withHeaders().string(csvString);
+        }
+        else {
+          table = KnoxShellTable.builder().csv().withHeaders().url(url);
+        }
       }
       else {
-        table = KnoxShellTable.builder().csv().url(url);
+        if (url.startsWith("$")) {
+          // a knoxshell variable is a csv file as a string
+          String csvString = (String) getVariables().get(url.substring(1));
+          table = KnoxShellTable.builder().csv().string(csvString);
+        }
+        else {
+          table = KnoxShellTable.builder().csv().url(url);
+        }
 
 Review comment:
   These lines can be replaced like:
   ```
       CSVKnoxShellTableBuilder csvTableBuilder = withHeaders ?  KnoxShellTable.builder().csv() :  KnoxShellTable.builder().csv().withHeaders();
       table = url.startsWith("$") ?  csvTableBuilder.env(url) ?  csvTableBuilder.url(url);
   ```
   Please note that `env(String variableName)` should be added in `CSVKnoxShellTableBuilder`.
   
   IMO the implementation is cleaner this way.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [knox] smolnar82 commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #296: KNOX-2240 - KnoxShell Custom Command for WEBHDFS Use
URL: https://github.com/apache/knox/pull/296#discussion_r396264979
 
 

 ##########
 File path: gateway-shell/src/main/java/org/apache/knox/gateway/shell/commands/WebHDFSCommand.java
 ##########
 @@ -0,0 +1,376 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.knox.gateway.shell.commands;
+
+import java.io.Console;
+import java.io.File;
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.TimeZone;
+
+import org.apache.knox.gateway.shell.CredentialCollectionException;
+import org.apache.knox.gateway.shell.CredentialCollector;
+import org.apache.knox.gateway.shell.KnoxSession;
+import org.apache.knox.gateway.shell.KnoxShellException;
+import org.apache.knox.gateway.shell.hdfs.Hdfs;
+import org.apache.knox.gateway.shell.hdfs.Status.Response;
+import org.apache.knox.gateway.shell.table.KnoxShellTable;
+import org.apache.knox.gateway.util.JsonUtils;
+import org.codehaus.groovy.tools.shell.Groovysh;
+
+public class WebHDFSCommand extends AbstractKnoxShellCommand {
+  private static final String KNOXMOUNTPOINTS = "__knoxmountpoints";
+  private Map<String, KnoxSession> sessions = new HashMap<>();
+
+  public WebHDFSCommand(Groovysh shell) {
+    super(shell, ":filesystem", ":fs");
+  }
+
+  @Override
+  public String getUsage() {
+    String usage = "Usage: \n" +
+                   "  :fs ls {target-path} \n" +
+                   "  :fs cat {target-path} \n" +
+                   "  :fs get {from-path} {to-path} \n" +
+                   "  :fs put {from-path} {tp-path} \n" +
+                   "  :fs rm {target-path} \n" +
+                   "  :fs mkdir {dir-path} \n";
+    return usage;
+  }
+
+  @Override
+  public Object execute(List<String> args) {
 
 Review comment:
   Each of the below cases should be in its own private method. For instance: `mount(...), unmount(...), ..., get(...)`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services