You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@streams.apache.org by sb...@apache.org on 2014/02/09 23:46:29 UTC

svn commit: r1566460 - in /incubator/streams/branches/sblackmon/streams-contrib: ./ streams-persist-hdfs/ streams-persist-hdfs/src/ streams-persist-hdfs/src/main/ streams-persist-hdfs/src/main/java/ streams-persist-hdfs/src/main/java/org/ streams-persi...

Author: sblackmon
Date: Sun Feb  9 22:46:28 2014
New Revision: 1566460

URL: http://svn.apache.org/r1566460
Log:
simple hdfs writer

make sure all machines running HdfsPersistWriter can resolve the hostnames the httpfs service uses to describe the cluster's datanodes

for now, edit pom directly to change hadoop versions.


Added:
    incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/
    incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/pom.xml
    incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/
    incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/
    incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/java/
    incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/java/org/
    incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/java/org/apache/
    incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/java/org/apache/streams/
    incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/java/org/apache/streams/hdfs/
    incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/java/org/apache/streams/hdfs/HdfsConfigurator.java
    incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/java/org/apache/streams/hdfs/WebHdfsPersistWriter.java
    incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/java/org/apache/streams/hdfs/WebHdfsPersistWriterTask.java
    incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/jsonschema/
    incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/jsonschema/org/
    incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/jsonschema/org/apache/
    incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/jsonschema/org/apache/streams/
    incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/jsonschema/org/apache/streams/hdfs/
    incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/jsonschema/org/apache/streams/hdfs/HdfsConfiguration.json
Modified:
    incubator/streams/branches/sblackmon/streams-contrib/pom.xml

Modified: incubator/streams/branches/sblackmon/streams-contrib/pom.xml
URL: http://svn.apache.org/viewvc/incubator/streams/branches/sblackmon/streams-contrib/pom.xml?rev=1566460&r1=1566459&r2=1566460&view=diff
==============================================================================
--- incubator/streams/branches/sblackmon/streams-contrib/pom.xml (original)
+++ incubator/streams/branches/sblackmon/streams-contrib/pom.xml Sun Feb  9 22:46:28 2014
@@ -37,6 +37,7 @@
 
     <modules>
         <module>streams-persist-console</module>
+        <module>streams-persist-hdfs</module>
         <module>streams-persist-kafka</module>
         <module>streams-provider-datasift</module>
         <module>streams-provider-facebook</module>

Added: incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/pom.xml
URL: http://svn.apache.org/viewvc/incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/pom.xml?rev=1566460&view=auto
==============================================================================
--- incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/pom.xml (added)
+++ incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/pom.xml Sun Feb  9 22:46:28 2014
@@ -0,0 +1,102 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>streams-contrib</artifactId>
+        <groupId>org.apache.streams</groupId>
+        <version>0.1-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>streams-persist-hdfs</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.streams</groupId>
+            <artifactId>streams-config</artifactId>
+            <version>0.1-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.streams</groupId>
+            <artifactId>streams-core</artifactId>
+            <version>0.1-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.streams</groupId>
+            <artifactId>streams-pojo</artifactId>
+            <version>0.1-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.streams</groupId>
+            <artifactId>streams-util</artifactId>
+            <version>0.1-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <version>2.0.0-cdh4.5.0</version>
+            <scope>compile</scope>
+            <type>jar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-hdfs</artifactId>
+            <version>2.0.0-cdh4.5.0</version>
+            <type>pom</type>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-hdfs-httpfs</artifactId>
+            <version>2.0.0-cdh4.5.0</version>
+            <type>pom</type>
+            <scope>compile</scope>
+        </dependency>
+    </dependencies>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <version>1.8</version>
+                <executions>
+                    <execution>
+                        <id>add-source</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>add-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>target/generated-sources/jsonschema2pojo</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.jsonschema2pojo</groupId>
+                <artifactId>jsonschema2pojo-maven-plugin</artifactId>
+                <configuration>
+                    <addCompileSourceRoot>true</addCompileSourceRoot>
+                    <generateBuilders>true</generateBuilders>
+                    <sourcePaths>
+                        <sourcePath>src/main/jsonschema/org/apache/streams/hdfs</sourcePath>
+                    </sourcePaths>
+                    <outputDirectory>target/generated-sources/jsonschema2pojo</outputDirectory>
+                    <targetPackage>org.apache.streams.hdfs.pojo</targetPackage>
+                    <useLongIntegers>true</useLongIntegers>
+                    <useJodaDates>false</useJodaDates>
+                </configuration>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>generate</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>
\ No newline at end of file

