Return-Path: X-Original-To: apmail-ignite-commits-archive@minotaur.apache.org Delivered-To: apmail-ignite-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 2D2B118376 for ; Fri, 31 Jul 2015 02:28:48 +0000 (UTC) Received: (qmail 52517 invoked by uid 500); 31 Jul 2015 02:28:48 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 52487 invoked by uid 500); 31 Jul 2015 02:28:48 -0000 Mailing-List: contact commits-help@ignite.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.incubator.apache.org Delivered-To: mailing list commits@ignite.incubator.apache.org Received: (qmail 52478 invoked by uid 99); 31 Jul 2015 02:28:48 -0000 Received: from Unknown (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 31 Jul 2015 02:28:48 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd3-us-west.apache.org (ASF Mail Server at spamd3-us-west.apache.org) with ESMTP id 6FBB11955AC for ; Fri, 31 Jul 2015 02:28:47 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 0.791 X-Spam-Level: X-Spam-Status: No, score=0.791 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, T_RP_MATCHES_RCVD=-0.01, URIBL_BLOCKED=0.001] autolearn=disabled Received: from mx1-us-east.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id luFzAIRIdE8n for ; Fri, 31 Jul 2015 02:28:30 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-us-east.apache.org (ASF Mail Server at mx1-us-east.apache.org) with SMTP id DA3E150683 for ; Fri, 31 Jul 2015 02:28:15 +0000 (UTC) Received: (qmail 50892 invoked by uid 99); 31 Jul 2015 02:28:15 -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; Fri, 31 Jul 2015 02:28:15 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 6517CDFC3B; Fri, 31 Jul 2015 02:28:15 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: anovikov@apache.org To: commits@ignite.incubator.apache.org Date: Fri, 31 Jul 2015 02:28:41 -0000 Message-Id: In-Reply-To: <7bd5c4640d6441b3bde52706e7c27616@git.apache.org> References: <7bd5c4640d6441b3bde52706e7c27616@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [28/41] incubator-ignite git commit: # ignite-843 Rename http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96522874/modules/web-control-center/src/main/js/routes/generator/java.js ---------------------------------------------------------------------- diff --git a/modules/web-control-center/src/main/js/routes/generator/java.js b/modules/web-control-center/src/main/js/routes/generator/java.js deleted file mode 100644 index 41b9bb8..0000000 --- a/modules/web-control-center/src/main/js/routes/generator/java.js +++ /dev/null @@ -1,788 +0,0 @@ -/* - * 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. - */ - -var _ = require('lodash'); - -var generatorUtils = require("./common"); - -/** - * Convert some name to valid java name. - * - * @param name to convert. - * @returns {string} Valid java name. - */ -function toJavaName(name) { - var javaName = name.replace(/[^A-Za-z_0-9]+/, '_'); - - return javaName.charAt(0).toLocaleUpperCase() + javaName.slice(1); -} - -exports.generateClusterConfiguration = function(cluster, javaClass, clientNearConfiguration) { - var res = generatorUtils.builder(); - - res.datasourceBeans = []; - - if (javaClass) { - res.line('/**'); - res.line(' * ' + generatorUtils.mainComment()); - res.line(' */'); - res.startBlock('public class ConfigurationFactory {'); - res.line('/**'); - res.line(' * Configure grid.'); - res.line(' */'); - res.startBlock('public IgniteConfiguration createConfiguration() {'); - } - - res.importClass('org.apache.ignite.configuration.IgniteConfiguration'); - - res.line('IgniteConfiguration cfg = new IgniteConfiguration();'); - res.line(); - - if (clientNearConfiguration) { - res.line('cfg.setClientMode(true);'); - - res.line(); - } - - if (cluster.discovery) { - var d = cluster.discovery; - - res.importClass('org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi'); - res.line('TcpDiscoverySpi discovery = new TcpDiscoverySpi();'); - - switch (d.kind) { - case 'Multicast': - res.importClass('org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder'); - - addBeanWithProperties(res, d.Multicast, 'discovery', 'ipFinder', 'ipFinder', - 'TcpDiscoveryMulticastIpFinder', { - multicastGroup: null, - multicastPort: null, - responseWaitTime: null, - addressRequestAttempts: null, - localAddress: null - }, true); - - break; - - case 'Vm': - res.importClass('org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder'); - - addBeanWithProperties(res, d.Vm, 'discovery', 'ipFinder', 'ipFinder', 'TcpDiscoveryVmIpFinder', { - addresses: {type: 'list'} - }, true); - - break; - - case 'S3': - res.importClass('org.apache.ignite.spi.discovery.tcp.ipfinder.s3.TcpDiscoveryS3IpFinder'); - - if (d.S3) - addBeanWithProperties(res, d.S3, 'discovery', 'ipFinder', 'ipFinder', 'TcpDiscoveryS3IpFinder', - {bucketName: null}, true); - else - res.line('discovery.setIpFinder(new TcpDiscoveryS3IpFinder());'); - - break; - - case 'Cloud': - res.importClass('org.apache.ignite.spi.discovery.tcp.ipfinder.cloud.TcpDiscoveryCloudIpFinder'); - - addBeanWithProperties(res, d.Cloud, 'discovery', 'ipFinder', 'ipFinder', 'TcpDiscoveryCloudIpFinder', { - credential: null, - credentialPath: null, - identity: null, - provider: null, - regions: {type: 'list'}, - zones: {type: 'list'} - }, true); - - break; - - case 'GoogleStorage': - res.importClass('org.apache.ignite.spi.discovery.tcp.ipfinder.gce.TcpDiscoveryGoogleStorageIpFinder'); - - addBeanWithProperties(res, d.GoogleStorage, 'discovery', 'ipFinder', 'ipFinder', - 'TcpDiscoveryGoogleStorageIpFinder', { - projectName: null, - bucketName: null, - serviceAccountP12FilePath: null - }, true); - - //if (d.GoogleStorage.addrReqAttempts) todo ???? - // res.line(''); - - break; - - case 'Jdbc': - res.importClass('org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.TcpDiscoveryJdbcIpFinder'); - - res.line(); - res.line('TcpDiscoveryJdbcIpFinder ipFinder = new TcpDiscoveryJdbcIpFinder();'); - res.line('ipFinder.setInitSchema(' + (d.Jdbc.initSchema != null || d.Jdbc.initSchema) + ');'); - res.line('discovery.setIpFinder(ipFinder);'); - res.needEmptyLine = true; - - break; - - case 'SharedFs': - res.importClass('org.apache.ignite.spi.discovery.tcp.ipfinder.sharedfs.TcpDiscoverySharedFsIpFinder'); - - addBeanWithProperties(res, d.SharedFs, 'discovery', 'ipFinder', 'ipFinder', - 'TcpDiscoverySharedFsIpFinder', {path: null}, true); - - break; - - default: - throw "Unknown discovery kind: " + d.kind; - } - - res.emptyLineIfNeeded(); - - res.line('cfg.setDiscoverySpi(discovery);'); - - res.needEmptyLine = true; - } - - if (cluster.caches && cluster.caches.length > 0) { - res.emptyLineIfNeeded(); - - var names = []; - - for (var i = 0; i < cluster.caches.length; i++) { - res.emptyLineIfNeeded(); - - var cache = cluster.caches[i]; - - var cacheName = 'cache' + toJavaName(cache.name); - - names.push(cacheName); - - generateCacheConfiguration(res, cache, cacheName); - - res.needEmptyLine = true; - } - - res.emptyLineIfNeeded(); - - res.append('cfg.setCacheConfiguration('); - - for (i = 0; i < names.length; i++) { - if (i > 0) - res.append(', '); - - res.append(names[i]); - } - - res.line(');'); - - res.needEmptyLine = true; - } - - addBeanWithProperties(res, cluster.atomicConfiguration, 'cfg', 'atomicConfiguration', 'atomicCfg', - generatorUtils.atomicConfiguration.className, generatorUtils.atomicConfiguration.fields); - - res.needEmptyLine = true; - - addProperty(res, cluster, 'cfg', 'networkTimeout'); - addProperty(res, cluster, 'cfg', 'networkSendRetryDelay'); - addProperty(res, cluster, 'cfg', 'networkSendRetryCount'); - addProperty(res, cluster, 'cfg', 'segmentCheckFrequency'); - addProperty(res, cluster, 'cfg', 'waitForSegmentOnStart'); - addProperty(res, cluster, 'cfg', 'discoveryStartupDelay'); - - res.needEmptyLine = true; - - addProperty(res, cluster, 'cfg', 'deploymentMode', 'DeploymentMode'); - - res.needEmptyLine = true; - - if (cluster.includeEventTypes && cluster.includeEventTypes.length > 0) { - res.emptyLineIfNeeded(); - - if (cluster.includeEventTypes.length == 1) { - res.importClass('org.apache.ignite.events.EventType'); - - res.line('cfg.setIncludeEventTypes(EventType.' + cluster.includeEventTypes[0] + ');'); - } - else { - res.append('int[] events = new int[EventType.' + cluster.includeEventTypes[0] + '.length'); - - for (i = 1; i < cluster.includeEventTypes.length; i++) { - res.line(); - - res.append(' + EventType.' + cluster.includeEventTypes[i] + '.length'); - } - - res.line('];'); - res.line(); - res.line('int k = 0;'); - - for (i = 0; i < cluster.includeEventTypes.length; i++) { - res.line(); - - var e = cluster.includeEventTypes[i]; - - res.line('System.arraycopy(EventType.' + e + ', 0, events, k, EventType.' + e + '.length);'); - res.line('k += EventType.' + e + '.length;'); - } - - res.line(); - res.line('cfg.setIncludeEventTypes(events);'); - } - - res.needEmptyLine = true; - } - - res.needEmptyLine = true; - - var marshaller = cluster.marshaller; - - if (marshaller && marshaller.kind) { - var marshallerDesc = generatorUtils.marshallers[marshaller.kind]; - - addBeanWithProperties(res, marshaller[marshaller.kind], 'cfg', 'marshaller', 'marshaller', - marshallerDesc.className, marshallerDesc.fields, true); - - addBeanWithProperties(res, marshaller[marshaller.kind], 'marshaller', marshallerDesc.className, marshallerDesc.fields, true); - } - - addProperty(res, cluster, 'cfg', 'marshalLocalJobs'); - addProperty(res, cluster, 'cfg', 'marshallerCacheKeepAliveTime'); - addProperty(res, cluster, 'cfg', 'marshallerCacheThreadPoolSize'); - - res.needEmptyLine = true; - - addProperty(res, cluster, 'cfg', 'metricsExpireTime'); - addProperty(res, cluster, 'cfg', 'metricsHistorySize'); - addProperty(res, cluster, 'cfg', 'metricsLogFrequency'); - addProperty(res, cluster, 'cfg', 'metricsUpdateFrequency'); - res.needEmptyLine = true; - - addProperty(res, cluster, 'cfg', 'peerClassLoadingEnabled'); - addMultiparamProperty(res, cluster, 'cfg', 'peerClassLoadingLocalClassPathExclude'); - addProperty(res, cluster, 'cfg', 'peerClassLoadingMissedResourcesCacheSize'); - addProperty(res, cluster, 'cfg', 'peerClassLoadingThreadPoolSize'); - res.needEmptyLine = true; - - if (cluster.swapSpaceSpi && cluster.swapSpaceSpi.kind == 'FileSwapSpaceSpi') { - addBeanWithProperties(res, cluster.swapSpaceSpi.FileSwapSpaceSpi, 'cfg', 'swapSpaceSpi', 'swapSpi', - generatorUtils.swapSpaceSpi.className, generatorUtils.swapSpaceSpi.fields, true); - - res.needEmptyLine = true; - } - - addProperty(res, cluster, 'cfg', 'clockSyncSamples'); - addProperty(res, cluster, 'cfg', 'clockSyncFrequency'); - addProperty(res, cluster, 'cfg', 'timeServerPortBase'); - addProperty(res, cluster, 'cfg', 'timeServerPortRange'); - - res.needEmptyLine = true; - - addProperty(res, cluster, 'cfg', 'publicThreadPoolSize'); - addProperty(res, cluster, 'cfg', 'systemThreadPoolSize'); - addProperty(res, cluster, 'cfg', 'managementThreadPoolSize'); - addProperty(res, cluster, 'cfg', 'igfsThreadPoolSize'); - - res.needEmptyLine = true; - - addBeanWithProperties(res, cluster.transactionConfiguration, 'cfg', 'transactionConfiguration', - 'transactionConfiguration', generatorUtils.transactionConfiguration.className, - generatorUtils.transactionConfiguration.fields); - - res.needEmptyLine = true; - - addProperty(res, cluster, 'cfg', 'cacheSanityCheckEnabled'); - - res.needEmptyLine = true; - - if (javaClass) { - res.line(); - res.line('return cfg;'); - res.endBlock('}'); - res.endBlock('}'); - - return res.generateImports() + '\n\n' + res.join('') - } - - return res.join(''); -}; - -function createEvictionPolicy(res, evictionPolicy, varName, propertyName) { - if (evictionPolicy && evictionPolicy.kind) { - var e = generatorUtils.evictionPolicies[evictionPolicy.kind]; - - var obj = evictionPolicy[evictionPolicy.kind.toUpperCase()]; - - addBeanWithProperties(res, obj, varName, propertyName, propertyName, e.className, e.fields, true); - } -} - -function addCacheTypeMetadataDatabaseFields(res, meta, varName, fieldsProperty) { - var fields = meta[fieldsProperty]; - - if (fields && fields.length > 0) { - res.line('addCacheTypeMetadataDatabaseFields for ' + varName); - //res.startBlock(''); - // - //res.startBlock(''); - // - //_.forEach(fields, function (field) { - // res.startBlock(''); - //}); - // - //res.endBlock(''); - //res.endBlock(''); - } -} - -function addCacheTypeMetadataQueryFields(res, meta, varName, fieldsProperty) { - var fields = meta[fieldsProperty]; - - if (fields && fields.length > 0) { - res.line('addCacheTypeMetadataQueryFields for ' + varName); - //res.startBlock(''); - // - //res.startBlock(''); - // - //_.forEach(fields, function (field) { - // addElement(res, 'entry', 'key', field.name, 'value', generatorUtils.javaBuildInClass(field.className)); - //}); - // - //res.endBlock(''); - // - //res.endBlock(''); - } -} - -function addCacheTypeMetadataGroups(res, meta, varName) { - var groups = meta.groups; - - if (groups && groups.length > 0) { - res.line('addCacheTypeMetadataGroups for ' + varName); - //res.startBlock(''); - //res.startBlock(''); - // - //_.forEach(groups, function (group) { - // var fields = group.fields; - // - // if (fields && fields.length > 0) { - // res.startBlock(''); - // res.startBlock(''); - // - // _.forEach(fields, function (field) { - // res.startBlock(''); - // - // res.startBlock(''); - // res.line(''); - // res.line(''); - // res.endBlock(''); - // - // res.endBlock(''); - // }); - // - // res.endBlock(''); - // res.endBlock(''); - // } - //}); - // - //res.endBlock(''); - //res.endBlock(''); - } -} - -function generateCacheTypeMetadataConfiguration(res, meta, varCacheTypeMetadata) { - if (!res) - res = generatorUtils.builder(); - - res.importClass('org.apache.ignite.cache.CacheTypeMetadata'); - - var varType = varCacheTypeMetadata + 'Item'; - - addProperty(res, meta, varType, 'databaseSchema'); - addProperty(res, meta, varType, 'databaseTable'); - - addProperty(res, meta, varType, 'keyType'); - addProperty(res, meta, varType, 'valueType'); - - addCacheTypeMetadataDatabaseFields(res, meta, varType, 'keyFields'); - addCacheTypeMetadataDatabaseFields(res, meta, varType, 'valueFields'); - - addCacheTypeMetadataQueryFields(res, meta, varType, 'queryFields'); - addCacheTypeMetadataQueryFields(res, meta, varType, 'ascendingFields'); - addCacheTypeMetadataQueryFields(res, meta, varType, 'descendingFields'); - - addListProperty(res, meta, varType, 'textFields'); - - addCacheTypeMetadataGroups(res, varType, meta); - - res.line(varCacheTypeMetadata + '.add(' + varType + ')'); - - return res; -} - -/** - * Generate java code for cache configuration. - * - * @param cache Cache config. - * @param varName Variable name. - * @param res Result builder. - * @returns {*} Append generated java code to builder and return it. - */ -function generateCacheConfiguration(res, cache, varName) { - if (!res) - res = generatorUtils.builder(); - - res.emptyLineIfNeeded(); - - res.importClass('org.apache.ignite.cache.CacheAtomicityMode'); - res.importClass('org.apache.ignite.cache.CacheMode'); - res.importClass('org.apache.ignite.configuration.CacheConfiguration'); - - res.line('CacheConfiguration ' + varName + ' = new CacheConfiguration();'); - - res.needEmptyLine = true; - - addProperty(res, cache, varName, 'name'); - - addProperty(res, cache, varName, 'mode', 'CacheMode', 'cacheMode'); - - addProperty(res, cache, varName, 'atomicityMode', 'CacheAtomicityMode'); - addProperty(res, cache, varName, 'backups'); - addProperty(res, cache, varName, 'startSize'); - addProperty(res, cache, varName, 'readFromBackup'); - - res.needEmptyLine = true; - - addProperty(res, cache, varName, 'memoryMode', 'CacheMemoryMode'); - addProperty(res, cache, varName, 'offHeapMaxMemory'); - addProperty(res, cache, varName, 'swapEnabled'); - addProperty(res, cache, varName, 'copyOnRead'); - - res.needEmptyLine = true; - - createEvictionPolicy(res, cache.evictionPolicy, varName, 'evictionPolicy'); - - if (cache.nearCacheEnabled) { - res.needEmptyLine = true; - - res.importClass('org.apache.ignite.configuration.NearCacheConfiguration'); - - addBeanWithProperties(res, cache.nearConfiguration, varName, 'nearConfiguration', 'nearConfiguration', - 'NearCacheConfiguration', {nearStartSize: null}, true); - - if (cache.nearConfiguration && cache.nearConfiguration.nearEvictionPolicy && cache.nearConfiguration.nearEvictionPolicy.kind) { - createEvictionPolicy(res, cache.nearConfiguration.nearEvictionPolicy, 'nearConfiguration', 'nearEvictionPolicy'); - } - } - - res.needEmptyLine = true; - - addProperty(res, cache, varName, 'sqlEscapeAll'); - addProperty(res, cache, varName, 'sqlOnheapRowCacheSize'); - addProperty(res, cache, varName, 'longQueryWarningTimeout'); - - if (cache.indexedTypes && cache.indexedTypes.length > 0) { - res.emptyLineIfNeeded(); - - res.append(varName + '.setIndexedTypes('); - - for (var i = 0; i < cache.indexedTypes.length; i++) { - if (i > 0) - res.append(', '); - - var pair = cache.indexedTypes[i]; - - res.append(toJavaCode(pair.keyClass, 'class')).append(', ').append(toJavaCode(pair.valueClass, 'class')) - } - - res.line(');'); - } - - addMultiparamProperty(res, cache, varName, 'sqlFunctionClasses', 'class'); - - res.needEmptyLine = true; - - addProperty(res, cache, varName, 'rebalanceMode', 'CacheRebalanceMode'); - addProperty(res, cache, varName, 'rebalanceThreadPoolSize'); - addProperty(res, cache, varName, 'rebalanceBatchSize'); - addProperty(res, cache, varName, 'rebalanceOrder'); - addProperty(res, cache, varName, 'rebalanceDelay'); - addProperty(res, cache, varName, 'rebalanceTimeout'); - addProperty(res, cache, varName, 'rebalanceThrottle'); - - res.needEmptyLine = true; - - if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind) { - var storeFactory = cache.cacheStoreFactory[cache.cacheStoreFactory.kind]; - var data = generatorUtils.storeFactories[cache.cacheStoreFactory.kind]; - - var sfVarName = 'storeFactory' + toJavaName(cache.name); - var dsVarName = 'none'; - - if (storeFactory.dialect) { - var dataSourceBean = storeFactory.dataSourceBean; - - dsVarName = 'dataSource' + generatorUtils.toJavaName(dataSourceBean); - - if (!_.contains(res.datasourceBeans, dataSourceBean)) { - res.datasourceBeans.push(dataSourceBean); - - var dataSource = generatorUtils.dataSources[storeFactory.dialect]; - - res.line(); - res.line(dataSource.className + ' ' + dsVarName + ' = new ' + dataSource.className + '();'); - res.line(dsVarName + '.setURL(_URL_);'); - res.line(dsVarName + '.setUsername(_User_Name_);'); - res.line(dsVarName + '.setPassword(_Password_);'); - } - } - - addBeanWithProperties(res, storeFactory, varName, 'cacheStoreFactory', sfVarName, data.className, - data.fields, true); - - if (dsVarName != 'none') - res.line(sfVarName + '.setDataSource(' + dsVarName + ');'); - } - - res.needEmptyLine = true; - - addProperty(res, cache, varName, 'loadPreviousValue'); - addProperty(res, cache, varName, 'readThrough'); - addProperty(res, cache, varName, 'writeThrough'); - - res.needEmptyLine = true; - - addProperty(res, cache, varName, 'invalidate'); - addProperty(res, cache, varName, 'defaultLockTimeout'); - addProperty(res, cache, varName, 'transactionManagerLookupClassName'); - - res.needEmptyLine = true; - - addProperty(res, cache, varName, 'writeBehindEnabled'); - addProperty(res, cache, varName, 'writeBehindBatchSize'); - addProperty(res, cache, varName, 'writeBehindFlushSize'); - addProperty(res, cache, varName, 'writeBehindFlushFrequency'); - addProperty(res, cache, varName, 'writeBehindFlushThreadCount'); - - res.needEmptyLine = true; - - addProperty(res, cache, varName, 'statisticsEnabled'); - addProperty(res, cache, varName, 'managementEnabled'); - - res.needEmptyLine = true; - - addProperty(res, cache, varName, 'maxConcurrentAsyncOperations'); - - res.needEmptyLine = true; - - // Generate cache type metadata configs. - if ((cache.queryMetadata && cache.queryMetadata.length > 0) || - (cache.storeMetadata && cache.storeMetadata.length > 0)) { - res.emptyLineIfNeeded(); - - var varCacheTypeMetadata = varName + 'TypeMetadata'; - - res.line('Collection ' + varCacheTypeMetadata + ' = new ArrayList();'); - res.line(); - - var metaNames = []; - - if (cache.queryMetadata && cache.queryMetadata.length > 0) { - _.forEach(cache.queryMetadata, function (meta) { - if (!_.contains(metaNames, meta.name)) { - metaNames.push(meta.name); - - generateCacheTypeMetadataConfiguration(res, meta, varCacheTypeMetadata); - } - }); - } - - if (cache.storeMetadata && cache.storeMetadata.length > 0) { - _.forEach(cache.storeMetadata, function (meta) { - if (!_.contains(metaNames, meta.name)) { - metaNames.push(meta.name); - - generateCacheTypeMetadataConfiguration(meta, res); - } - }); - } - - res.line(varName + '.setCacheTypeMetadata(' + varCacheTypeMetadata + ');'); - } - - return res; -} - -function toJavaCode(val, type) { - if (val == null) - return 'null'; - - if (type == 'float') - return val + 'f'; - - if (type == 'class') - return val + '.class'; - - if (type) - return type + '.' + val; - - if (typeof(val) == 'string') - return '"' + val.replace('"', '\\"') + '"'; - - if (typeof(val) == 'number' || typeof(val) == 'boolean') - return '' + val; - - throw "Unknown type: " + typeof(val) + ' (' + val + ')'; -} - -function addProperty(res, obj, objVariableName, propName, enumType, setterName) { - var val = obj[propName]; - - if (generatorUtils.isDefined(val)) { - res.emptyLineIfNeeded(); - - res.line(objVariableName + '.' + getSetterName(setterName ? setterName : propName) - + '(' + toJavaCode(val, enumType) + ');'); - } -} - -function getSetterName(propName) { - return 'set' + propName.charAt(0).toLocaleUpperCase() + propName.slice(1); -} - -function addListProperty(res, obj, objVariableName, propName, enumType, setterName) { - var val = obj[propName]; - - if (val && val.length > 0) { - res.append(objVariableName + '.' + getSetterName(setterName ? setterName : propName) + '(Arrays.asList('); - - for (var i = 0; i < val.length; i++) { - if (i > 0) - res.append(', '); - - res.append(toJavaCode(val[i], enumType)); - } - - res.line('));'); - } -} - -function addMultiparamProperty(res, obj, objVariableName, propName, type, setterName) { - var val = obj[propName]; - - if (val && val.length > 0) { - res.append(objVariableName + '.' + getSetterName(setterName ? setterName : propName) + '('); - - for (var i = 0; i < val.length; i++) { - if (i > 0) - res.append(', '); - - res.append(toJavaCode(val[i], type)); - } - - res.line(');'); - } -} - -function addBeanWithProperties(res, bean, objVarName, beanPropName, beanVarName, beanClass, props, createBeanAlthoughNoProps) { - if (bean && generatorUtils.hasProperty(bean, props)) { - if (!res.emptyLineIfNeeded()) { - res.line(); - } - - res.line(beanClass + ' ' + beanVarName + ' = new ' + beanClass + '();'); - - for (var propName in props) { - if (props.hasOwnProperty(propName)) { - var descr = props[propName]; - - if (descr) { - switch (descr.type) { - case 'list': - addListProperty(res, bean, beanVarName, propName, descr.elementsType, descr.setterName); - break; - - case 'enum': - addProperty(res, bean, beanVarName, propName, descr.enumClass, descr.setterName); - break; - - case 'float': - addProperty(res, bean, beanVarName, propName, 'float', descr.setterName); - break; - - case 'propertiesAsList': - var val = bean[propName]; - - if (val && val.length > 0) { - res.line('Properties ' + descr.propVarName + ' = new Properties();'); - - for (var i = 0; i < val.length; i++) { - var nameAndValue = val[i]; - - var eqIndex = nameAndValue.indexOf('='); - if (eqIndex >= 0) { - res.line(descr.propVarName + '.setProperty(' - + nameAndValue.substring(0, eqIndex) + ', ' - + nameAndValue.substr(eqIndex + 1) + ');'); - } - - } - - res.line(beanVarName + '.' + getSetterName(propName) + '(' + descr.propVarName + ');'); - } - break; - - case 'className': - if (bean[propName]) { - res.line(beanVarName + '.' + getSetterName(propName) + '(new ' + generatorUtils.knownClasses[bean[propName]].className + '());'); - } - - break; - - default: - addProperty(res, bean, beanVarName, propName, null, descr.setterName); - } - } - else { - addProperty(res, bean, beanVarName, propName); - } - } - } - - res.line(objVarName + '.' + getSetterName(beanPropName) + '(' + beanVarName + ');'); - - res.needEmptyLine = true; - } - else if (createBeanAlthoughNoProps) { - res.emptyLineIfNeeded(); - - res.line(objVarName + '.' + getSetterName(beanPropName) + '(new ' + beanClass + '());'); - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96522874/modules/web-control-center/src/main/js/routes/generator/xml.js ---------------------------------------------------------------------- diff --git a/modules/web-control-center/src/main/js/routes/generator/xml.js b/modules/web-control-center/src/main/js/routes/generator/xml.js deleted file mode 100644 index 079f268..0000000 --- a/modules/web-control-center/src/main/js/routes/generator/xml.js +++ /dev/null @@ -1,736 +0,0 @@ -/* - * 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. - */ - -var _ = require('lodash'); - -var generatorUtils = require("./common"); -var dataStructures = require("../../helpers/data-structures.js"); - -exports.generateClusterConfiguration = function(cluster, clientNearConfiguration) { - var res = generatorUtils.builder(); - - res.datasources = []; - res.deep = 1; - - if (clientNearConfiguration) { - res.startBlock(''); - - if (clientNearConfiguration.nearStartSize) - addProperty(res, clientNearConfiguration, 'nearStartSize'); - - if (clientNearConfiguration.nearEvictionPolicy && clientNearConfiguration.nearEvictionPolicy.kind) - createEvictionPolicy(res, clientNearConfiguration.nearEvictionPolicy, 'nearEvictionPolicy'); - - res.endBlock(''); - - res.line(); - } - - // Generate Ignite Configuration. - res.startBlock(''); - - if (clientNearConfiguration) { - res.line(''); - - res.line(); - } - - // Generate discovery. - if (cluster.discovery) { - res.startBlock(''); - res.startBlock(''); - res.startBlock(''); - - var d = cluster.discovery; - - switch (d.kind) { - case 'Multicast': - res.startBlock(''); - - addProperty(res, d.Multicast, 'multicastGroup'); - addProperty(res, d.Multicast, 'multicastPort'); - addProperty(res, d.Multicast, 'responseWaitTime'); - addProperty(res, d.Multicast, 'addressRequestAttempts'); - addProperty(res, d.Multicast, 'localAddress'); - - res.endBlock(''); - - break; - - case 'Vm': - if (d.Vm.addresses.length > 0) { - res.startBlock(''); - - addListProperty(res, d.Vm, 'addresses'); - - res.endBlock(''); - } - else { - res.line(''); - } - - break; - - case 'S3': - res.startBlock(''); - - if (d.S3 && d.S3.bucketName) - res.line(''); - - res.endBlock(''); - - break; - - case 'Cloud': - res.startBlock(''); - - addProperty(res, d.Cloud, 'credential'); - addProperty(res, d.Cloud, 'credentialPath'); - addProperty(res, d.Cloud, 'identity'); - addProperty(res, d.Cloud, 'provider'); - addListProperty(res, d.Cloud, 'regions'); - addListProperty(res, d.Cloud, 'zones'); - - res.endBlock(''); - - break; - - case 'GoogleStorage': - res.startBlock(''); - - addProperty(res, d.GoogleStorage, 'projectName'); - addProperty(res, d.GoogleStorage, 'bucketName'); - addProperty(res, d.GoogleStorage, 'serviceAccountP12FilePath'); - - //if (d.GoogleStorage.addrReqAttempts) todo ???? - // res.line(''); - - res.endBlock(''); - - break; - - case 'Jdbc': - res.startBlock(''); - res.line(''); - res.endBlock(''); - - break; - - case 'SharedFs': - if (d.SharedFs.path) { - res.startBlock(''); - addProperty(res, d.SharedFs, 'path'); - res.endBlock(''); - } - else { - res.line(''); - } - - break; - - default: - throw "Unknown discovery kind: " + d.kind; - } - - res.endBlock(''); - res.endBlock(''); - res.endBlock(''); - - res.needEmptyLine = true - } - - // Generate atomics group. - addBeanWithProperties(res, cluster.atomicConfiguration, 'atomicConfiguration', - generatorUtils.atomicConfiguration.className, generatorUtils.atomicConfiguration.fields); - res.needEmptyLine = true; - - // Generate communication group. - addProperty(res, cluster, 'networkTimeout'); - addProperty(res, cluster, 'networkSendRetryDelay'); - addProperty(res, cluster, 'networkSendRetryCount'); - addProperty(res, cluster, 'segmentCheckFrequency'); - addProperty(res, cluster, 'waitForSegmentOnStart'); - addProperty(res, cluster, 'discoveryStartupDelay'); - res.needEmptyLine = true; - - // Generate deployment group. - addProperty(res, cluster, 'deploymentMode'); - res.needEmptyLine = true; - - // Generate events group. - if (cluster.includeEventTypes && cluster.includeEventTypes.length > 0) { - res.emptyLineIfNeeded(); - - res.startBlock(''); - - if (cluster.includeEventTypes.length == 1) - res.line(''); - else { - res.startBlock(''); - - for (i = 0; i < cluster.includeEventTypes.length; i++) { - if (i > 0) - res.line(); - - var eventGroup = cluster.includeEventTypes[i]; - - res.line(''); - - var eventList = dataStructures.eventGroups[eventGroup]; - - for (var k = 0; k < eventList.length; k++) { - res.line('') - } - } - - res.endBlock(''); - } - - res.endBlock(''); - - res.needEmptyLine = true; - } - - // Generate marshaller group. - var marshaller = cluster.marshaller; - - if (marshaller && marshaller.kind) { - var marshallerDesc = generatorUtils.marshallers[marshaller.kind]; - - addBeanWithProperties(res, marshaller[marshaller.kind], 'marshaller', marshallerDesc.className, marshallerDesc.fields, true); - res.needEmptyLine = true; - } - - addProperty(res, cluster, 'marshalLocalJobs'); - addProperty(res, cluster, 'marshallerCacheKeepAliveTime'); - addProperty(res, cluster, 'marshallerCacheThreadPoolSize'); - res.needEmptyLine = true; - - // Generate metrics group. - addProperty(res, cluster, 'metricsExpireTime'); - addProperty(res, cluster, 'metricsHistorySize'); - addProperty(res, cluster, 'metricsLogFrequency'); - addProperty(res, cluster, 'metricsUpdateFrequency'); - res.needEmptyLine = true; - - // Generate PeerClassLoading group. - addProperty(res, cluster, 'peerClassLoadingEnabled'); - addListProperty(res, cluster, 'peerClassLoadingLocalClassPathExclude'); - addProperty(res, cluster, 'peerClassLoadingMissedResourcesCacheSize'); - addProperty(res, cluster, 'peerClassLoadingThreadPoolSize'); - res.needEmptyLine = true; - - // Generate swap group. - if (cluster.swapSpaceSpi && cluster.swapSpaceSpi.kind == 'FileSwapSpaceSpi') { - addBeanWithProperties(res, cluster.swapSpaceSpi.FileSwapSpaceSpi, 'swapSpaceSpi', - generatorUtils.swapSpaceSpi.className, generatorUtils.swapSpaceSpi.fields, true); - - res.needEmptyLine = true; - } - - // Generate time group. - addProperty(res, cluster, 'clockSyncSamples'); - addProperty(res, cluster, 'clockSyncFrequency'); - addProperty(res, cluster, 'timeServerPortBase'); - addProperty(res, cluster, 'timeServerPortRange'); - res.needEmptyLine = true; - - // Generate thread pools group. - addProperty(res, cluster, 'publicThreadPoolSize'); - addProperty(res, cluster, 'systemThreadPoolSize'); - addProperty(res, cluster, 'managementThreadPoolSize'); - addProperty(res, cluster, 'igfsThreadPoolSize'); - res.needEmptyLine = true; - - // Generate transactions group. - addBeanWithProperties(res, cluster.transactionConfiguration, 'transactionConfiguration', - generatorUtils.transactionConfiguration.className, generatorUtils.transactionConfiguration.fields); - res.needEmptyLine = true; - - // Generate caches configs. - if (cluster.caches && cluster.caches.length > 0) { - res.emptyLineIfNeeded(); - - res.startBlock(''); - res.startBlock(''); - - for (var i = 0; i < cluster.caches.length; i++) { - if (i > 0) - res.line(); - - var cache = cluster.caches[i]; - - generateCacheConfiguration(res, cache); - } - - res.endBlock(''); - res.endBlock(''); - - res.needEmptyLine = true; - } - - res.endBlock(''); - - // Build final XML: - // 1. Add header. - var xml = '\n\n'; - - xml += '\n'; - xml += ' 0) { - xml += ' \n'; - xml += ' \n'; - xml += ' \n'; - xml += ' \n\n'; - - xml += ' \n'; - - _.forEach(res.datasources, function(item) { - var beanId = item.dataSourceBean; - - xml += ' \n'; - xml += ' \n'; - xml += ' \n'; - xml += ' \n'; - xml += ' \n\n'; - }); - } - - // 3. Add main content. - xml += res.join(''); - - // 4. Add footer. - xml += '\n'; - - return xml; -}; - -function createEvictionPolicy(res, evictionPolicy, propertyName) { - if (evictionPolicy && evictionPolicy.kind) { - var e = generatorUtils.evictionPolicies[evictionPolicy.kind]; - - var obj = evictionPolicy[evictionPolicy.kind.toUpperCase()]; - - addBeanWithProperties(res, obj, propertyName, e.className, e.fields, true); - } -} - -function addCacheTypeMetadataDatabaseFields(res, meta, fieldsProperty) { - var fields = meta[fieldsProperty]; - - if (fields && fields.length > 0) { - res.startBlock(''); - - res.startBlock(''); - - _.forEach(fields, function (field) { - res.startBlock(''); - }); - - res.endBlock(''); - res.endBlock(''); - } -} - -function addCacheTypeMetadataQueryFields(res, meta, fieldsProperty) { - var fields = meta[fieldsProperty]; - - if (fields && fields.length > 0) { - res.startBlock(''); - - res.startBlock(''); - - _.forEach(fields, function (field) { - addElement(res, 'entry', 'key', field.name, 'value', generatorUtils.javaBuildInClass(field.className)); - }); - - res.endBlock(''); - - res.endBlock(''); - } -} - -function addCacheTypeMetadataGroups(res, meta) { - var groups = meta.groups; - - if (groups && groups.length > 0) { - res.startBlock(''); - res.startBlock(''); - - _.forEach(groups, function (group) { - var fields = group.fields; - - if (fields && fields.length > 0) { - res.startBlock(''); - res.startBlock(''); - - _.forEach(fields, function (field) { - res.startBlock(''); - - res.startBlock(''); - res.line(''); - res.line(''); - res.endBlock(''); - - res.endBlock(''); - }); - - res.endBlock(''); - res.endBlock(''); - } - }); - - res.endBlock(''); - res.endBlock(''); - } -} - -function generateCacheTypeMetadataConfiguration(res, meta) { - if (!res) - res = generatorUtils.builder(); - - res.startBlock(''); - - return res; -} - -function generateCacheConfiguration(res, cacheCfg) { - if (!res) - res = generatorUtils.builder(); - - res.startBlock(''); - - addProperty(res, cacheCfg, 'name'); - - res.needEmptyLine = true; - - addProperty(res, cacheCfg, 'mode', 'cacheMode'); - - addProperty(res, cacheCfg, 'atomicityMode'); - addProperty(res, cacheCfg, 'backups'); - addProperty(res, cacheCfg, 'startSize'); - addProperty(res, cacheCfg, 'readFromBackup'); - - res.needEmptyLine = true; - - addProperty(res, cacheCfg, 'memoryMode'); - addProperty(res, cacheCfg, 'offHeapMaxMemory'); - addProperty(res, cacheCfg, 'swapEnabled'); - addProperty(res, cacheCfg, 'copyOnRead'); - - res.needEmptyLine = true; - - createEvictionPolicy(res, cacheCfg.evictionPolicy, 'evictionPolicy'); - - res.needEmptyLine = true; - - if (cacheCfg.nearCacheEnabled) { - res.emptyLineIfNeeded(); - - res.startBlock(''); - res.startBlock(''); - - if (cacheCfg.nearConfiguration && cacheCfg.nearConfiguration.nearStartSize) - addProperty(res, cacheCfg.nearConfiguration, 'nearStartSize'); - - if (cacheCfg.nearConfiguration && cacheCfg.nearConfiguration.nearEvictionPolicy.kind) - createEvictionPolicy(res, cacheCfg.nearConfiguration.nearEvictionPolicy, 'nearEvictionPolicy'); - - res.endBlock(''); - res.endBlock(''); - } - - res.needEmptyLine = true; - - addProperty(res, cacheCfg, 'sqlEscapeAll'); - addProperty(res, cacheCfg, 'sqlOnheapRowCacheSize'); - addProperty(res, cacheCfg, 'longQueryWarningTimeout'); - - if (cacheCfg.indexedTypes && cacheCfg.indexedTypes.length > 0) { - res.startBlock(''); - res.startBlock(''); - - for (var i = 0; i < cacheCfg.indexedTypes.length; i++) { - var pair = cacheCfg.indexedTypes[i]; - - res.line('' + generatorUtils.javaBuildInClass(pair.keyClass) + ''); - res.line('' + generatorUtils.javaBuildInClass(pair.valueClass) + ''); - } - - res.endBlock(''); - res.endBlock(''); - } - - addListProperty(res, cacheCfg, 'sqlFunctionClasses', 'array'); - - res.needEmptyLine = true; - - addProperty(res, cacheCfg, 'rebalanceMode'); - addProperty(res, cacheCfg, 'rebalanceThreadPoolSize'); - addProperty(res, cacheCfg, 'rebalanceBatchSize'); - addProperty(res, cacheCfg, 'rebalanceOrder'); - addProperty(res, cacheCfg, 'rebalanceDelay'); - addProperty(res, cacheCfg, 'rebalanceTimeout'); - addProperty(res, cacheCfg, 'rebalanceThrottle'); - - res.needEmptyLine = true; - - if (cacheCfg.cacheStoreFactory && cacheCfg.cacheStoreFactory.kind) { - var storeFactory = cacheCfg.cacheStoreFactory[cacheCfg.cacheStoreFactory.kind]; - var data = generatorUtils.storeFactories[cacheCfg.cacheStoreFactory.kind]; - - addBeanWithProperties(res, storeFactory, 'cacheStoreFactory', data.className, data.fields, true); - - if (storeFactory.dialect) { - if (_.findIndex(res.datasources, function (ds) { - return ds.dataSourceBean == storeFactory.dataSourceBean; - }) < 0) { - res.datasources.push({ - dataSourceBean: storeFactory.dataSourceBean, - className: generatorUtils.dataSources[storeFactory.dialect] - }); - } - } - } - - res.needEmptyLine = true; - - addProperty(res, cacheCfg, 'loadPreviousValue'); - addProperty(res, cacheCfg, 'readThrough'); - addProperty(res, cacheCfg, 'writeThrough'); - - res.needEmptyLine = true; - - addProperty(res, cacheCfg, 'invalidate'); - addProperty(res, cacheCfg, 'defaultLockTimeout'); - addProperty(res, cacheCfg, 'transactionManagerLookupClassName'); - - res.needEmptyLine = true; - - addProperty(res, cacheCfg, 'writeBehindEnabled'); - addProperty(res, cacheCfg, 'writeBehindBatchSize'); - addProperty(res, cacheCfg, 'writeBehindFlushSize'); - addProperty(res, cacheCfg, 'writeBehindFlushFrequency'); - addProperty(res, cacheCfg, 'writeBehindFlushThreadCount'); - - res.needEmptyLine = true; - - addProperty(res, cacheCfg, 'statisticsEnabled'); - addProperty(res, cacheCfg, 'managementEnabled'); - - res.needEmptyLine = true; - - addProperty(res, cacheCfg, 'maxConcurrentAsyncOperations'); - - // Generate cache type metadata configs. - if ((cacheCfg.queryMetadata && cacheCfg.queryMetadata.length > 0) || - (cacheCfg.storeMetadata && cacheCfg.storeMetadata.length > 0)) { - res.emptyLineIfNeeded(); - - res.startBlock(''); - res.startBlock(''); - - var metaNames = []; - - if (cacheCfg.queryMetadata && cacheCfg.queryMetadata.length > 0) { - _.forEach(cacheCfg.queryMetadata, function (meta) { - if (!_.contains(metaNames, meta.name)) { - metaNames.push(meta.name); - - generateCacheTypeMetadataConfiguration(res, meta); - } - }); - } - - if (cacheCfg.storeMetadata && cacheCfg.storeMetadata.length > 0) { - _.forEach(cacheCfg.storeMetadata, function (meta) { - if (!_.contains(metaNames, meta.name)) { - metaNames.push(meta.name); - - generateCacheTypeMetadataConfiguration(res, meta); - } - }); - } - - res.endBlock(''); - res.endBlock(''); - } - - res.endBlock(''); - - return res; -} - -function addElement(res, tag, attr1, val1, attr2, val2) { - var elem = '<' + tag; - - if (attr1) { - elem += ' ' + attr1 + '="' + val1 + '"' - } - - if (attr2) { - elem += ' ' + attr2 + '="' + val2 + '"' - } - - elem += '/>'; - - res.emptyLineIfNeeded(); - res.line(elem); -} - -function addProperty(res, obj, propName, setterName) { - var val = obj[propName]; - - if (generatorUtils.isDefined(val)) - addElement(res, 'property', 'name', setterName ? setterName : propName, 'value', escapeAttr(val)); -} - -function addBeanWithProperties(res, bean, beanPropName, beanClass, props, createBeanAlthoughNoProps) { - if (bean && generatorUtils.hasProperty(bean, props)) { - res.emptyLineIfNeeded(); - res.startBlock(''); - res.startBlock(''); - - for (var propName in props) { - if (props.hasOwnProperty(propName)) { - var descr = props[propName]; - - if (descr) { - if (descr.type == 'list') { - addListProperty(res, bean, propName, descr.setterName); - } - else if (descr.type == 'className') { - if (bean[propName]) { - res.startBlock(''); - res.line(''); - res.endBlock(''); - } - } - else if (descr.type == 'propertiesAsList') { - var val = bean[propName]; - - if (val && val.length > 0) { - res.startBlock(''); - res.startBlock(''); - - for (var i = 0; i < val.length; i++) { - var nameAndValue = val[i]; - - var eqIndex = nameAndValue.indexOf('='); - if (eqIndex >= 0) { - res.line('' + - + escape(nameAndValue.substr(eqIndex + 1)) + ''); - } - } - - res.endBlock(''); - res.endBlock(''); - } - } - else - addProperty(res, bean, propName, descr.setterName); - } - else - addProperty(res, bean, propName); - } - } - - res.endBlock(''); - res.endBlock(''); - } - else if (createBeanAlthoughNoProps) { - res.emptyLineIfNeeded(); - res.line(''); - res.line(' '); - res.line(''); - } -} -function addListProperty(res, obj, propName, listType, rowFactory) { - var val = obj[propName]; - - if (val && val.length > 0) { - res.emptyLineIfNeeded(); - - if (!listType) - listType = 'list'; - - if (!rowFactory) - rowFactory = function(val) { return '' + escape(val) + '' }; - - res.startBlock(''); - res.startBlock('<' + listType + '>'); - - for (var i = 0; i < val.length; i++) - res.line(rowFactory(val[i])); - - res.endBlock(''); - res.endBlock(''); - } -} - -function escapeAttr(s) { - if (typeof(s) != 'string') - return s; - - return s.replace(/&/g, '&').replace(/"/g, '"'); -} - -function escape(s) { - if (typeof(s) != 'string') - return s; - - return s.replace(/&/g, '&').replace(//g, '>'); -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96522874/modules/web-control-center/src/main/js/routes/metadata.js ---------------------------------------------------------------------- diff --git a/modules/web-control-center/src/main/js/routes/metadata.js b/modules/web-control-center/src/main/js/routes/metadata.js deleted file mode 100644 index 0af624e..0000000 --- a/modules/web-control-center/src/main/js/routes/metadata.js +++ /dev/null @@ -1,95 +0,0 @@ -/* - * 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. - */ - -var router = require('express').Router(); -var db = require('../db'); - -/* GET metadata page. */ -router.get('/', function (req, res) { - res.render('configuration/metadata'); -}); - -/** - * Get spaces and metadata accessed for user account. - * - * @param req Request. - * @param res Response. - */ -router.post('/list', function (req, res) { - var user_id = req.currentUserId(); - - // Get owned space and all accessed space. - db.Space.find({$or: [{owner: user_id}, {usedBy: {$elemMatch: {account: user_id}}}]}, function (err, spaces) { - if (err) - return res.status(500).send(err.message); - - var space_ids = spaces.map(function (value) { - return value._id; - }); - - // Get all metadata for spaces. - db.CacheTypeMetadata.find({space: {$in: space_ids}}).sort('name').exec(function (err, metadatas) { - if (err) - return res.status(500).send(err.message); - - res.json({spaces: spaces, metadatas: metadatas}); - }); - }); -}); - -/** - * Save metadata. - */ -router.post('/save', function (req, res) { - if (req.body._id) - db.CacheTypeMetadata.update({_id: req.body._id}, req.body, {upsert: true}, function (err) { - if (err) - return res.status(500).send(err.message); - - res.send(req.body._id); - }); - else { - db.CacheTypeMetadata.findOne({space: req.body.space, name: req.body.name}, function (err, metadata) { - if (err) - return res.status(500).send(err.message); - - if (metadata) - return res.status(500).send('Cache type metadata with name: "' + metadata.name + '" already exist.'); - - (new db.CacheTypeMetadata(req.body)).save(function (err, metadata) { - if (err) - return res.status(500).send(err.message); - - res.send(metadata._id); - }); - }); - } -}); - -/** - * Remove metadata by ._id. - */ -router.post('/remove', function (req, res) { - db.CacheTypeMetadata.remove(req.body, function (err) { - if (err) - return res.status(500).send(err.message); - - res.sendStatus(200); - }) -}); - -module.exports = router; \ No newline at end of file http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96522874/modules/web-control-center/src/main/js/routes/profile.js ---------------------------------------------------------------------- diff --git a/modules/web-control-center/src/main/js/routes/profile.js b/modules/web-control-center/src/main/js/routes/profile.js deleted file mode 100644 index 0269e7d..0000000 --- a/modules/web-control-center/src/main/js/routes/profile.js +++ /dev/null @@ -1,97 +0,0 @@ -/* - * 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. - */ - -var router = require('express').Router(); -var db = require('../db'); - -router.all('/profile/*', function (req, res, next) { - var userId = req.body._id; - - if (userId != req.currentUserId() && userId != req.user._id) - return res.sendStatus(403); - else - next(); -}); - -/** - * Get user profile page. - */ -router.get('/', function (req, res) { - var user_id = req.currentUserId(); - - db.Account.findById(user_id, function (err) { - if (err) - return res.status(500).send(err.message); - - res.render('settings/profile'); - }); -}); - -/** - * Save user profile. - */ -router.post('/saveUser', function (req, res) { - var params = req.body; - - if (params.newPassword) { - var newPassword = params.newPassword; - - if (!newPassword || newPassword.length == 0) - return res.status(500).send('Wrong value for new password'); - - db.Account.findById(params._id, function (err, user) { - if (err) - return res.status(500).send(err); - - user.setPassword(newPassword, function (err, updatedUser) { - if (err) - return res.status(500).send(err.message); - - if (params.userName) - updatedUser.username = params.userName; - - if (params.email) - updatedUser.email = params.email; - - updatedUser.save(function (err) { - if (err) - return res.status(500).send(err.message); - - res.json(user); - }); - }); - }); - } - else if (params.userName || params.email) { - var upd = {}; - - if (params.userName) - upd.username = params.userName; - - if (params.email) - upd.email = params.email; - - db.Account.findByIdAndUpdate(params._id, upd, {new: true}, function (err, val) { - if (err) - return res.status(500).send(err.message); - - res.json(val); - }) - } -}); - -module.exports = router; \ No newline at end of file http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96522874/modules/web-control-center/src/main/js/routes/public.js ---------------------------------------------------------------------- diff --git a/modules/web-control-center/src/main/js/routes/public.js b/modules/web-control-center/src/main/js/routes/public.js deleted file mode 100644 index b3cb983..0000000 --- a/modules/web-control-center/src/main/js/routes/public.js +++ /dev/null @@ -1,123 +0,0 @@ -/* - * 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. - */ - -var router = require('express').Router(); -var passport = require('passport'); -var db = require('../db'); - -// GET dropdown-menu template. -router.get('/select', function (req, res) { - res.render('templates/select', {}); -}); - -// GET dynamic tabs template. -router.get('/tab', function (req, res) { - res.render('templates/tab', {}); -}); - -// GET confirmation dialog. -router.get('/confirm', function (req, res) { - res.render('templates/confirm', {}); -}); - -// GET copy dialog. -router.get('/copy', function (req, res) { - res.render('templates/copy', {}); -}); - -/* GET login page. */ -router.get('/login', function (req, res) { - res.render('login'); -}); - -/** - * Register new account. - */ -router.post('/register', function (req, res) { - db.Account.count(function (err, cnt) { - if (err) - return res.status(401).send(err.message); - - req.body.admin = cnt == 0; - - db.Account.register(new db.Account(req.body), req.body.password, function (err, account) { - if (err) - return res.status(401).send(err.message); - - if (!account) - return res.status(500).send('Failed to create account.'); - - new db.Space({name: 'Personal space', owner: account._id}).save(); - - req.logIn(account, {}, function (err) { - if (err) - return res.status(401).send(err.message); - - return res.redirect('/configuration/clusters'); - }); - }); - }); -}); - -/** - * Login in exist account. - */ -router.post('/login', function (req, res, next) { - passport.authenticate('local', function (err, user) { - if (err) - return res.status(401).send(err.message); - - if (!user) - return res.status(401).send('Invalid email or password'); - - req.logIn(user, {}, function (err) { - if (err) - return res.status(401).send(err.message); - - res.redirect('/configuration/clusters'); - }); - })(req, res, next); -}); - -/** - * Logout. - */ -router.get('/logout', function (req, res) { - req.logout(); - - res.redirect('/'); -}); - -/* GET home page. */ -router.get('/', function (req, res) { - if (req.isAuthenticated()) - res.redirect('/configuration/clusters'); - else - res.render('index'); -}); - -///* GET sql page. */ -//router.get('/sql', function(req, res) { -// res.render('sql', { user: req.user }); -//}); -// -///* GET clients page. */ -//router.get('/clients', function(req, res) { -// res.render('clients', { user: req.user }); -//}); - -module.exports = router; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96522874/modules/web-control-center/src/main/js/routes/sql.js ---------------------------------------------------------------------- diff --git a/modules/web-control-center/src/main/js/routes/sql.js b/modules/web-control-center/src/main/js/routes/sql.js deleted file mode 100644 index ce4565d..0000000 --- a/modules/web-control-center/src/main/js/routes/sql.js +++ /dev/null @@ -1,24 +0,0 @@ -/* - * 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. - */ - -var router = require('express').Router(); -var db = require('../db'); -router.get('/', function(req, res) { - res.render('sql/sql'); -}); - -module.exports = router; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96522874/modules/web-control-center/src/main/js/routes/summary.js ---------------------------------------------------------------------- diff --git a/modules/web-control-center/src/main/js/routes/summary.js b/modules/web-control-center/src/main/js/routes/summary.js deleted file mode 100644 index 9f8df2a..0000000 --- a/modules/web-control-center/src/main/js/routes/summary.js +++ /dev/null @@ -1,108 +0,0 @@ -/* - * 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. - */ - -var db = require('../db'); - -var router = require('express').Router(); - -var generatorXml = require('./generator/xml'); -var generatorJava = require('./generator/java'); -var generatorDocker = require('./generator/docker'); - -/* GET summary page. */ -router.get('/', function (req, res) { - res.render('configuration/summary'); -}); - -router.post('/generator', function (req, res) { - // Get cluster. - db.Cluster.findById(req.body._id).deepPopulate('caches caches.queryMetadata caches.storeMetadata').exec(function (err, cluster) { - if (err) - return res.status(500).send(err.message); - - if (!cluster) - return res.sendStatus(404); - - var clientCache = req.body.clientNearConfiguration; - - if (clientCache) - return res.send({ - xmlClient: generatorXml.generateClusterConfiguration(cluster, clientCache), - javaClient: generatorJava.generateClusterConfiguration(cluster, req.body.javaClass, clientCache) - }); - - return res.send({ - xmlServer: generatorXml.generateClusterConfiguration(cluster), - javaSnippetServer: generatorJava.generateClusterConfiguration(cluster, false), - javaClassServer: generatorJava.generateClusterConfiguration(cluster, true), - docker: generatorDocker.generateClusterConfiguration(cluster, '%OS%') - }); - }); -}); - -router.post('/download', function (req, res) { - // Get cluster. - db.Cluster.findById(req.body._id).populate('caches').exec(function (err, cluster) { - if (err) - return res.status(500).send(err.message); - - if (!cluster) - return res.sendStatus(404); - - var clientNearConfiguration = req.body.clientNearConfiguration; - - var archiver = require('archiver'); - - // Creating archive. - var zip = archiver('zip'); - - zip.on('error', function (err) { - res.status(500).send({error: err.message}); - }); - - // On stream closed we can end the request. - res.on('close', function () { - return res.status(200).send('OK').end(); - }); - - // Set the archive name. - res.attachment(cluster.name + (clientNearConfiguration ? '-client' : '') + '-configuration.zip'); - - var generatorCommon = require('./generator/common'); - - // Send the file to the page output. - zip.pipe(res); - - var javaClass = req.body.javaClass; - - if (!clientNearConfiguration) { - zip.append(generatorDocker.generateClusterConfiguration(cluster, req.body.os), {name: "Dockerfile"}); - - var props = generatorCommon.generateProperties(cluster); - - if (props) - zip.append(props, {name: "secret.properties"}); - } - - zip.append(generatorXml.generateClusterConfiguration(cluster, clientNearConfiguration), {name: cluster.name + ".xml"}) - .append(generatorJava.generateClusterConfiguration(cluster, javaClass, clientNearConfiguration), - {name: javaClass ? 'ConfigurationFactory.java' : cluster.name + '.snipplet.java'}) - .finalize(); - }); -}); - -module.exports = router; \ No newline at end of file http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96522874/modules/web-control-center/src/main/js/views/configuration/caches.jade ---------------------------------------------------------------------- diff --git a/modules/web-control-center/src/main/js/views/configuration/caches.jade b/modules/web-control-center/src/main/js/views/configuration/caches.jade deleted file mode 100644 index 15d8f40..0000000 --- a/modules/web-control-center/src/main/js/views/configuration/caches.jade +++ /dev/null @@ -1,74 +0,0 @@ -//- - 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. - -extends sidebar - -append scripts - script(src='/caches-controller.js') - -include ../includes/controls - -block content - .docs-header - h1 Create and Configure Ignite Caches - hr - .docs-body(ng-controller='cachesController') - +block-callout('{{screenTip.workflowTitle}}', 'joinTip(screenTip.workflowContent)', '{{screenTip.whatsNextTitle}}', 'joinTip(screenTip.whatsNextContent)') - div(ng-hide='caches.length == 0') - .padding-dflt - lable.labelHeader Caches: - table.links(st-table='caches') - tbody - tr(ng-repeat='row in caches track by row._id') - td.col-sm-6(ng-class='{active: row._id == selectedItem._id}') - a(event-focus='click' event-focus-id='defaultFocusId' ng-click='selectItem(row)') {{$index + 1}}) {{row.name}}, {{row.mode | displayValue:modes:'Cache mode not set'}}, {{row.atomicityMode | displayValue:atomicities:'Cache atomicity not set'}} - .padding-top-dflt - button.btn.btn-primary(event-focus='click' event-focus-id='defaultFocusId' ng-click='createItem()') Add cache - hr - form.form-horizontal(name='inputForm' ng-if='backupItem' novalidate) - div(bs-collapse data-start-collapsed='false') - .panel.panel-default - .panel-heading - h3 - a(bs-collapse-toggle) General - .panel-collapse(bs-collapse-target) - .panel-body - .settings-row(ng-repeat='field in general') - +form-row(['col-sm-3'], ['col-sm-3']) - .panel-group(bs-collapse data-allow-multiple='true') - div(bs-collapse data-start-collapsed='true') - .panel-title(ng-show='expanded') - h3 - a(bs-collapse-toggle='0' ng-click='expanded = !expanded;') {{expanded ? 'Hide advanced settings...' : 'Show advanced settings...'}} - .panel-collapse(bs-collapse-target) - .span(bs-collapse data-start-collapsed='true' data-allow-multiple='true') - .panel.panel-default(ng-repeat='group in advanced') - .panel-heading - h3 - a(bs-collapse-toggle) {{group.label}} - i.tipLabel.fa.fa-question-circle(ng-if='group.tip' bs-tooltip='joinTip(group.tip)' type='button') - i.tipLabel.fa.fa-question-circle.blank(ng-if='!group.tip') - .panel-collapse(bs-collapse-target) - .panel-body - .settings-row(ng-repeat='field in group.fields') - +form-row - .panel-title - h3 - a(bs-collapse-toggle='0' ng-click='expanded = !expanded;') {{expanded ? 'Hide advanced settings...' : 'Show advanced settings...'}} - div - button.btn.btn-primary(ng-disabled='inputForm.$invalid' ng-click='saveItem()') Save - button.btn.btn-primary(ng-show='backupItem._id' ng-disabled='inputForm.$invalid' ng-click='saveItemAs()') Copy - button.btn.btn-primary.btn-second(ng-show='backupItem._id' ng-click='removeItem()') Remove http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96522874/modules/web-control-center/src/main/js/views/configuration/clusters.jade ---------------------------------------------------------------------- diff --git a/modules/web-control-center/src/main/js/views/configuration/clusters.jade b/modules/web-control-center/src/main/js/views/configuration/clusters.jade deleted file mode 100644 index 239a31f..0000000 --- a/modules/web-control-center/src/main/js/views/configuration/clusters.jade +++ /dev/null @@ -1,77 +0,0 @@ -//- - 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. - -extends sidebar - -append scripts - script(src='/clusters-controller.js') - -include ../includes/controls - -block content - .docs-header - h1 Create and Configure Ignite Clusters - hr - .docs-body(ng-controller='clustersController') - +block-callout('{{screenTip.workflowTitle}}', 'joinTip(screenTip.workflowContent)', '{{screenTip.whatsNextTitle}}', 'joinTip(screenTip.whatsNextContent)') - div(ng-hide='clusters.length == 0') - .padding-dflt - lable.labelHeader Clusters: - table.links(st-table='clusters') - tbody - tr(ng-repeat='row in clusters track by row._id') - td.col-sm-6(ng-class='{active: row._id == selectedItem._id}') - a(event-focus='click' event-focus-id='defaultFocusId' ng-click='selectItem(row)') {{$index + 1}}) {{row.name}}, {{row.discovery.kind | displayValue:discoveries:'Discovery not set'}} - .padding-top-dflt - button.btn.btn-primary(event-focus='click' event-focus-id='defaultFocusId' ng-click='createItem()')  Add cluster - label(style='margin-left: 10px; margin-right: 10px') Use template: - button.btn.btn-default.base-control(ng-init='create.template = templates[0].value' ng-model='create.template' data-template='/select' data-placeholder='Choose cluster template' bs-options='item.value as item.label for item in templates' bs-select) - i.tiplabel.fa.fa-question-circle(bs-tooltip data-title='{{joinTip(templateTip)}}' type='button') - hr - form.form-horizontal(name='inputForm' ng-if='backupItem' novalidate) - div(bs-collapse data-start-collapsed='false') - .panel.panel-default - .panel-heading - h3 - a(bs-collapse-toggle) General - .panel-collapse(bs-collapse-target) - .panel-body - .settings-row(ng-repeat='field in general') - +form-row - .panel-group(bs-collapse data-allow-multiple='true') - div(bs-collapse data-start-collapsed='true') - .panel-title(ng-show='expanded') - h3 - a(bs-collapse-toggle='0' ng-click='expanded = !expanded;') {{expanded ? 'Hide advanced settings...' : 'Show advanced settings...'}} - .panel-collapse(bs-collapse-target) - .span(bs-collapse data-start-collapsed='true' data-allow-multiple='true') - .panel.panel-default(ng-repeat='group in advanced') - .panel-heading - h3 - a(bs-collapse-toggle) {{group.label}} - i.tipLabel.fa.fa-question-circle(ng-if='group.tip' bs-tooltip='joinTip(group.tip)' type='button') - i.tipLabel.fa.fa-question-circle.blank(ng-if='!group.tip') - .panel-collapse(bs-collapse-target) - .panel-body - .settings-row(ng-repeat='field in group.fields') - +form-row - .panel-title - h3 - a(bs-collapse-toggle='0' ng-click='expanded = !expanded;') {{expanded ? 'Hide advanced settings...' : 'Show advanced settings...'}} - div - button.btn.btn-primary(ng-disabled='inputForm.$invalid' ng-click='saveItem()') Save - button.btn.btn-primary(ng-show='backupItem._id' ng-disabled='inputForm.$invalid' ng-click='saveItemAs()') Copy - button.btn.btn-primary(ng-show='backupItem._id' ng-click='removeItem()') Remove \ No newline at end of file