Return-Path: X-Original-To: apmail-ambari-commits-archive@www.apache.org Delivered-To: apmail-ambari-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 6CAE219454 for ; Thu, 7 Apr 2016 08:00:27 +0000 (UTC) Received: (qmail 46221 invoked by uid 500); 7 Apr 2016 08:00:27 -0000 Delivered-To: apmail-ambari-commits-archive@ambari.apache.org Received: (qmail 46189 invoked by uid 500); 7 Apr 2016 08:00:27 -0000 Mailing-List: contact commits-help@ambari.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: ambari-dev@ambari.apache.org Delivered-To: mailing list commits@ambari.apache.org Received: (qmail 46176 invoked by uid 99); 7 Apr 2016 08:00:27 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 07 Apr 2016 08:00:27 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 2C632DFDE0; Thu, 7 Apr 2016 08:00:27 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: pallavkul@apache.org To: commits@ambari.apache.org Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: ambari git commit: AMBARI-14084. Views: Provide refresh list of available views with newly deployed views w/o restart (Ashwin Rajeev via pallavkul) Date: Thu, 7 Apr 2016 08:00:27 +0000 (UTC) Repository: ambari Updated Branches: refs/heads/trunk e68968dd9 -> 1df39c324 AMBARI-14084. Views: Provide refresh list of available views with newly deployed views w/o restart (Ashwin Rajeev via pallavkul) Project: http://git-wip-us.apache.org/repos/asf/ambari/repo Commit: http://git-wip-us.apache.org/repos/asf/ambari/commit/1df39c32 Tree: http://git-wip-us.apache.org/repos/asf/ambari/tree/1df39c32 Diff: http://git-wip-us.apache.org/repos/asf/ambari/diff/1df39c32 Branch: refs/heads/trunk Commit: 1df39c324fc97d4db4cc19023d1e8aed92efc36d Parents: e68968d Author: Pallav Kulshreshtha Authored: Thu Apr 7 13:29:37 2016 +0530 Committer: Pallav Kulshreshtha Committed: Thu Apr 7 13:29:37 2016 +0530 ---------------------------------------------------------------------- .../controllers/ambariViews/ViewsListCtrl.js | 5 + .../app/views/ambariViews/listTable.html | 4 + .../ambari/server/controller/AmbariServer.java | 5 + .../ambari/server/view/DirectoryWatcher.java | 30 +++ .../server/view/ViewDirectoryWatcher.java | 265 +++++++++++++++++++ .../apache/ambari/server/view/ViewRegistry.java | 36 ++- .../server/view/ViewDirectoryWatcherTest.java | 163 ++++++++++++ 7 files changed, 507 insertions(+), 1 deletion(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ambari/blob/1df39c32/ambari-admin/src/main/resources/ui/admin-web/app/scripts/controllers/ambariViews/ViewsListCtrl.js ---------------------------------------------------------------------- diff --git a/ambari-admin/src/main/resources/ui/admin-web/app/scripts/controllers/ambariViews/ViewsListCtrl.js b/ambari-admin/src/main/resources/ui/admin-web/app/scripts/controllers/ambariViews/ViewsListCtrl.js index 75f6198..6d1dc52 100644 --- a/ambari-admin/src/main/resources/ui/admin-web/app/scripts/controllers/ambariViews/ViewsListCtrl.js +++ b/ambari-admin/src/main/resources/ui/admin-web/app/scripts/controllers/ambariViews/ViewsListCtrl.js @@ -125,4 +125,9 @@ angular.module('ambariAdminConsole') $location.path('/views/'+viewName+'/new'); } }; + + $scope.reloadViews = function () { + loadViews(); + } + }]); \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ambari/blob/1df39c32/ambari-admin/src/main/resources/ui/admin-web/app/views/ambariViews/listTable.html ---------------------------------------------------------------------- diff --git a/ambari-admin/src/main/resources/ui/admin-web/app/views/ambariViews/listTable.html b/ambari-admin/src/main/resources/ui/admin-web/app/views/ambariViews/listTable.html index 4b87070..ae71d78 100644 --- a/ambari-admin/src/main/resources/ui/admin-web/app/views/ambariViews/listTable.html +++ b/ambari-admin/src/main/resources/ui/admin-web/app/views/ambariViews/listTable.html @@ -20,6 +20,10 @@
http://git-wip-us.apache.org/repos/asf/ambari/blob/1df39c32/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariServer.java ---------------------------------------------------------------------- diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariServer.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariServer.java index 5802f08..0e45f79 100644 --- a/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariServer.java +++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariServer.java @@ -118,6 +118,7 @@ import org.apache.ambari.server.utils.AmbariPath; import org.apache.ambari.server.utils.RetryHelper; import org.apache.ambari.server.utils.StageUtils; import org.apache.ambari.server.utils.VersionUtils; +import org.apache.ambari.server.view.ViewDirectoryWatcher; import org.apache.ambari.server.view.ViewRegistry; import org.apache.velocity.app.Velocity; import org.eclipse.jetty.http.HttpVersion; @@ -260,6 +261,9 @@ public class AmbariServer { @Inject DelegatingFilterProxy springSecurityFilter; + @Inject + ViewDirectoryWatcher viewDirectoryWatcher; + public String getServerOsType() { return configs.getServerOsType(); } @@ -484,6 +488,7 @@ public class AmbariServer { SecurityContextHolder.setStrategyName(SecurityContextHolder.MODE_INHERITABLETHREADLOCAL); viewRegistry.readViewArchives(); + viewDirectoryWatcher.start(); handlerList.addHandler(root); server.setHandler(handlerList); http://git-wip-us.apache.org/repos/asf/ambari/blob/1df39c32/ambari-server/src/main/java/org/apache/ambari/server/view/DirectoryWatcher.java ---------------------------------------------------------------------- diff --git a/ambari-server/src/main/java/org/apache/ambari/server/view/DirectoryWatcher.java b/ambari-server/src/main/java/org/apache/ambari/server/view/DirectoryWatcher.java new file mode 100644 index 0000000..6bc8b39 --- /dev/null +++ b/ambari-server/src/main/java/org/apache/ambari/server/view/DirectoryWatcher.java @@ -0,0 +1,30 @@ +/** + * 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.ambari.server.view; + +public interface DirectoryWatcher { + + void start(); + + boolean isRunning(); + + void stop(); + +} http://git-wip-us.apache.org/repos/asf/ambari/blob/1df39c32/ambari-server/src/main/java/org/apache/ambari/server/view/ViewDirectoryWatcher.java ---------------------------------------------------------------------- diff --git a/ambari-server/src/main/java/org/apache/ambari/server/view/ViewDirectoryWatcher.java b/ambari-server/src/main/java/org/apache/ambari/server/view/ViewDirectoryWatcher.java new file mode 100644 index 0000000..c3d443a --- /dev/null +++ b/ambari-server/src/main/java/org/apache/ambari/server/view/ViewDirectoryWatcher.java @@ -0,0 +1,265 @@ +/** + * 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.ambari.server.view; + +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import com.google.inject.Inject; +import com.google.inject.Singleton; +import org.apache.ambari.server.configuration.Configuration; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardWatchEventKinds; +import java.nio.file.WatchEvent; +import java.nio.file.WatchKey; +import java.nio.file.WatchService; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.zip.ZipFile; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.lang.Thread.sleep; + +@Singleton +public class ViewDirectoryWatcher implements DirectoryWatcher { + + public static final int FIXED_FILE_COUNTER = 30; + public static final int FILE_CHECK_INTERVAL_MILLIS = 200; + // Global configuration + @Inject + Configuration configuration; + + // View Registry + @Inject + ViewRegistry viewRegistry; + + private WatchService watchService; + + // Executor service on which the watcher will run + private ExecutorService executorService = Executors.newSingleThreadExecutor(); + + private Future watchTask; + + private static Log LOG = LogFactory.getLog(ViewDirectoryWatcher.class); + + // Callbacks to hook into file processing + private List> hooks = Lists.newArrayList(loggingHook()); + + public void addHook(Function hook) { + hooks.add(hook); + } + + private Function loggingHook() { + return new Function() { + @Nullable + @Override + public Boolean apply(@Nullable Path path) { + LOG.info("Finished processing the view definition for" + path); + return true; + } + }; + } + + public void start() { + + try { + Path path = buildWatchService(); + Runnable task = startWatching(path); + watchTask = executorService.submit(task); + } catch (Exception e) { + LOG.error("There were errors in starting the view directory watcher. This task will not run", e); + } + } + + + @SuppressWarnings("unchecked") + private static WatchEvent cast(WatchEvent event) { + return (WatchEvent) event; + } + + private Runnable startWatching(final Path path) { + return new Runnable() { + @Override + public void run() { + try { + while (true) { + // wait for key , park the thread meanwhile + WatchKey key = watchService.take(); + LOG.info("Watcher Key was signalled"); + for (WatchEvent event : key.pollEvents()) { + LOG.info("Watcher recieved poll event"); + WatchEvent ev = cast(event); + Path resolvedPath = path.resolve(ev.context()); + LOG.info(String.format("Event %s: %s\n", ev.kind(), resolvedPath)); + if (!canBlockTillFileAvailable(resolvedPath)) { + LOG.info("Watcher detected that the file was either empty or corrupt"); + continue; + } + if (!verify(resolvedPath)) { + LOG.info("The uploaded file was 1> Empty 2> Not a regular file or 3> Not a valid Jar archive file"); + continue; + } + try { + LOG.info("Starting view extraction"); + viewRegistry.readViewArchive(resolvedPath); + // fire registered hooks + for (Function hook : hooks) { + hook.apply(resolvedPath); + } + } catch (Exception e) { + LOG.error("Cannot read the view archive, offending file: " + resolvedPath, e); + } + + } + + // reset key + if (!key.reset()) { + //watch key is invalid, break out + LOG.error("The watch key could not be reset, Directory watcher will not run anymore"); + break; + } + + + } + } catch (InterruptedException x) { + LOG.info("Cancelling the directory watcher", x); + return; + } + + } + }; + } + + + /** + * Routine to make the file watcher block the thread till the file is completely copied + * Check the length of the file continuously till there are 20 consecutive intervals when + * the file length does not change + * FILE_CHECK_INTERVAL_MILLIS defines the check interval both for detecting empty files + * and subsequent checks to detect if a file has finished copying + * + * The process which copies the jar into the views dir is external and we dont really + * know when it would finish, this is also highly OS and FS dependent. The following routine + * introduces a heuristic to detect when a file has finished copying by looking at subsequent + * lengths of the file which was detected as being created + * + * This would block for ~ 7 seconds in most cases + * + * + * @param resolvedPath + * @return false if the file check failed, true otherwise + */ + private boolean canBlockTillFileAvailable(Path resolvedPath) throws InterruptedException { + long oldLength; + long newSize; + + long emptyCheck = 0; + int fixed = 0; + // get the underlying file + File file = resolvedPath.toAbsolutePath().toFile(); + + // empty file check + while (file.length() == 0 && emptyCheck < 5) { + sleep(FILE_CHECK_INTERVAL_MILLIS); + emptyCheck++; + } + // The file seems to be empty + if (emptyCheck == 5) + return false; + + // check the file size + oldLength = file.length(); + + // Check if file copy is done + while (true) { + LOG.info("Waiting for file to be completely copied"); + sleep(FILE_CHECK_INTERVAL_MILLIS); + newSize = file.length(); + if (newSize > oldLength) { + oldLength = newSize; + continue; + } else if (oldLength == newSize) { + fixed++; + } else { + // this can never happen, + return false; + } + if (fixed > FIXED_FILE_COUNTER) { + LOG.info("File " + resolvedPath + " has finished copying"); + return true; + } + } + + + } + + /** + * Sanity check to validate if the detected path is a valid archive file + * is not a directory, also check that the file is not empty + * + * @param resolvedPath + * @return + */ + private boolean verify(Path resolvedPath) { + try { + File file = resolvedPath.toAbsolutePath().toFile(); + checkArgument(!file.isDirectory()); + checkArgument(file.length() > 0); + new ZipFile(file); + } catch (Exception e) { + LOG.info("Verification failed ", e); + return false; + } + return true; + } + + private Path buildWatchService() throws IOException { + // Get the directory for view Archives + //Attach a file watcher at this directory, Extracted work directory will be ignored + File viewsDir = configuration.getViewsDir(); + Path path = Paths.get(viewsDir.getAbsolutePath()); + + watchService = path.getFileSystem().newWatchService(); + //Watch vews directory for creation events + path.register(watchService, StandardWatchEventKinds.ENTRY_CREATE); + return path; + + } + + @Override + public boolean isRunning() { + if (watchTask != null) + return !(watchTask.isDone()); + return false; + } + + @Override + public void stop() { + watchTask.cancel(true); + } + +} http://git-wip-us.apache.org/repos/asf/ambari/blob/1df39c32/ambari-server/src/main/java/org/apache/ambari/server/view/ViewRegistry.java ---------------------------------------------------------------------- diff --git a/ambari-server/src/main/java/org/apache/ambari/server/view/ViewRegistry.java b/ambari-server/src/main/java/org/apache/ambari/server/view/ViewRegistry.java index d9b1fac..037d1e5 100644 --- a/ambari-server/src/main/java/org/apache/ambari/server/view/ViewRegistry.java +++ b/ambari-server/src/main/java/org/apache/ambari/server/view/ViewRegistry.java @@ -102,6 +102,7 @@ import javax.inject.Singleton; import java.beans.IntrospectionException; import java.io.File; import java.net.URL; +import java.nio.file.Path; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -1424,12 +1425,43 @@ public class ViewRegistry { privilegeDAO.remove(privilegeEntity); } + + /** + * Extract a view archive at the specified path + * @param path + */ + public void readViewArchive(Path path) { + + File viewDir = configuration.getViewsDir(); + String extractedArchivesPath = viewDir.getAbsolutePath() + + File.separator + EXTRACTED_ARCHIVES_DIR; + + File archiveFile = path.toAbsolutePath().toFile(); + if (extractor.ensureExtractedArchiveDirectory(extractedArchivesPath)) { + try { + final ViewConfig viewConfig = archiveUtility.getViewConfigFromArchive(archiveFile); + String viewName = ViewEntity.getViewName(viewConfig.getName(), viewConfig.getVersion()); + final String extractedArchiveDirPath = extractedArchivesPath + File.separator + viewName; + final File extractedArchiveDirFile = archiveUtility.getFile(extractedArchiveDirPath); + final ViewEntity viewDefinition = new ViewEntity(viewConfig, configuration, extractedArchiveDirPath); + addDefinition(viewDefinition); + readViewArchive(viewDefinition, archiveFile, extractedArchiveDirFile, ambariMetaInfoProvider.get().getServerVersion()); + } catch (Exception e){ + LOG.error("Could not process archive at path "+path, e); + } + } + + } + + + + // read the view archives. private void readViewArchives(boolean systemOnly, boolean useExecutor, String viewNameRegExp) { try { - File viewDir = configuration.getViewsDir(); + File viewDir = configuration.getViewsDir(); String extractedArchivesPath = viewDir.getAbsolutePath() + File.separator + EXTRACTED_ARCHIVES_DIR; @@ -1767,6 +1799,8 @@ public class ViewRegistry { return new ViewAmbariStreamProvider(streamProvider, ambariSessionManager, AmbariServer.getController()); } + + /** * Module for stand alone view registry. */ http://git-wip-us.apache.org/repos/asf/ambari/blob/1df39c32/ambari-server/src/test/java/org/apache/ambari/server/view/ViewDirectoryWatcherTest.java ---------------------------------------------------------------------- diff --git a/ambari-server/src/test/java/org/apache/ambari/server/view/ViewDirectoryWatcherTest.java b/ambari-server/src/test/java/org/apache/ambari/server/view/ViewDirectoryWatcherTest.java new file mode 100644 index 0000000..d9f8919 --- /dev/null +++ b/ambari-server/src/test/java/org/apache/ambari/server/view/ViewDirectoryWatcherTest.java @@ -0,0 +1,163 @@ +/** + * 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.ambari.server.view; + +import com.google.common.base.Function; +import org.apache.ambari.server.configuration.Configuration; +import org.apache.commons.io.FileDeleteStrategy; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.concurrent.CountDownLatch; +import java.util.zip.ZipEntry; +import java.util.zip.ZipOutputStream; + +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.easymock.EasyMock.*; + +public class ViewDirectoryWatcherTest { + + private static final Configuration configuration = createNiceMock(Configuration.class); + private static final ViewRegistry viewRegistry = createNiceMock(ViewRegistry.class); + private File testDir; + + + @Before + public void setUp() throws Exception { + reset(configuration, viewRegistry); + testDir = new File(System.getProperty("java.io.tmpdir"), "test_dir"); + if (testDir.exists()) { + FileDeleteStrategy.FORCE.delete(testDir); + } + testDir.mkdirs(); + + } + + + @Test + public void testDirectoryWatcherStart() throws Exception { + ViewDirectoryWatcher viewDirectoryWatcher = new ViewDirectoryWatcher(); + + expect(configuration.getViewsDir()).andReturn(testDir).once(); + viewDirectoryWatcher.configuration = configuration; + viewDirectoryWatcher.viewRegistry = viewRegistry; + replay(configuration); + final CountDownLatch countDownLatch = new CountDownLatch(1); + viewDirectoryWatcher.addHook(new Function() { + @Nullable + @Override + public Boolean apply(@Nullable Path path) { + countDownLatch.countDown(); + return true; + } + + }); + viewDirectoryWatcher.start(); + countDownLatch.await(1, SECONDS); + // Expect watecher to start + Assert.assertTrue(viewDirectoryWatcher.isRunning()); + verify(configuration); + } + + + @Test + public void testDirectoryExtractionOnFileAdd() throws Exception { + ViewDirectoryWatcher viewDirectoryWatcher = new ViewDirectoryWatcher(); + expect(configuration.getViewsDir()).andReturn(testDir).once(); + viewDirectoryWatcher.configuration = configuration; + viewDirectoryWatcher.viewRegistry = viewRegistry; + viewRegistry.readViewArchive(Paths.get(testDir.getAbsolutePath(), "file.jar")); + replay(configuration, viewRegistry); + final CountDownLatch countDownLatch = new CountDownLatch(1); + viewDirectoryWatcher.addHook(new Function() { + @Nullable + @Override + public Boolean apply(@Nullable Path path) { + countDownLatch.countDown(); + return true; + } + }); + viewDirectoryWatcher.start(); + // Create a new File at destination + createZipFile(); + countDownLatch.await(7, SECONDS); + + // Expect watcher to respond + verify(configuration, viewRegistry); + } + + + @Test + public void testDirectoryWatcherStop() throws Exception { + + ViewDirectoryWatcher viewDirectoryWatcher = new ViewDirectoryWatcher(); + expect(configuration.getViewsDir()).andReturn(testDir).once(); + viewDirectoryWatcher.configuration = configuration; + viewDirectoryWatcher.viewRegistry = viewRegistry; + replay(configuration); + + viewDirectoryWatcher.start(); + //Time to start + Thread.sleep(100); + viewDirectoryWatcher.stop(); + Assert.assertFalse(viewDirectoryWatcher.isRunning()); + verify(configuration); + } + + + private void createZipFile() throws IOException { + File file = new File(System.getProperty("java.io.tmpdir") + File.separator + "view.xml"); + file.createNewFile(); + + // input file + FileInputStream in = new FileInputStream(file); + + // out put file + ZipOutputStream out = new ZipOutputStream(new FileOutputStream(new File(testDir, "file.jar"))); + + // name the file inside the zip file + out.putNextEntry(new ZipEntry("view.xml")); + + // buffer size + byte[] b = new byte[1024]; + int count; + + while ((count = in.read(b)) > 0) { + System.out.println(); + out.write(b, 0, count); + } + out.close(); + in.close(); + + } + +} + + + +