Added: incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/java/org/apache/streams/hdfs/HdfsConfigurator.java
URL: http://svn.apache.org/viewvc/incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/java/org/apache/streams/hdfs/HdfsConfigurator.java?rev=1566460&view=auto
==============================================================================
--- incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/java/org/apache/streams/hdfs/HdfsConfigurator.java (added)
+++ incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/java/org/apache/streams/hdfs/HdfsConfigurator.java Sun Feb  9 22:46:28 2014
@@ -0,0 +1,32 @@
+package org.apache.streams.hdfs;
+
+import com.typesafe.config.Config;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Created by sblackmon on 12/10/13.
+ */
+public class HdfsConfigurator {
+
+    private final static Logger LOGGER = LoggerFactory.getLogger(HdfsConfigurator.class);
+
+    public static HdfsConfiguration detectConfiguration(Config hdfs) {
+        String host = hdfs.getString("host");
+        Long port = hdfs.getLong("port");
+        String path = hdfs.getString("path");
+        String user = hdfs.getString("user");
+        String password = hdfs.getString("password");
+
+        HdfsConfiguration hdfsConfiguration = new HdfsConfiguration();
+
+        hdfsConfiguration.setHost(host);
+        hdfsConfiguration.setPort(port);
+        hdfsConfiguration.setPath(path);
+        hdfsConfiguration.setUser(user);
+        hdfsConfiguration.setPassword(password);
+
+        return hdfsConfiguration;
+    }
+
+}

