Return-Path: X-Original-To: apmail-accumulo-commits-archive@www.apache.org Delivered-To: apmail-accumulo-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id A11F8DAFD for ; Thu, 22 Nov 2012 00:56:04 +0000 (UTC) Received: (qmail 62881 invoked by uid 500); 22 Nov 2012 00:56:04 -0000 Delivered-To: apmail-accumulo-commits-archive@accumulo.apache.org Received: (qmail 62856 invoked by uid 500); 22 Nov 2012 00:56:04 -0000 Mailing-List: contact commits-help@accumulo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@accumulo.apache.org Delivered-To: mailing list commits@accumulo.apache.org Received: (qmail 62845 invoked by uid 99); 22 Nov 2012 00:56:04 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 22 Nov 2012 00:56:04 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 22 Nov 2012 00:55:54 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id 93B5523889BF for ; Thu, 22 Nov 2012 00:55:32 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1412398 [2/2] - in /accumulo/trunk: ./ bin/ conf/examples/vfs-classloader/ core/src/main/java/org/apache/accumulo/core/client/mock/ core/src/main/java/org/apache/accumulo/core/conf/ core/src/main/java/org/apache/accumulo/core/file/ core/sr... Date: Thu, 22 Nov 2012 00:55:28 -0000 To: commits@accumulo.apache.org From: dlmarion@apache.org X-Mailer: svnmailer-1.0.8-patched Message-Id: <20121122005532.93B5523889BF@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Added: accumulo/trunk/start/src/main/java/org/apache/commons/vfs2/provider/HdfsFileSystemConfigBuilder.java URL: http://svn.apache.org/viewvc/accumulo/trunk/start/src/main/java/org/apache/commons/vfs2/provider/HdfsFileSystemConfigBuilder.java?rev=1412398&view=auto ============================================================================== --- accumulo/trunk/start/src/main/java/org/apache/commons/vfs2/provider/HdfsFileSystemConfigBuilder.java (added) +++ accumulo/trunk/start/src/main/java/org/apache/commons/vfs2/provider/HdfsFileSystemConfigBuilder.java Thu Nov 22 00:55:22 2012 @@ -0,0 +1,45 @@ +/* + * 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.commons.vfs2.provider; + +import org.apache.commons.vfs2.FileSystem; +import org.apache.commons.vfs2.FileSystemConfigBuilder; + +public class HdfsFileSystemConfigBuilder extends FileSystemConfigBuilder { + + private final static HdfsFileSystemConfigBuilder BUILDER = new HdfsFileSystemConfigBuilder(); + + private String hdfsUri = null; + + public String getHdfsUri() { + return hdfsUri; + } + + public void setHdfsUri(String hdfsUri) { + this.hdfsUri = hdfsUri; + } + + @Override + protected Class getConfigClass() { + return ReadOnlyHdfsFileSystem.class; + } + + public static HdfsFileSystemConfigBuilder getInstance() { + return BUILDER; + } + +} Added: accumulo/trunk/start/src/main/java/org/apache/commons/vfs2/provider/HdfsReadOnlyFileContentInfoFactory.java URL: http://svn.apache.org/viewvc/accumulo/trunk/start/src/main/java/org/apache/commons/vfs2/provider/HdfsReadOnlyFileContentInfoFactory.java?rev=1412398&view=auto ============================================================================== --- accumulo/trunk/start/src/main/java/org/apache/commons/vfs2/provider/HdfsReadOnlyFileContentInfoFactory.java (added) +++ accumulo/trunk/start/src/main/java/org/apache/commons/vfs2/provider/HdfsReadOnlyFileContentInfoFactory.java Thu Nov 22 00:55:22 2012 @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.commons.vfs2.provider; + +import org.apache.commons.vfs2.FileContent; +import org.apache.commons.vfs2.FileContentInfo; +import org.apache.commons.vfs2.FileContentInfoFactory; +import org.apache.commons.vfs2.FileSystemException; +import org.apache.commons.vfs2.impl.DefaultFileContentInfo; + +public class HdfsReadOnlyFileContentInfoFactory implements FileContentInfoFactory { + + public FileContentInfo create(FileContent fileContent) throws FileSystemException { + //TODO: Need to figure out a way to get this information from the file. + String content = "text/plain"; + String encoding = "UTF-8"; + return new DefaultFileContentInfo(content, encoding); + } + +} Added: accumulo/trunk/start/src/main/java/org/apache/commons/vfs2/provider/HdfsReadOnlyRandomAccessContent.java URL: http://svn.apache.org/viewvc/accumulo/trunk/start/src/main/java/org/apache/commons/vfs2/provider/HdfsReadOnlyRandomAccessContent.java?rev=1412398&view=auto ============================================================================== --- accumulo/trunk/start/src/main/java/org/apache/commons/vfs2/provider/HdfsReadOnlyRandomAccessContent.java (added) +++ accumulo/trunk/start/src/main/java/org/apache/commons/vfs2/provider/HdfsReadOnlyRandomAccessContent.java Thu Nov 22 00:55:22 2012 @@ -0,0 +1,176 @@ +/* + * 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.commons.vfs2.provider; + +import java.io.IOException; +import java.io.InputStream; + +import org.apache.commons.vfs2.RandomAccessContent; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +public class HdfsReadOnlyRandomAccessContent implements RandomAccessContent { + + private FileSystem fs = null; + private Path path = null; + private FSDataInputStream fis = null; + + public HdfsReadOnlyRandomAccessContent(Path path, FileSystem fs) throws IOException { + this.fs = fs; + this.path = path; + this.fis = this.fs.open(this.path); + } + + public void write(int b) throws IOException { + throw new UnsupportedOperationException(); + } + + public void write(byte[] b) throws IOException { + throw new UnsupportedOperationException(); + } + + public void write(byte[] b, int off, int len) throws IOException { + throw new UnsupportedOperationException(); + } + + public void writeBoolean(boolean v) throws IOException { + throw new UnsupportedOperationException(); + } + + public void writeByte(int v) throws IOException { + throw new UnsupportedOperationException(); + } + + public void writeShort(int v) throws IOException { + throw new UnsupportedOperationException(); + } + + public void writeChar(int v) throws IOException { + throw new UnsupportedOperationException(); + } + + public void writeInt(int v) throws IOException { + throw new UnsupportedOperationException(); + } + + public void writeLong(long v) throws IOException { + throw new UnsupportedOperationException(); + } + + public void writeFloat(float v) throws IOException { + throw new UnsupportedOperationException(); + } + + public void writeDouble(double v) throws IOException { + throw new UnsupportedOperationException(); + } + + public void writeBytes(String s) throws IOException { + throw new UnsupportedOperationException(); + } + + public void writeChars(String s) throws IOException { + throw new UnsupportedOperationException(); + } + + public void writeUTF(String s) throws IOException { + throw new UnsupportedOperationException(); + } + + public void readFully(byte[] b) throws IOException { + throw new UnsupportedOperationException(); + } + + public void readFully(byte[] b, int off, int len) throws IOException { + throw new UnsupportedOperationException(); + } + + public int skipBytes(int n) throws IOException { + throw new UnsupportedOperationException(); + } + + public boolean readBoolean() throws IOException { + return this.fis.readBoolean(); + } + + public byte readByte() throws IOException { + return this.fis.readByte(); + } + + public int readUnsignedByte() throws IOException { + return this.fis.readUnsignedByte(); + } + + public short readShort() throws IOException { + return this.fis.readShort(); + } + + public int readUnsignedShort() throws IOException { + return this.fis.readUnsignedShort(); + } + + public char readChar() throws IOException { + return this.fis.readChar(); + } + + public int readInt() throws IOException { + return this.fis.readInt(); + } + + public long readLong() throws IOException { + return this.fis.readLong(); + } + + public float readFloat() throws IOException { + return this.fis.readFloat(); + } + + public double readDouble() throws IOException { + return this.fis.readDouble(); + } + + @SuppressWarnings("deprecation") + public String readLine() throws IOException { + return this.fis.readLine(); + } + + public String readUTF() throws IOException { + return this.fis.readUTF(); + } + + public long getFilePointer() throws IOException { + return this.fis.getPos(); + } + + public void seek(long pos) throws IOException { + this.fis.seek(pos); + } + + public long length() throws IOException { + return this.fs.getFileStatus(this.path).getLen(); + } + + public void close() throws IOException { + this.fis.close(); + } + + public InputStream getInputStream() throws IOException { + return fis; + } + +} Added: accumulo/trunk/start/src/main/java/org/apache/commons/vfs2/provider/ReadOnlyHdfsFileProvider.java URL: http://svn.apache.org/viewvc/accumulo/trunk/start/src/main/java/org/apache/commons/vfs2/provider/ReadOnlyHdfsFileProvider.java?rev=1412398&view=auto ============================================================================== --- accumulo/trunk/start/src/main/java/org/apache/commons/vfs2/provider/ReadOnlyHdfsFileProvider.java (added) +++ accumulo/trunk/start/src/main/java/org/apache/commons/vfs2/provider/ReadOnlyHdfsFileProvider.java Thu Nov 22 00:55:22 2012 @@ -0,0 +1,65 @@ +/* + * 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.commons.vfs2.provider; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; + +import org.apache.commons.vfs2.Capability; +import org.apache.commons.vfs2.FileName; +import org.apache.commons.vfs2.FileSystem; +import org.apache.commons.vfs2.FileSystemConfigBuilder; +import org.apache.commons.vfs2.FileSystemException; +import org.apache.commons.vfs2.FileSystemOptions; +import org.apache.commons.vfs2.provider.http.HttpFileNameParser; + +public class ReadOnlyHdfsFileProvider extends AbstractOriginatingFileProvider { + + public static final Collection capabilities = Collections.unmodifiableCollection(Arrays.asList(new Capability[] + { + Capability.GET_TYPE, + Capability.READ_CONTENT, + Capability.URI, + Capability.GET_LAST_MODIFIED, + Capability.ATTRIBUTES, + Capability.RANDOM_ACCESS_READ, + Capability.DIRECTORY_READ_CONTENT, + Capability.LIST_CHILDREN, + Capability.RANDOM_ACCESS_READ, + })); + + public ReadOnlyHdfsFileProvider() { + super(); + this.setFileNameParser(HttpFileNameParser.getInstance()); + } + + public Collection getCapabilities() { + return capabilities; + } + + @Override + protected FileSystem doCreateFileSystem(FileName rootName, FileSystemOptions fileSystemOptions) throws FileSystemException { + return new ReadOnlyHdfsFileSystem(rootName, fileSystemOptions); + } + + @Override + public FileSystemConfigBuilder getConfigBuilder() { + return HdfsFileSystemConfigBuilder.getInstance(); + } + +} Added: accumulo/trunk/start/src/main/java/org/apache/commons/vfs2/provider/ReadOnlyHdfsFileSystem.java URL: http://svn.apache.org/viewvc/accumulo/trunk/start/src/main/java/org/apache/commons/vfs2/provider/ReadOnlyHdfsFileSystem.java?rev=1412398&view=auto ============================================================================== --- accumulo/trunk/start/src/main/java/org/apache/commons/vfs2/provider/ReadOnlyHdfsFileSystem.java (added) +++ accumulo/trunk/start/src/main/java/org/apache/commons/vfs2/provider/ReadOnlyHdfsFileSystem.java Thu Nov 22 00:55:22 2012 @@ -0,0 +1,87 @@ +/* + * 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.commons.vfs2.provider; + +import java.io.IOException; +import java.util.Collection; + +import org.apache.commons.vfs2.Capability; +import org.apache.commons.vfs2.FileName; +import org.apache.commons.vfs2.FileObject; +import org.apache.commons.vfs2.FileSystemException; +import org.apache.commons.vfs2.FileSystemOptions; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.Logger; + +public class ReadOnlyHdfsFileSystem extends AbstractFileSystem { + + private static final Logger log = Logger.getLogger(ReadOnlyHdfsFileSystem.class); + + private FileSystem fs = null; + + protected ReadOnlyHdfsFileSystem(FileName rootName, FileSystemOptions fileSystemOptions) { + super(rootName, null, fileSystemOptions); + } + + @Override + public void close() { + try { + if (null != fs) + fs.close(); + } catch (IOException e) { + throw new RuntimeException("Error closing HDFS client", e); + } + super.close(); + } + + @Override + protected FileObject createFile(AbstractFileName name) throws Exception { + throw new FileSystemException("Operation not supported"); + } + + @Override + protected void addCapabilities(Collection capabilities) { + capabilities.addAll(ReadOnlyHdfsFileProvider.capabilities); + } + + @Override + public FileObject resolveFile(FileName name) throws FileSystemException { + + synchronized (this) { + if (null == this.fs) { + String hdfsUri = name.getRootURI(); + Configuration conf = new Configuration(true); + conf.set(org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY, hdfsUri); + this.fs = null; + try { + fs = org.apache.hadoop.fs.FileSystem.get(conf); + + } catch (IOException e) { + log.error("Error connecting to filesystem " + hdfsUri, e); + throw new FileSystemException("Error connecting to filesystem " + hdfsUri, e); + } + } + } + + Path filePath = new Path(name.getPath()); + return new HdfsFileObject((AbstractFileName) name, this, fs, filePath); + + } + +} Added: accumulo/trunk/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloContextClassLoaderTest.java URL: http://svn.apache.org/viewvc/accumulo/trunk/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloContextClassLoaderTest.java?rev=1412398&view=auto ============================================================================== --- accumulo/trunk/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloContextClassLoaderTest.java (added) +++ accumulo/trunk/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloContextClassLoaderTest.java Thu Nov 22 00:55:22 2012 @@ -0,0 +1,87 @@ +/* + * 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.accumulo.start.classloader.vfs; + +import java.net.URL; + +import org.apache.accumulo.test.AccumuloDFSBase; +import org.apache.commons.vfs2.FileObject; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class AccumuloContextClassLoaderTest extends AccumuloDFSBase { + + private static final Path TEST_DIR = new Path(HDFS_URI + "/test-dir"); + private static final Path TEST_DIR2 = new Path(HDFS_URI + "/test-dir2"); + + private FileSystem hdfs = null; + private AccumuloContextClassLoader cl = null; + + @Before + public void setup() throws Exception { + this.hdfs = cluster.getFileSystem(); + this.hdfs.mkdirs(TEST_DIR); + this.hdfs.mkdirs(TEST_DIR2); + + //Copy jar file to TEST_DIR + URL jarPath = this.getClass().getResource("/HelloWorld.jar"); + Path src = new Path(jarPath.toURI().toString()); + Path dst = new Path(TEST_DIR, src.getName()); + this.hdfs.copyFromLocalFile(src, dst); + + Path dst2 = new Path(TEST_DIR2, src.getName()); + this.hdfs.copyFromLocalFile(src, dst2); + + } + + @Test + public void differentContexts() throws Exception { + FileObject testDir = vfs.resolveFile(TEST_DIR.toUri().toString()); + FileObject[] dirContents = testDir.getChildren(); + cl = new AccumuloContextClassLoader(dirContents, vfs, ClassLoader.getSystemClassLoader()); + FileObject[] files = cl.getClassLoader(AccumuloContextClassLoader.DEFAULT_CONTEXT).getFiles(); + Assert.assertArrayEquals(dirContents, files); + + FileObject testDir2 = vfs.resolveFile(TEST_DIR2.toUri().toString()); + FileObject[] dirContents2 = testDir2.getChildren(); + cl.addContext("MYCONTEXT", dirContents2); + FileObject[] files2 = cl.getClassLoader("MYCONTEXT").getFiles(); + Assert.assertArrayEquals(dirContents2, files2); + + Class defaultContextClass = cl.loadClass("test.HelloWorld"); + Object o1 = defaultContextClass.newInstance(); + Assert.assertEquals("Hello World!", o1.toString()); + + Class myContextClass = cl.loadClass("MYCONTEXT", "test.HelloWorld"); + Object o2 = myContextClass.newInstance(); + Assert.assertEquals("Hello World!", o2.toString()); + + Assert.assertFalse(defaultContextClass.equals(myContextClass)); + + } + + @After + public void tearDown() throws Exception { + cl.close(); + this.hdfs.delete(TEST_DIR, true); + } + +} Added: accumulo/trunk/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloReloadingVFSClassLoaderTest.java URL: http://svn.apache.org/viewvc/accumulo/trunk/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloReloadingVFSClassLoaderTest.java?rev=1412398&view=auto ============================================================================== --- accumulo/trunk/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloReloadingVFSClassLoaderTest.java (added) +++ accumulo/trunk/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloReloadingVFSClassLoaderTest.java Thu Nov 22 00:55:22 2012 @@ -0,0 +1,104 @@ +/* + * 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.accumulo.start.classloader.vfs; + +import java.net.URL; + +import org.apache.accumulo.test.AccumuloDFSBase; +import org.apache.commons.vfs2.FileObject; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.Level; +import org.apache.log4j.Logger; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class AccumuloReloadingVFSClassLoaderTest extends AccumuloDFSBase { + + private static final Path TEST_DIR = new Path(HDFS_URI + "/test-dir"); + + private FileSystem hdfs = null; + private AccumuloReloadingVFSClassLoader cl = null; + + @Before + public void setup() throws Exception { + Logger.getRootLogger().setLevel(Level.ERROR); + + this.hdfs = cluster.getFileSystem(); + this.hdfs.mkdirs(TEST_DIR); + + //Copy jar file to TEST_DIR + URL jarPath = this.getClass().getResource("/HelloWorld.jar"); + Path src = new Path(jarPath.toURI().toString()); + Path dst = new Path(TEST_DIR, src.getName()); + this.hdfs.copyFromLocalFile(src, dst); + + } + + @Test + public void testConstructor() throws Exception { + FileObject testDir = vfs.resolveFile(TEST_DIR.toUri().toString()); + FileObject[] dirContents = testDir.getChildren(); + cl = new AccumuloReloadingVFSClassLoader(dirContents, vfs, ClassLoader.getSystemClassLoader()); + FileObject[] files = cl.getFiles(); + Assert.assertArrayEquals(dirContents, files); + } + + @Test + public void testReloading() throws Exception { + FileObject testDir = vfs.resolveFile(TEST_DIR.toUri().toString()); + FileObject[] dirContents = testDir.getChildren(); + cl = new AccumuloReloadingVFSClassLoader(dirContents, vfs, ClassLoader.getSystemClassLoader(), 1000); + FileObject[] files = cl.getFiles(); + Assert.assertArrayEquals(dirContents, files); + + Class clazz1 = cl.loadClass("test.HelloWorld"); + Object o1 = clazz1.newInstance(); + Assert.assertEquals("Hello World!", o1.toString()); + + //Check that the class is the same before the update + Class clazz1_5 = cl.loadClass("test.HelloWorld"); + Assert.assertEquals(clazz1, clazz1_5); + + //Update the class + URL jarPath = this.getClass().getResource("/HelloWorld.jar"); + Path src = new Path(jarPath.toURI().toString()); + Path dst = new Path(TEST_DIR, "HelloWorld.jar"); + this.hdfs.copyFromLocalFile(src, dst); + + //Wait for the monitor to notice + Thread.sleep(2000); + + Class clazz2 = cl.loadClass("test.HelloWorld"); + Object o2 = clazz2.newInstance(); + Assert.assertEquals("Hello World!", o2.toString()); + + //This is false because they are loaded by a different classloader + Assert.assertFalse(clazz1.equals(clazz2)); + Assert.assertFalse(o1.equals(o2)); + + } + + @After + public void tearDown() throws Exception { + cl.close(); + this.hdfs.delete(TEST_DIR, true); + } + +} Added: accumulo/trunk/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoaderTest.java URL: http://svn.apache.org/viewvc/accumulo/trunk/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoaderTest.java?rev=1412398&view=auto ============================================================================== --- accumulo/trunk/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoaderTest.java (added) +++ accumulo/trunk/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoaderTest.java Thu Nov 22 00:55:22 2012 @@ -0,0 +1,206 @@ +/* + * 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.accumulo.start.classloader.vfs; + +import java.io.File; +import java.net.URL; +import java.net.URLClassLoader; + +import org.apache.accumulo.test.AccumuloDFSBase; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.Logger; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.core.classloader.annotations.SuppressStaticInitializationFor; +import org.powermock.modules.junit4.PowerMockRunner; +import org.powermock.reflect.Whitebox; + +@RunWith(PowerMockRunner.class) +@PrepareForTest(AccumuloVFSClassLoader.class) +@SuppressStaticInitializationFor({"org.apache.accumulo.start.classloader.AccumuloVFSClassLoader", + "org.apache.log4j.LogManager"}) +@PowerMockIgnore({"org.apache.log4j.*", + "org.apache.hadoop.log.metrics", "org.apache.commons.logging.*", + "org.xml.*", "javax.xml.*", "org.w3c.dom.*", "org.apache.hadoop.*"}) +public class AccumuloVFSClassLoaderTest extends AccumuloDFSBase { + + + /* + * Test that if not enabled, the AccumuloClassLoader is used + */ + @Test + public void testNoConfigChanges() throws Exception { + Whitebox.setInternalState(AccumuloVFSClassLoader.class, "ACC_CONF", new Configuration()); + Whitebox.setInternalState(AccumuloVFSClassLoader.class, "lock", new Object()); + Whitebox.setInternalState(AccumuloVFSClassLoader.class, "log", Logger.getLogger(AccumuloVFSClassLoader.class)); + ClassLoader acl = AccumuloVFSClassLoader.getClassLoader(); + Assert.assertTrue((acl instanceof URLClassLoader)); + Whitebox.setInternalState(AccumuloVFSClassLoader.class, "loader", (AccumuloContextClassLoader) null); + } + + /* + * Test that if enabled, but not configured, that the code creates the 2nd level classloader + */ + @Test + public void testDefaultConfig() throws Exception { + Configuration conf = new Configuration(); + URL defaultDir = this.getClass().getResource("/disabled"); + conf.addResource(new File(defaultDir.getPath() + "/conf/accumulo-site.xml").toURI().toURL()); + Whitebox.setInternalState(AccumuloVFSClassLoader.class, "ACC_CONF", conf); + Whitebox.setInternalState(AccumuloVFSClassLoader.class, "lock", new Object()); + Whitebox.setInternalState(AccumuloVFSClassLoader.class, "log", Logger.getLogger(AccumuloVFSClassLoader.class)); + ClassLoader acl = AccumuloVFSClassLoader.getClassLoader(); + Assert.assertTrue((acl instanceof URLClassLoader)); + Whitebox.setInternalState(AccumuloVFSClassLoader.class, "loader", (AccumuloContextClassLoader) null); +// URLClassLoader ucl = (URLClassLoader) acl; +// URL[] classpath = ucl.getURLs(); +// System.out.println(Arrays.toString(classpath)); + } + + /* + * Test with default context configured + */ + @Test + public void testDefaultContextConfigured() throws Exception { + + //Create default context directory + @SuppressWarnings("resource") + FileSystem hdfs = cluster.getFileSystem(); + Path DEFAULT = new Path("/accumulo/classpath"); + hdfs.mkdirs(DEFAULT); + + //Copy jar file to TEST_DIR + URL jarPath = this.getClass().getResource("/HelloWorld.jar"); + Path src = new Path(jarPath.toURI().toString()); + Path dst = new Path(DEFAULT, src.getName()); + hdfs.copyFromLocalFile(src, dst); + + URL defaultDir = this.getClass().getResource("/default"); + conf.addResource(new File(defaultDir.getPath() + "/conf/accumulo-site.xml").toURI().toURL()); + Whitebox.setInternalState(AccumuloVFSClassLoader.class, "ACC_CONF", conf); + Whitebox.setInternalState(AccumuloVFSClassLoader.class, "lock", new Object()); + Whitebox.setInternalState(AccumuloVFSClassLoader.class, "log", Logger.getLogger(AccumuloVFSClassLoader.class)); + ClassLoader acl = AccumuloVFSClassLoader.getClassLoader(); + Assert.assertTrue((acl instanceof AccumuloContextClassLoader)); + AccumuloContextClassLoader accl = (AccumuloContextClassLoader) acl; + AccumuloReloadingVFSClassLoader arvcl = accl.getClassLoader(AccumuloContextClassLoader.DEFAULT_CONTEXT); + Assert.assertEquals(1, arvcl.getFiles().length); + Assert.assertTrue(arvcl.getFiles()[0].getURL().toString().equals("hdfs://localhost:8020/accumulo/classpath/HelloWorld.jar")); + Class clazz1 = arvcl.loadClass("test.HelloWorld"); + Object o1 = clazz1.newInstance(); + Assert.assertEquals("Hello World!", o1.toString()); + Whitebox.setInternalState(AccumuloVFSClassLoader.class, "loader", (AccumuloContextClassLoader) null); + + hdfs.delete(DEFAULT, true); + + } + + @Test + public void testAdditionalContextConfigured() throws Exception { + + //Create default and application1 context directory + @SuppressWarnings("resource") + FileSystem hdfs = cluster.getFileSystem(); + Path DEFAULT = new Path("/accumulo/classpath"); + hdfs.mkdirs(DEFAULT); + Path APPLICATION = new Path("/application1/classpath"); + hdfs.mkdirs(APPLICATION); + + + //Copy jar file to DEFAULT and APPLICATION directories + URL jarPath = this.getClass().getResource("/HelloWorld.jar"); + Path src = new Path(jarPath.toURI().toString()); + Path dst = new Path(DEFAULT, src.getName()); + hdfs.copyFromLocalFile(src, dst); + dst = new Path(APPLICATION, src.getName()); + hdfs.copyFromLocalFile(src, dst); + + URL defaultDir = this.getClass().getResource("/application1"); + conf.addResource(new File(defaultDir.getPath() + "/conf/accumulo-site.xml").toURI().toURL()); + Whitebox.setInternalState(AccumuloVFSClassLoader.class, "ACC_CONF", conf); + Whitebox.setInternalState(AccumuloVFSClassLoader.class, "lock", new Object()); + Whitebox.setInternalState(AccumuloVFSClassLoader.class, "log", Logger.getLogger(AccumuloVFSClassLoader.class)); + + ClassLoader acl = AccumuloVFSClassLoader.getClassLoader(); + Assert.assertTrue((acl instanceof AccumuloContextClassLoader)); + AccumuloContextClassLoader accl = (AccumuloContextClassLoader) acl; + //DEFAULT CONTEXT + AccumuloReloadingVFSClassLoader arvcl = accl.getClassLoader(AccumuloContextClassLoader.DEFAULT_CONTEXT); + Assert.assertEquals(1, arvcl.getFiles().length); + Assert.assertTrue(arvcl.getFiles()[0].getURL().toString().equals("hdfs://localhost:8020/accumulo/classpath/HelloWorld.jar")); + Class clazz1 = arvcl.loadClass("test.HelloWorld"); + Object o1 = clazz1.newInstance(); + Assert.assertEquals("Hello World!", o1.toString()); + + //APPLICATION CONTEXT + AccumuloReloadingVFSClassLoader arvcl2 = accl.getClassLoader("application1"); + Assert.assertEquals(1, arvcl2.getFiles().length); + Assert.assertTrue(arvcl2.getFiles()[0].getURL().toString().equals("hdfs://localhost:8020/application1/classpath/HelloWorld.jar")); + Class clazz2 = arvcl2.loadClass("test.HelloWorld"); + Object o2 = clazz2.newInstance(); + Assert.assertEquals("Hello World!", o2.toString()); + + Assert.assertTrue(!o1.equals(o2)); + Whitebox.setInternalState(AccumuloVFSClassLoader.class, "loader", (AccumuloContextClassLoader) null); + + hdfs.delete(DEFAULT, true); + hdfs.delete(APPLICATION, true); + + } + + @Test + public void testLoadClass() throws Exception { + //Create default and application1 context directory + @SuppressWarnings("resource") + FileSystem hdfs = cluster.getFileSystem(); + Path DEFAULT = new Path("/accumulo/classpath"); + hdfs.mkdirs(DEFAULT); + Path APPLICATION = new Path("/application1/classpath"); + hdfs.mkdirs(APPLICATION); + + + //Copy jar file to DEFAULT and APPLICATION directories + URL jarPath = this.getClass().getResource("/HelloWorld.jar"); + Path src = new Path(jarPath.toURI().toString()); + Path dst = new Path(DEFAULT, src.getName()); + hdfs.copyFromLocalFile(src, dst); + dst = new Path(APPLICATION, src.getName()); + hdfs.copyFromLocalFile(src, dst); + + URL defaultDir = this.getClass().getResource("/application1"); + conf.addResource(new File(defaultDir.getPath() + "/conf/accumulo-site.xml").toURI().toURL()); + Whitebox.setInternalState(AccumuloVFSClassLoader.class, "ACC_CONF", conf); + Whitebox.setInternalState(AccumuloVFSClassLoader.class, "lock", new Object()); + Whitebox.setInternalState(AccumuloVFSClassLoader.class, "log", Logger.getLogger(AccumuloVFSClassLoader.class)); + + Class clazz1 = AccumuloVFSClassLoader.loadClass("test.HelloWorld"); + Object o1 = clazz1.newInstance(); + Assert.assertEquals("Hello World!", o1.toString()); + + Whitebox.setInternalState(AccumuloVFSClassLoader.class, "loader", (AccumuloContextClassLoader) null); + + hdfs.delete(DEFAULT, true); + hdfs.delete(APPLICATION, true); + + } + +} Added: accumulo/trunk/start/src/test/java/org/apache/accumulo/test/AccumuloDFSBase.java URL: http://svn.apache.org/viewvc/accumulo/trunk/start/src/test/java/org/apache/accumulo/test/AccumuloDFSBase.java?rev=1412398&view=auto ============================================================================== --- accumulo/trunk/start/src/test/java/org/apache/accumulo/test/AccumuloDFSBase.java (added) +++ accumulo/trunk/start/src/test/java/org/apache/accumulo/test/AccumuloDFSBase.java Thu Nov 22 00:55:22 2012 @@ -0,0 +1,137 @@ +/* + * 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.accumulo.test; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; + +import org.apache.commons.vfs2.CacheStrategy; +import org.apache.commons.vfs2.FileSystemException; +import org.apache.commons.vfs2.cache.DefaultFilesCache; +import org.apache.commons.vfs2.cache.SoftRefFilesCache; +import org.apache.commons.vfs2.impl.DefaultFileReplicator; +import org.apache.commons.vfs2.impl.DefaultFileSystemManager; +import org.apache.commons.vfs2.impl.FileContentInfoFilenameFactory; +import org.apache.commons.vfs2.provider.ReadOnlyHdfsFileProvider; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.log4j.Level; +import org.apache.log4j.Logger; + +public class AccumuloDFSBase { + + //Turn off the MiniDFSCluster logging + static { + System.setProperty("org.apache.commons.logging.Log", "org.apache.commons.logging.impl.NoOpLog"); + } + + protected static final String HDFS_URI = "hdfs://localhost:8020"; + + protected static Configuration conf = null; + protected static DefaultFileSystemManager vfs = null; + protected static MiniDFSCluster cluster = null; + static { + Logger.getRootLogger().setLevel(Level.ERROR); + + //Put the MiniDFSCluster directory in the target directory + System.setProperty("test.build.data", "target/build/test/data"); + + //Setup HDFS + conf = new Configuration(); + conf.set(FileSystem.FS_DEFAULT_NAME_KEY, HDFS_URI); + conf.set("hadoop.security.token.service.use_ip", "true"); + + // MiniDFSCluster will check the permissions on the data directories, but does not + // do a good job of setting them properly. We need to get the users umask and set + // the appropriate Hadoop property so that the data directories will be created + // with the correct permissions. + try { + Process p = Runtime.getRuntime().exec("/bin/sh -c umask"); + BufferedReader bri = new BufferedReader(new InputStreamReader(p.getInputStream())); + String line = bri.readLine(); + p.waitFor(); +// System.out.println("umask response: " + line); + Short umask = Short.parseShort(line.trim(), 8); + //Need to set permission to 777 xor umask + // leading zero makes java interpret as base 8 + int newPermission = 0777 ^ umask; +// System.out.println("Umask is: " + String.format("%03o", umask)); +// System.out.println("Perm is: " + String.format("%03o", newPermission)); + conf.set("dfs.datanode.data.dir.perm", String.format("%03o", newPermission)); + } catch (Exception e) { + throw new RuntimeException("Error getting umask from O/S", e); + } + + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024 * 100); //100K blocksize + + try { + cluster = new MiniDFSCluster(8020, conf, 1, true, true, true, null, null, null, null); + cluster.waitActive(); + } catch (IOException e) { + throw new RuntimeException("Error setting up mini cluster", e); + } + + //Set up the VFS + vfs = new DefaultFileSystemManager(); + try { + vfs.setFilesCache(new DefaultFilesCache()); + vfs.addProvider("res", new org.apache.commons.vfs2.provider.res.ResourceFileProvider()); + vfs.addProvider("zip", new org.apache.commons.vfs2.provider.zip.ZipFileProvider()); + vfs.addProvider("gz", new org.apache.commons.vfs2.provider.gzip.GzipFileProvider()); + vfs.addProvider("ram", new org.apache.commons.vfs2.provider.ram.RamFileProvider()); + vfs.addProvider("file", new org.apache.commons.vfs2.provider.local.DefaultLocalFileProvider()); + vfs.addProvider("jar", new org.apache.commons.vfs2.provider.jar.JarFileProvider()); + vfs.addProvider("http", new org.apache.commons.vfs2.provider.http.HttpFileProvider()); + vfs.addProvider("https", new org.apache.commons.vfs2.provider.https.HttpsFileProvider()); + vfs.addProvider("ftp", new org.apache.commons.vfs2.provider.ftp.FtpFileProvider()); + vfs.addProvider("ftps", new org.apache.commons.vfs2.provider.ftps.FtpsFileProvider()); + vfs.addProvider("war", new org.apache.commons.vfs2.provider.jar.JarFileProvider()); + vfs.addProvider("par", new org.apache.commons.vfs2.provider.jar.JarFileProvider()); + vfs.addProvider("ear", new org.apache.commons.vfs2.provider.jar.JarFileProvider()); + vfs.addProvider("sar", new org.apache.commons.vfs2.provider.jar.JarFileProvider()); + vfs.addProvider("ejb3", new org.apache.commons.vfs2.provider.jar.JarFileProvider()); + vfs.addProvider("tmp", new org.apache.commons.vfs2.provider.temp.TemporaryFileProvider()); + vfs.addProvider("tar", new org.apache.commons.vfs2.provider.tar.TarFileProvider()); + vfs.addProvider("tbz2", new org.apache.commons.vfs2.provider.tar.TarFileProvider()); + vfs.addProvider("tgz", new org.apache.commons.vfs2.provider.tar.TarFileProvider()); + vfs.addProvider("bz2", new org.apache.commons.vfs2.provider.bzip2.Bzip2FileProvider()); + vfs.addProvider("hdfs", new ReadOnlyHdfsFileProvider()); + vfs.addExtensionMap("jar", "jar"); + vfs.addExtensionMap("zip", "zip"); + vfs.addExtensionMap("gz", "gz"); + vfs.addExtensionMap("tar", "tar"); + vfs.addExtensionMap("tbz2", "tar"); + vfs.addExtensionMap("tgz", "tar"); + vfs.addExtensionMap("bz2", "bz2"); + vfs.addMimeTypeMap("application/x-tar", "tar"); + vfs.addMimeTypeMap("application/x-gzip", "gz"); + vfs.addMimeTypeMap("application/zip", "zip"); + vfs.setFileContentInfoFactory(new FileContentInfoFilenameFactory()); + vfs.setFilesCache(new SoftRefFilesCache()); + vfs.setReplicator(new DefaultFileReplicator()); + vfs.setCacheStrategy(CacheStrategy.ON_RESOLVE); + vfs.init(); + } catch (FileSystemException e) { + throw new RuntimeException("Error setting up VFS", e); + } + + } + +} Added: accumulo/trunk/start/src/test/java/org/apache/commons/vfs2/provider/ReadOnlyHdfsFileProviderTest.java URL: http://svn.apache.org/viewvc/accumulo/trunk/start/src/test/java/org/apache/commons/vfs2/provider/ReadOnlyHdfsFileProviderTest.java?rev=1412398&view=auto ============================================================================== --- accumulo/trunk/start/src/test/java/org/apache/commons/vfs2/provider/ReadOnlyHdfsFileProviderTest.java (added) +++ accumulo/trunk/start/src/test/java/org/apache/commons/vfs2/provider/ReadOnlyHdfsFileProviderTest.java Thu Nov 22 00:55:22 2012 @@ -0,0 +1,246 @@ +/* + * 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.commons.vfs2.provider; + +import java.io.IOException; +import java.util.Map; + +import org.apache.accumulo.test.AccumuloDFSBase; +import org.apache.commons.vfs2.FileObject; +import org.apache.commons.vfs2.FileSystemException; +import org.apache.commons.vfs2.FileType; +import org.apache.commons.vfs2.impl.DefaultFileSystemManager; +import org.apache.commons.vfs2.util.RandomAccessMode; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class ReadOnlyHdfsFileProviderTest extends AccumuloDFSBase { + + private static final String TEST_DIR1 = HDFS_URI + "/test-dir"; + private static final Path DIR1_PATH = new Path("/test-dir"); + private static final String TEST_FILE1 = TEST_DIR1 + "/accumulo-test-1.jar"; + private static final Path FILE1_PATH = new Path(DIR1_PATH, "accumulo-test-1.jar"); + + private DefaultFileSystemManager manager = null; + private FileSystem hdfs = null; + + @Before + public void setup() throws Exception { + manager = new DefaultFileSystemManager(); + manager.addProvider("hdfs", new ReadOnlyHdfsFileProvider()); + manager.init(); + this.hdfs = cluster.getFileSystem(); + } + + private FileObject createTestFile(FileSystem hdfs) throws IOException { + //Create the directory + hdfs.mkdirs(DIR1_PATH); + FileObject dir = manager.resolveFile(TEST_DIR1); + Assert.assertNotNull(dir); + Assert.assertTrue(dir.exists()); + Assert.assertTrue(dir.getType().equals(FileType.FOLDER)); + + //Create the file in the directory + hdfs.create(FILE1_PATH).close(); + FileObject f = manager.resolveFile(TEST_FILE1); + Assert.assertNotNull(f); + Assert.assertTrue(f.exists()); + Assert.assertTrue(f.getType().equals(FileType.FILE)); + return f; + } + + @Test + public void testInit() throws Exception { + FileObject fo = manager.resolveFile(TEST_FILE1); + Assert.assertNotNull(fo); + } + + @Test + public void testExistsFails() throws Exception { + FileObject fo = manager.resolveFile(TEST_FILE1); + Assert.assertNotNull(fo); + Assert.assertFalse(fo.exists()); + } + + @Test + public void testExistsSucceeds() throws Exception { + FileObject fo = manager.resolveFile(TEST_FILE1); + Assert.assertNotNull(fo); + Assert.assertFalse(fo.exists()); + + //Create the file + @SuppressWarnings("unused") + FileObject f = createTestFile(hdfs); + + } + + @Test(expected=UnsupportedOperationException.class) + public void testCanRenameTo() throws Exception { + FileObject fo = createTestFile(this.hdfs); + Assert.assertNotNull(fo); + fo.canRenameTo(fo); + } + + @Test + public void testDoListChildren() throws Exception { + FileObject fo = manager.resolveFile(TEST_DIR1); + Assert.assertNotNull(fo); + Assert.assertFalse(fo.exists()); + + //Create the test file + FileObject file = createTestFile(hdfs); + FileObject dir = file.getParent(); + + FileObject[] children = dir.getChildren(); + Assert.assertTrue(children.length == 1); + Assert.assertTrue(children[0].getName().equals(file.getName())); + + } + + @Test + public void testGetContentSize() throws Exception { + FileObject fo = manager.resolveFile(TEST_DIR1); + Assert.assertNotNull(fo); + Assert.assertFalse(fo.exists()); + + //Create the test file + FileObject file = createTestFile(hdfs); + Assert.assertEquals(0, file.getContent().getSize()); + } + + @Test + public void testGetInputStream() throws Exception { + FileObject fo = manager.resolveFile(TEST_DIR1); + Assert.assertNotNull(fo); + Assert.assertFalse(fo.exists()); + + //Create the test file + FileObject file = createTestFile(hdfs); + file.getContent().getInputStream().close(); + } + + @Test + public void testIsHidden() throws Exception { + FileObject fo = manager.resolveFile(TEST_DIR1); + Assert.assertNotNull(fo); + Assert.assertFalse(fo.exists()); + + //Create the test file + FileObject file = createTestFile(hdfs); + Assert.assertFalse(file.isHidden()); + } + + @Test + public void testIsReadable() throws Exception { + FileObject fo = manager.resolveFile(TEST_DIR1); + Assert.assertNotNull(fo); + Assert.assertFalse(fo.exists()); + + //Create the test file + FileObject file = createTestFile(hdfs); + Assert.assertTrue(file.isReadable()); + } + + @Test + public void testIsWritable() throws Exception { + FileObject fo = manager.resolveFile(TEST_DIR1); + Assert.assertNotNull(fo); + Assert.assertFalse(fo.exists()); + + //Create the test file + FileObject file = createTestFile(hdfs); + Assert.assertFalse(file.isWriteable()); + } + + @Test + public void testLastModificationTime() throws Exception { + FileObject fo = manager.resolveFile(TEST_DIR1); + Assert.assertNotNull(fo); + Assert.assertFalse(fo.exists()); + + //Create the test file + FileObject file = createTestFile(hdfs); + Assert.assertFalse(-1 == file.getContent().getLastModifiedTime()); + } + + @Test + public void testGetAttributes() throws Exception { + FileObject fo = manager.resolveFile(TEST_DIR1); + Assert.assertNotNull(fo); + Assert.assertFalse(fo.exists()); + + //Create the test file + FileObject file = createTestFile(hdfs); + Map attributes = file.getContent().getAttributes(); + Assert.assertTrue(attributes.containsKey(HdfsFileObject.BLOCK_SIZE)); + Assert.assertTrue(attributes.containsKey(HdfsFileObject.GROUP)); + Assert.assertTrue(attributes.containsKey(HdfsFileObject.LAST_ACCESS_TIME)); + Assert.assertTrue(attributes.containsKey(HdfsFileObject.LENGTH)); + Assert.assertTrue(attributes.containsKey(HdfsFileObject.MODIFICATION_TIME)); + Assert.assertTrue(attributes.containsKey(HdfsFileObject.OWNER)); + Assert.assertTrue(attributes.containsKey(HdfsFileObject.PERMISSIONS)); + } + + @Test(expected=FileSystemException.class) + public void testRandomAccessContent() throws Exception { + FileObject fo = manager.resolveFile(TEST_DIR1); + Assert.assertNotNull(fo); + Assert.assertFalse(fo.exists()); + + //Create the test file + FileObject file = createTestFile(hdfs); + file.getContent().getRandomAccessContent(RandomAccessMode.READWRITE).close(); + } + + @Test + public void testRandomAccessContent2() throws Exception { + FileObject fo = manager.resolveFile(TEST_DIR1); + Assert.assertNotNull(fo); + Assert.assertFalse(fo.exists()); + + //Create the test file + FileObject file = createTestFile(hdfs); + file.getContent().getRandomAccessContent(RandomAccessMode.READ).close(); + } + + @Test + public void testEquals() throws Exception { + FileObject fo = manager.resolveFile(TEST_DIR1); + Assert.assertNotNull(fo); + Assert.assertFalse(fo.exists()); + + //Create the test file + FileObject file = createTestFile(hdfs); + //Get a handle to the same file + FileObject file2 = manager.resolveFile(TEST_FILE1); + Assert.assertEquals(file, file2); + } + + @After + public void tearDown() throws Exception { + if (null != hdfs) { + hdfs.delete(DIR1_PATH, true); + hdfs.close(); + } + manager.close(); + } + +} Added: accumulo/trunk/start/src/test/java/org/apache/commons/vfs2/provider/VfsClassLoaderTest.java URL: http://svn.apache.org/viewvc/accumulo/trunk/start/src/test/java/org/apache/commons/vfs2/provider/VfsClassLoaderTest.java?rev=1412398&view=auto ============================================================================== --- accumulo/trunk/start/src/test/java/org/apache/commons/vfs2/provider/VfsClassLoaderTest.java (added) +++ accumulo/trunk/start/src/test/java/org/apache/commons/vfs2/provider/VfsClassLoaderTest.java Thu Nov 22 00:55:22 2012 @@ -0,0 +1,145 @@ +/* + * 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.commons.vfs2.provider; + +import java.net.URL; + +import org.apache.accumulo.test.AccumuloDFSBase; +import org.apache.commons.vfs2.FileChangeEvent; +import org.apache.commons.vfs2.FileListener; +import org.apache.commons.vfs2.FileObject; +import org.apache.commons.vfs2.impl.DefaultFileMonitor; +import org.apache.commons.vfs2.impl.VFSClassLoader; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class VfsClassLoaderTest extends AccumuloDFSBase { + + private static final Path TEST_DIR = new Path(HDFS_URI + "/test-dir"); + + private FileSystem hdfs = null; + private VFSClassLoader cl = null; + + @Before + public void setup() throws Exception { + + this.hdfs = cluster.getFileSystem(); + this.hdfs.mkdirs(TEST_DIR); + + //Copy jar file to TEST_DIR + URL jarPath = this.getClass().getResource("/HelloWorld.jar"); + Path src = new Path(jarPath.toURI().toString()); + Path dst = new Path(TEST_DIR, src.getName()); + this.hdfs.copyFromLocalFile(src, dst); + + + FileObject testDir = vfs.resolveFile(TEST_DIR.toUri().toString()); + FileObject[] dirContents = testDir.getChildren(); + + //Point the VFSClassLoader to all of the objects in TEST_DIR + this.cl = new VFSClassLoader(dirContents, vfs); + } + + @Test + public void testGetClass() throws Exception { + Class helloWorldClass = this.cl.loadClass("test.HelloWorld"); + Object o = helloWorldClass.newInstance(); + Assert.assertEquals("Hello World!", o.toString()); + } + + @Test + public void testFileMonitor() throws Exception { + MyFileMonitor listener = new MyFileMonitor(); + DefaultFileMonitor monitor = new DefaultFileMonitor(listener); + monitor.setRecursive(true); + FileObject testDir = vfs.resolveFile(TEST_DIR.toUri().toString()); + monitor.addFile(testDir); + monitor.start(); + + //Copy jar file to a new file name + URL jarPath = this.getClass().getResource("/HelloWorld.jar"); + Path src = new Path(jarPath.toURI().toString()); + Path dst = new Path(TEST_DIR, "HelloWorld2.jar"); + this.hdfs.copyFromLocalFile(src, dst); + + Thread.sleep(2000); + Assert.assertTrue(listener.isFileCreated()); + + //Update the jar + jarPath = this.getClass().getResource("/HelloWorld.jar"); + src = new Path(jarPath.toURI().toString()); + dst = new Path(TEST_DIR, "HelloWorld2.jar"); + this.hdfs.copyFromLocalFile(src, dst); + + Thread.sleep(2000); + Assert.assertTrue(listener.isFileChanged()); + + this.hdfs.delete(dst, false); + Thread.sleep(2000); + Assert.assertTrue(listener.isFileDeleted()); + + monitor.stop(); + + } + + + @After + public void tearDown() throws Exception { + this.hdfs.delete(TEST_DIR, true); + } + + + public static class MyFileMonitor implements FileListener { + + private boolean fileChanged = false; + private boolean fileDeleted = false; + private boolean fileCreated = false; + + public void fileCreated(FileChangeEvent event) throws Exception { + //System.out.println(event.getFile() + " created"); + this.fileCreated = true; + } + + public void fileDeleted(FileChangeEvent event) throws Exception { + //System.out.println(event.getFile() + " deleted"); + this.fileDeleted = true; + } + + public void fileChanged(FileChangeEvent event) throws Exception { + //System.out.println(event.getFile() + " changed"); + this.fileChanged = true; + } + + public boolean isFileChanged() { + return fileChanged; + } + + public boolean isFileDeleted() { + return fileDeleted; + } + + public boolean isFileCreated() { + return fileCreated; + } + + + } +} Added: accumulo/trunk/start/src/test/resources/HelloWorld.jar URL: http://svn.apache.org/viewvc/accumulo/trunk/start/src/test/resources/HelloWorld.jar?rev=1412398&view=auto ============================================================================== Binary file - no diff available. Propchange: accumulo/trunk/start/src/test/resources/HelloWorld.jar ------------------------------------------------------------------------------ svn:mime-type = application/octet-stream Added: accumulo/trunk/start/src/test/resources/application1/conf/accumulo-site.xml URL: http://svn.apache.org/viewvc/accumulo/trunk/start/src/test/resources/application1/conf/accumulo-site.xml?rev=1412398&view=auto ============================================================================== --- accumulo/trunk/start/src/test/resources/application1/conf/accumulo-site.xml (added) +++ accumulo/trunk/start/src/test/resources/application1/conf/accumulo-site.xml Thu Nov 22 00:55:22 2012 @@ -0,0 +1,120 @@ + + + + + + + + + instance.zookeeper.host + localhost:2181 + comma separated list of zookeeper servers + + + + logger.dir.walog + walogs + The directory used to store write-ahead logs on the local + filesystem. It is possible to specify a comma-separated list of + directories. + + + + instance.secret + DEFAULT + A secret unique to a given instance that all servers must + know in order to communicate with one another. + Change it before initialization. To change it later use ./bin/accumulo + org.apache.accumulo.server.util.ChangeSecret [oldpasswd] [newpasswd], + and then update this file. + + + + + tserver.memory.maps.max + 80M + + + + tserver.cache.data.size + 7M + + + + tserver.cache.index.size + 20M + + + + trace.password + + secret + + + + trace.user + root + + + + tserver.sort.buffer.size + 50M + + + + tserver.walog.max.size + 100M + + + + general.classpaths + + $ACCUMULO_HOME/server/target/classes/, + $ACCUMULO_HOME/core/target/classes/, + $ACCUMULO_HOME/start/target/classes/, + $ACCUMULO_HOME/examples/target/classes/, + $ACCUMULO_HOME/lib/[^.].$ACCUMULO_VERSION.jar, + $ACCUMULO_HOME/lib/[^.].*.jar, + $ZOOKEEPER_HOME/zookeeper[^.].*.jar, + $HADOOP_HOME/conf, + $HADOOP_HOME/[^.].*.jar, + $HADOOP_HOME/lib/[^.].*.jar, + + Classpaths that accumulo checks for updates and class + files. + When using the Security Manager, please remove the ".../target/classes/" + values. + + + + + + classloader.vfs.enabled + true + + + classloader.vfs.context.classpath.system + hdfs://localhost:8020/accumulo/classpath + + + classloader.vfs.context.names + application1 + + + classloader.vfs.context.classpath.application1 + hdfs://localhost:8020/application1/classpath + + + Added: accumulo/trunk/start/src/test/resources/default/conf/accumulo-site.xml URL: http://svn.apache.org/viewvc/accumulo/trunk/start/src/test/resources/default/conf/accumulo-site.xml?rev=1412398&view=auto ============================================================================== --- accumulo/trunk/start/src/test/resources/default/conf/accumulo-site.xml (added) +++ accumulo/trunk/start/src/test/resources/default/conf/accumulo-site.xml Thu Nov 22 00:55:22 2012 @@ -0,0 +1,112 @@ + + + + + + + + + instance.zookeeper.host + localhost:2181 + comma separated list of zookeeper servers + + + + logger.dir.walog + walogs + The directory used to store write-ahead logs on the local + filesystem. It is possible to specify a comma-separated list of + directories. + + + + instance.secret + DEFAULT + A secret unique to a given instance that all servers must + know in order to communicate with one another. + Change it before initialization. To change it later use ./bin/accumulo + org.apache.accumulo.server.util.ChangeSecret [oldpasswd] [newpasswd], + and then update this file. + + + + + tserver.memory.maps.max + 80M + + + + tserver.cache.data.size + 7M + + + + tserver.cache.index.size + 20M + + + + trace.password + + secret + + + + trace.user + root + + + + tserver.sort.buffer.size + 50M + + + + tserver.walog.max.size + 100M + + + + general.classpaths + + $ACCUMULO_HOME/server/target/classes/, + $ACCUMULO_HOME/core/target/classes/, + $ACCUMULO_HOME/start/target/classes/, + $ACCUMULO_HOME/examples/target/classes/, + $ACCUMULO_HOME/lib/[^.].$ACCUMULO_VERSION.jar, + $ACCUMULO_HOME/lib/[^.].*.jar, + $ZOOKEEPER_HOME/zookeeper[^.].*.jar, + $HADOOP_HOME/conf, + $HADOOP_HOME/[^.].*.jar, + $HADOOP_HOME/lib/[^.].*.jar, + + Classpaths that accumulo checks for updates and class + files. + When using the Security Manager, please remove the ".../target/classes/" + values. + + + + + + classloader.vfs.enabled + true + + + classloader.vfs.context.classpath.system + hdfs://localhost:8020/accumulo/classpath + + + Added: accumulo/trunk/start/src/test/resources/disabled/conf/accumulo-site.xml URL: http://svn.apache.org/viewvc/accumulo/trunk/start/src/test/resources/disabled/conf/accumulo-site.xml?rev=1412398&view=auto ============================================================================== --- accumulo/trunk/start/src/test/resources/disabled/conf/accumulo-site.xml (added) +++ accumulo/trunk/start/src/test/resources/disabled/conf/accumulo-site.xml Thu Nov 22 00:55:22 2012 @@ -0,0 +1,108 @@ + + + + + + + + + instance.zookeeper.host + localhost:2181 + comma separated list of zookeeper servers + + + + logger.dir.walog + walogs + The directory used to store write-ahead logs on the local + filesystem. It is possible to specify a comma-separated list of + directories. + + + + instance.secret + DEFAULT + A secret unique to a given instance that all servers must + know in order to communicate with one another. + Change it before initialization. To change it later use ./bin/accumulo + org.apache.accumulo.server.util.ChangeSecret [oldpasswd] [newpasswd], + and then update this file. + + + + + tserver.memory.maps.max + 80M + + + + tserver.cache.data.size + 7M + + + + tserver.cache.index.size + 20M + + + + trace.password + + secret + + + + trace.user + root + + + + tserver.sort.buffer.size + 50M + + + + tserver.walog.max.size + 100M + + + + general.classpaths + + $ACCUMULO_HOME/server/target/classes/, + $ACCUMULO_HOME/core/target/classes/, + $ACCUMULO_HOME/start/target/classes/, + $ACCUMULO_HOME/examples/target/classes/, + $ACCUMULO_HOME/lib/[^.].$ACCUMULO_VERSION.jar, + $ACCUMULO_HOME/lib/[^.].*.jar, + $ZOOKEEPER_HOME/zookeeper[^.].*.jar, + $HADOOP_HOME/conf, + $HADOOP_HOME/[^.].*.jar, + $HADOOP_HOME/lib/[^.].*.jar, + + Classpaths that accumulo checks for updates and class + files. + When using the Security Manager, please remove the ".../target/classes/" + values. + + + + + + classloader.vfs.enabled + true + + + Added: accumulo/trunk/test/src/test/resources/conf/accumulo-site.xml URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/resources/conf/accumulo-site.xml?rev=1412398&view=auto ============================================================================== --- accumulo/trunk/test/src/test/resources/conf/accumulo-site.xml (added) +++ accumulo/trunk/test/src/test/resources/conf/accumulo-site.xml Thu Nov 22 00:55:22 2012 @@ -0,0 +1,109 @@ + + + + + + + + + instance.zookeeper.host + localhost:2181 + comma separated list of zookeeper servers + + + + logger.dir.walog + walogs + The directory used to store write-ahead logs on the local filesystem. It is possible to specify a comma-separated list of directories. + + + + instance.secret + DEFAULT + A secret unique to a given instance that all servers must know in order to communicate with one another. + Change it before initialization. To change it later use ./bin/accumulo org.apache.accumulo.server.util.ChangeSecret [oldpasswd] [newpasswd], + and then update this file. + + + + + tserver.memory.maps.max + 80M + + + + tserver.cache.data.size + 7M + + + + tserver.cache.index.size + 20M + + + + trace.password + + secret + + + + trace.user + root + + + + tserver.sort.buffer.size + 50M + + + + tserver.walog.max.size + 100M + + + + general.classpaths + + $ACCUMULO_HOME/server/target/classes/, + $ACCUMULO_HOME/core/target/classes/, + $ACCUMULO_HOME/start/target/classes/, + $ACCUMULO_HOME/examples/target/classes/, + $ACCUMULO_HOME/lib/[^.].$ACCUMULO_VERSION.jar, + $ACCUMULO_HOME/lib/[^.].*.jar, + $ZOOKEEPER_HOME/zookeeper[^.].*.jar, + $HADOOP_HOME/conf, + $HADOOP_HOME/[^.].*.jar, + $HADOOP_HOME/lib/[^.].*.jar, + + Classpaths that accumulo checks for updates and class files. + When using the Security Manager, please remove the ".../target/classes/" values. + + + +