Added: incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/java/org/apache/streams/hdfs/WebHdfsPersistWriter.java
URL: http://svn.apache.org/viewvc/incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/java/org/apache/streams/hdfs/WebHdfsPersistWriter.java?rev=1566460&view=auto
==============================================================================
--- incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/java/org/apache/streams/hdfs/WebHdfsPersistWriter.java (added)
+++ incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/java/org/apache/streams/hdfs/WebHdfsPersistWriter.java Sun Feb  9 22:46:28 2014
@@ -0,0 +1,330 @@
+package org.apache.streams.hdfs;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Strings;
+import com.typesafe.config.Config;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.streams.config.StreamsConfigurator;
+import org.apache.streams.core.StreamsDatum;
+import org.apache.streams.core.StreamsPersistWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.security.PrivilegedExceptionAction;
+import java.util.*;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+public class WebHdfsPersistWriter implements StreamsPersistWriter, Runnable
+{
+    private final static Logger LOGGER = LoggerFactory.getLogger(WebHdfsPersistWriter.class);
+
+    private final static char DELIMITER = '\t';
+    private final static int  DEFAULT_LINES_PER_FILE = 50000;
+
+    private FileSystem client;
+    private Path path;
+    private String filePart = "default";
+    private int linesPerFile = 1000;
+    private int totalRecordsWritten = 0;
+    private final List<Path> writtenFiles = new ArrayList<Path>();
+    private int fileLineCounter = 0;
+    private OutputStreamWriter currentWriter = null;
+
+    public boolean terminate = false;
+
+    protected volatile Queue<StreamsDatum> persistQueue;
+
+    private ObjectMapper mapper = new ObjectMapper();
+
+    private HdfsConfiguration config;
+
+    public WebHdfsPersistWriter() {
+        Config config = StreamsConfigurator.config.getConfig("hdfs");
+        this.config = HdfsConfigurator.detectConfiguration(config);
+        this.persistQueue  = new ConcurrentLinkedQueue<StreamsDatum>();
+    }
+
+    public WebHdfsPersistWriter(Queue<StreamsDatum> persistQueue) {
+        Config config = StreamsConfigurator.config.getConfig("hdfs");
+        this.config = HdfsConfigurator.detectConfiguration(config);
+        this.persistQueue = persistQueue;
+    }
+
+    public WebHdfsPersistWriter(HdfsConfiguration config) {
+        this.config = config;
+        this.persistQueue = new ConcurrentLinkedQueue<StreamsDatum>();
+    }
+
+    public WebHdfsPersistWriter(HdfsConfiguration config, Queue<StreamsDatum> persistQueue) {
+        this.config = config;
+        this.persistQueue = persistQueue;
+    }
+
+    public WebHdfsPersistWriter(HdfsConfiguration config, Queue<StreamsDatum> persistQueue, Path path) {
+        this.config = config;
+        this.persistQueue = persistQueue;
+        this.path = path;
+    }
+
+    public WebHdfsPersistWriter(HdfsConfiguration config, Queue<StreamsDatum> persistQueue, Path path, String filePart) {
+        this.config = config;
+        this.persistQueue = persistQueue;
+        this.path = path;
+        this.filePart = filePart;
+    }
+
+    public WebHdfsPersistWriter(HdfsConfiguration config, Queue<StreamsDatum> persistQueue, Path path, String filePart, int linesPerFile) {
+        this.config = config;
+        this.persistQueue = persistQueue;
+        this.path = path;
+        this.filePart = filePart;
+        this.linesPerFile = linesPerFile;
+    }
+
+    private static final int  BYTES_IN_MB = 1024*1024;
+    private static final int  BYTES_BEFORE_FLUSH = 5 * BYTES_IN_MB;
+    private volatile int  totalByteCount = 0;
+    private volatile int  byteCount = 0;
+
+    public URI getURI() throws URISyntaxException { return new URI(WebHdfsFileSystem.SCHEME + "://" + config.getHost() + ":" + config.getPort()); }
+    public boolean isConnected() 		                { return (client != null); }
+
+    public final synchronized FileSystem getFileSystem()
+    {
+        // Check to see if we are connected.
+        if(!isConnected())
+            connectToWebHDFS();
+        return this.client;
+    }
+
+    private synchronized void connectToWebHDFS()
+    {
+        try
+        {
+            LOGGER.info("User : {}", this.config.getUser());
+            UserGroupInformation ugi = UserGroupInformation.createRemoteUser(this.config.getUser());
+            ugi.setAuthenticationMethod(UserGroupInformation.AuthenticationMethod.SIMPLE);
+
+            ugi.doAs(new PrivilegedExceptionAction<Void>() {
+                public Void run() throws Exception {
+                    Configuration conf = new Configuration();
+                    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
+                    LOGGER.info("WebURI : {}", getURI().toString());
+                    client = FileSystem.get(getURI(), conf);
+                    LOGGER.info("Connected to WebHDFS");
+
+                    /*
+                    * ************************************************************************************************
+                    * This code is an example of how you would work with HDFS and you weren't going over
+                    * the webHDFS protocol.
+                    *
+                    * Smashew: 2013-10-01
+                    * ************************************************************************************************
+                    conf.set("fs.defaultFS", "hdfs://hadoop.mdigitallife.com:8020/user/" + userName);
+                    conf.set("namenode.host","0.0.0.0");
+                    conf.set("hadoop.job.ugi", userName);
+                    conf.set(DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY, "runner");
+                    fileSystem.createNewFile(new Path("/user/"+ userName + "/test"));
+                    FileStatus[] status = fs.listStatus(new Path("/user/" + userName));
+                    for(int i=0;i<status.length;i++)
+                    {
+                        LOGGER.info("Directory: {}", status[i].getPath());
+                    }
+                    */
+                    return null;
+                }
+            });
+        }
+        catch (Exception e)
+        {
+            LOGGER.error("There was an error connecting to WebHDFS, please check your settings and try again");
+            e.printStackTrace();
+        }
+    }
+    
+    @Override
+    public void write(StreamsDatum streamsDatum) {
+
+        synchronized (this)
+        {
+            // Check to see if we need to reset the file that we are currently working with
+            if (this.currentWriter == null || (this.fileLineCounter > this.linesPerFile))
+                try {
+                    resetFile();
+                } catch (Exception e) {
+                    e.printStackTrace();
+                }
+
+            String line = convertResultToString(streamsDatum);
+            try {
+                this.currentWriter.write(line);
+            } catch (IOException e) {
+                e.printStackTrace();
+            }
+            int bytesInLine = line.getBytes().length;
+
+            totalRecordsWritten++;
+            totalByteCount += bytesInLine;
+            byteCount += bytesInLine;
+
+            if(byteCount > BYTES_BEFORE_FLUSH)
+                try {
+                    flush();
+                } catch (IOException e) {
+                    e.printStackTrace();
+                }
+
+            this.fileLineCounter++;
+        }
+    }
+
+    public void flush() throws IOException
+    {
+        if(this.currentWriter != null && byteCount > BYTES_BEFORE_FLUSH)
+        {
+            this.currentWriter.flush();
+            byteCount = 0;
+        }
+    }
+
+    private synchronized void resetFile() throws Exception
+    {
+        // this will keep it thread safe, so we don't create too many files
+        if(this.fileLineCounter == 0 && this.currentWriter != null)
+            return;
+
+        // if there is a current writer, we must close it first.
+        if (this.currentWriter != null)
+        {
+            flush();
+            close();
+        }
+
+        this.fileLineCounter = 0;
+
+        // Create the path for where the file is going to live.
+        Path filePath = this.path.suffix("/" + this.filePart + "-" + new Date().getTime() + ".tsv");
+
+        try
+        {
+            // Check to see if a file of the same name exists, if it does, then we are not going to be able to proceed.
+            if(client.exists(filePath))
+                throw new RuntimeException("Unable to create file: " + filePath);
+
+            this.currentWriter = new OutputStreamWriter(client.create(filePath));
+
+            // Add another file to the list of written files.
+            writtenFiles.add(filePath);
+
+            LOGGER.info("File Created: {}", filePath);
+        }
+        catch (Exception e)
+        {
+            LOGGER.error("COULD NOT CreateFile: {}", filePath);
+            LOGGER.error(e.getMessage());
+            throw e;
+        }
+    }
+
+    public synchronized void close() throws IOException
+    {
+        if(this.currentWriter != null)
+        {
+            this.currentWriter.flush();
+            this.currentWriter.close();
+            this.currentWriter = null;
+            LOGGER.info("File Closed");
+        }
+    }
+
+    private String convertResultToString(StreamsDatum entry)
+    {
+        String metadata = null;
+        try {
+            metadata = mapper.writeValueAsString(entry.getMetadata());
+        } catch (JsonProcessingException e) {
+            e.printStackTrace();
+        }
+
+        String documentJson = null;
+        try {
+            documentJson = mapper.writeValueAsString(entry.getDocument());
+        } catch (JsonProcessingException e) {
+            e.printStackTrace();
+        }
+
+        if(Strings.isNullOrEmpty(documentJson))
+            return null;
+        else
+            return new StringBuilder()
+                    .append(entry.getSequenceid())
+                    .append(DELIMITER)
+                    .append(entry.getTimestamp())
+                    .append(DELIMITER)
+                    .append(metadata)
+                    .append(DELIMITER)
+                    .append(documentJson)
+                    .append("\n")
+                    .toString();
+    }
+
+    @Override
+    public void start() {
+
+        connectToWebHDFS();
+
+    }
+
+    @Override
+    public void stop() {
+
+        try {
+            flush();
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+        try {
+            close();
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+    }
+
+    @Override
+    public void setPersistQueue(Queue<StreamsDatum> persistQueue) {
+        this.persistQueue = persistQueue;
+    }
+
+    @Override
+    public Queue<StreamsDatum> getPersistQueue() {
+        return persistQueue;
+    }
+
+
+    @Override
+    public void run() {
+
+        start();
+
+        Thread task = new Thread(new WebHdfsPersistWriterTask(this));
+        task.start();
+
+        while( !terminate ) {
+            try {
+                Thread.sleep(new Random().nextInt(100));
+            } catch (InterruptedException e) { }
+        }
+
+        stop();
+    }
+}

Added: incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/java/org/apache/streams/hdfs/WebHdfsPersistWriterTask.java
URL: http://svn.apache.org/viewvc/incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/java/org/apache/streams/hdfs/WebHdfsPersistWriterTask.java?rev=1566460&view=auto
==============================================================================
--- incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/java/org/apache/streams/hdfs/WebHdfsPersistWriterTask.java (added)
+++ incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/java/org/apache/streams/hdfs/WebHdfsPersistWriterTask.java Sun Feb  9 22:46:28 2014
@@ -0,0 +1,38 @@
+package org.apache.streams.hdfs;
+
+import org.apache.streams.core.StreamsDatum;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Random;
+
+public class WebHdfsPersistWriterTask implements Runnable {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(WebHdfsPersistWriterTask.class);
+
+    private WebHdfsPersistWriter writer;
+
+    public WebHdfsPersistWriterTask(WebHdfsPersistWriter writer) {
+        this.writer = writer;
+    }
+
+    @Override
+    public void run() {
+
+        while(true) {
+            if( writer.getPersistQueue().peek() != null ) {
+                try {
+                    StreamsDatum entry = writer.persistQueue.remove();
+                    writer.write(entry);
+                } catch (Exception e) {
+                    e.printStackTrace();
+                }
+            }
+            try {
+                Thread.sleep(new Random().nextInt(1));
+            } catch (InterruptedException e) {}
+        }
+
+    }
+
+}

Added: incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/jsonschema/org/apache/streams/hdfs/HdfsConfiguration.json
URL: http://svn.apache.org/viewvc/incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/jsonschema/org/apache/streams/hdfs/HdfsConfiguration.json?rev=1566460&view=auto
==============================================================================
--- incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/jsonschema/org/apache/streams/hdfs/HdfsConfiguration.json (added)
+++ incubator/streams/branches/sblackmon/streams-contrib/streams-persist-hdfs/src/main/jsonschema/org/apache/streams/hdfs/HdfsConfiguration.json Sun Feb  9 22:46:28 2014
@@ -0,0 +1,29 @@
+{
+    "type": "object",
+    "$schema": "http://json-schema.org/draft-03/schema",
+    "id": "#",
+    "javaType" : "org.apache.streams.hdfs.HdfsConfiguration",
+    "javaInterfaces": ["java.io.Serializable"],
+    "properties": {
+        "host": {
+            "type": "string",
+            "description": "WebHdfs host"
+        },
+        "port": {
+            "type": "integer",
+            "description": "WebHdfs port"
+        },
+        "path": {
+            "type": "string",
+            "description": "Base path"
+        },
+        "user": {
+            "type": "string",
+            "description": "User"
+        },
+        "password": {
+            "type": "string",
+            "description": "Password"
+        }
+    }
+}
\ No newline at end of file