Remove unsupported MapR plugin versions
MapR v3.1.1 v4.0.1 v4.0.2 and v5.0.0.mrv1 as they are old and rarely used. mapr-spark version is removed as it will go up with latest MapR as a service in Mitaka release. Change-Id: I74ae4094adc3707c777f961c8d23492d91f0aa38 Implements: blueprint deprecate-old-mapr-versions
This commit is contained in:
parent
cfcca9bdc3
commit
f3c26b9852
@ -16,10 +16,8 @@ import sahara.plugins.mapr.domain.configuration_file as bcf
|
||||
import sahara.plugins.mapr.domain.node_process as np
|
||||
import sahara.plugins.mapr.domain.service as s
|
||||
import sahara.plugins.mapr.util.validation_utils as vu
|
||||
from sahara.plugins.mapr.versions import version_handler_factory as vhf
|
||||
from sahara.swift import swift_helper
|
||||
from sahara.topology import topology_helper as topo
|
||||
from sahara.utils import files as f
|
||||
|
||||
JOB_TRACKER = np.NodeProcess(
|
||||
name='jobtracker',
|
||||
@ -34,11 +32,6 @@ TASK_TRACKER = np.NodeProcess(
|
||||
open_ports=[50060]
|
||||
)
|
||||
|
||||
JACKSON_CORE_ASL = ('plugins/mapr/services/swift/resources/'
|
||||
'jackson-core-asl-1.9.13.jar')
|
||||
JACKSON_MAPPER_ASL = ('plugins/mapr/services/swift/resources/'
|
||||
'jackson-mapper-asl-1.9.13.jar')
|
||||
|
||||
|
||||
class MapReduce(s.Service):
|
||||
cluster_mode = 'classic'
|
||||
@ -128,28 +121,6 @@ class MapReduce(s.Service):
|
||||
{'name': 'hadoop.proxyuser.mapr.hosts', 'value': '*'}
|
||||
]
|
||||
|
||||
def configure(self, cluster_context, instances=None):
|
||||
version = cluster_context.cluster.hadoop_version
|
||||
handler = vhf.VersionHandlerFactory.get().get_handler(version)
|
||||
if handler._version == '3.1.1':
|
||||
self._update_jackson_libs(cluster_context, instances)
|
||||
|
||||
def _update_jackson_libs(self, context, instances):
|
||||
hadoop_lib = context.hadoop_lib
|
||||
core_asl = f.get_file_text(JACKSON_CORE_ASL)
|
||||
mapper_asl = f.get_file_text(JACKSON_MAPPER_ASL)
|
||||
core_asl_path = '%s/%s' % (hadoop_lib, 'jackson-core-asl-1.9.13.jar')
|
||||
mapper_path = '%s/%s' % (hadoop_lib, 'jackson-mapper-asl-1.9.13.jar')
|
||||
libs = {
|
||||
core_asl_path: core_asl,
|
||||
mapper_path: mapper_asl
|
||||
}
|
||||
for instance in instances:
|
||||
with instance.remote() as r:
|
||||
r.execute_command('rm %s/jackson-*.jar' % hadoop_lib,
|
||||
run_as_root=True)
|
||||
r.write_files_to(libs, run_as_root=True)
|
||||
|
||||
def get_file_path(self, file_name):
|
||||
template = 'plugins/mapr/services/mapreduce/resources/%s'
|
||||
return template % file_name
|
||||
|
Binary file not shown.
Binary file not shown.
@ -1,49 +0,0 @@
|
||||
# Copyright (c) 2015, MapR Technologies
|
||||
#
|
||||
# Licensed 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.
|
||||
|
||||
|
||||
from sahara.plugins.mapr.base import base_cluster_context as bc
|
||||
from sahara.plugins.mapr.services.yarn import yarn
|
||||
|
||||
|
||||
class Context(bc.BaseClusterContext):
|
||||
def __init__(self, cluster, version_handler, added=None, removed=None):
|
||||
super(Context, self).__init__(cluster, version_handler, added, removed)
|
||||
self._hadoop_version = yarn.YARNv241().version
|
||||
self._hadoop_lib = None
|
||||
self._hadoop_conf = None
|
||||
self._resource_manager_uri = None
|
||||
self._cluster_mode = None
|
||||
self._node_aware = True
|
||||
self._mapr_version = '4.0.1'
|
||||
self._ubuntu_ecosystem_repo = (
|
||||
'http://package.mapr.com/releases/ecosystem-4.x/ubuntu binary/')
|
||||
self._centos_ecosystem_repo = (
|
||||
'http://package.mapr.com/releases/ecosystem-4.x/redhat')
|
||||
|
||||
@property
|
||||
def hadoop_lib(self):
|
||||
if not self._hadoop_lib:
|
||||
self._hadoop_lib = '%s/share/hadoop/common' % self.hadoop_home
|
||||
return self._hadoop_lib
|
||||
|
||||
@property
|
||||
def hadoop_conf(self):
|
||||
if not self._hadoop_conf:
|
||||
self._hadoop_conf = '%s/etc/hadoop' % self.hadoop_home
|
||||
return self._hadoop_conf
|
||||
|
||||
@property
|
||||
def resource_manager_uri(self):
|
||||
return self._resource_manager_uri
|
@ -1,154 +0,0 @@
|
||||
# Copyright (c) 2015, MapR Technologies
|
||||
#
|
||||
# Licensed 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.
|
||||
|
||||
|
||||
import os
|
||||
|
||||
from sahara import conductor as c
|
||||
from sahara import context
|
||||
from sahara import exceptions as e
|
||||
from sahara.i18n import _
|
||||
import sahara.plugins.mapr.services.spark.spark as spark
|
||||
import sahara.plugins.mapr.util.general as g
|
||||
import sahara.plugins.mapr.versions.version_handler_factory as vhf
|
||||
import sahara.plugins.utils as plugin_utils
|
||||
from sahara.service.edp import job_utils
|
||||
from sahara.service.edp.spark import engine as base_engine
|
||||
from sahara.swift import utils as su
|
||||
from sahara.utils import edp
|
||||
|
||||
conductor = c.API
|
||||
|
||||
|
||||
class MapRSparkEngine(base_engine.SparkJobEngine):
|
||||
def run_job(self, job_execution):
|
||||
ctx = context.ctx()
|
||||
job = conductor.job_get(ctx, job_execution.job_id)
|
||||
|
||||
additional_sources, updated_job_configs = (
|
||||
job_utils.resolve_data_source_references(job_execution.job_configs)
|
||||
)
|
||||
|
||||
# We'll always run the driver program on the master
|
||||
master = plugin_utils.get_instance(
|
||||
self.cluster, spark.SPARK_MASTER.ui_name)
|
||||
|
||||
# TODO(tmckay): wf_dir should probably be configurable.
|
||||
# The only requirement is that the dir is writable by the image user
|
||||
wf_dir = job_utils.create_workflow_dir(
|
||||
master, '/tmp/spark-edp', job, job_execution.id, "700")
|
||||
paths, builtin_paths = self._upload_job_files(
|
||||
master, wf_dir, job, updated_job_configs)
|
||||
|
||||
# We can shorten the paths in this case since we'll run out of wf_dir
|
||||
paths = [os.path.basename(p) for p in paths]
|
||||
builtin_paths = [os.path.basename(p) for p in builtin_paths]
|
||||
|
||||
# TODO(tmckay): for now, paths[0] is always assumed to be the app
|
||||
# jar and we generate paths in order (mains, then libs).
|
||||
# When we have a Spark job type, we can require a "main" and set
|
||||
# the app jar explicitly to be "main"
|
||||
app_jar = paths.pop(0)
|
||||
job_class = updated_job_configs["configs"]["edp.java.main_class"]
|
||||
|
||||
# If we uploaded builtins then we are using a wrapper jar. It will
|
||||
# be the first one on the builtin list and the original app_jar needs
|
||||
# to be added to the 'additional' jars
|
||||
if builtin_paths:
|
||||
wrapper_jar = builtin_paths.pop(0)
|
||||
wrapper_class = 'org.openstack.sahara.edp.SparkWrapper'
|
||||
wrapper_xml = self._upload_wrapper_xml(
|
||||
master, wf_dir, updated_job_configs)
|
||||
wrapper_args = "%s %s" % (wrapper_xml, job_class)
|
||||
additional_jars = ",".join([app_jar] + paths + builtin_paths)
|
||||
else:
|
||||
wrapper_jar = wrapper_class = wrapper_args = ""
|
||||
additional_jars = ",".join(paths)
|
||||
|
||||
# All additional jars are passed with the --jars option
|
||||
if additional_jars:
|
||||
additional_jars = " --jars " + additional_jars
|
||||
|
||||
# Launch the spark job using spark-submit and deploy_mode = client
|
||||
cluster_context = self._get_cluster_context(self.cluster)
|
||||
spark_home_dir = spark.Spark().home_dir(cluster_context)
|
||||
|
||||
# TODO(tmckay): we need to clean up wf_dirs on long running clusters
|
||||
# TODO(tmckay): probably allow for general options to spark-submit
|
||||
args = updated_job_configs.get('args', [])
|
||||
args = " ".join([su.inject_swift_url_suffix(arg) for arg in args])
|
||||
|
||||
submit_args = {
|
||||
"spark_submit": "%s/bin/spark-submit" % spark_home_dir,
|
||||
"addnl_jars": additional_jars,
|
||||
"master_url": spark.SPARK_MASTER.submit_url(cluster_context),
|
||||
"args": args
|
||||
}
|
||||
if wrapper_jar and wrapper_class:
|
||||
# Substrings which may be empty have spaces
|
||||
# embedded if they are non-empty
|
||||
submit_args.update({
|
||||
"driver_cp": self.get_driver_classpath(),
|
||||
"wrapper_class": wrapper_class,
|
||||
"wrapper_jar": wrapper_jar,
|
||||
"wrapper_args": wrapper_args,
|
||||
})
|
||||
submit_cmd = ('%(spark_submit)s%(driver_cp)s'
|
||||
' --class %(wrapper_class)s%(addnl_jars)s'
|
||||
' --master %(master_url)s'
|
||||
' %(wrapper_jar)s %(wrapper_args)s %(args)s')
|
||||
else:
|
||||
submit_args.update({
|
||||
"job_class": job_class,
|
||||
"app_jar": app_jar,
|
||||
})
|
||||
submit_cmd = ('%(spark_submit)s --class %(job_class)s'
|
||||
'%(addnl_jars)s --master %(master_url)s'
|
||||
' %(app_jar)s %(args)s')
|
||||
submit_cmd = g._run_as('mapr', submit_cmd % submit_args)
|
||||
|
||||
job_execution = conductor.job_execution_get(ctx, job_execution.id)
|
||||
if job_execution.info['status'] == edp.JOB_STATUS_TOBEKILLED:
|
||||
return (None, edp.JOB_STATUS_KILLED, None)
|
||||
|
||||
# If an exception is raised here, the job_manager will mark
|
||||
# the job failed and log the exception
|
||||
# The redirects of stdout and stderr will preserve output in the wf_dir
|
||||
with master.remote() as r:
|
||||
# Upload the command launch script
|
||||
launch = os.path.join(wf_dir, "launch_command")
|
||||
r.write_file_to(launch, self._job_script())
|
||||
r.execute_command("chmod +x %s" % launch)
|
||||
ret, stdout = r.execute_command(
|
||||
"cd %s && ./launch_command %s > /dev/null 2>&1 & echo $!"
|
||||
% (wf_dir, submit_cmd), raise_when_error=False)
|
||||
|
||||
if ret == 0:
|
||||
# Success, we'll add the wf_dir in job_execution.extra and store
|
||||
# pid@instance_id as the job id
|
||||
# We know the job is running so return "RUNNING"
|
||||
return (stdout.strip() + "@" + master.id,
|
||||
edp.JOB_STATUS_RUNNING,
|
||||
{'spark-path': wf_dir})
|
||||
|
||||
# Hmm, no execption but something failed.
|
||||
# Since we're using backgrounding with redirect, this is unlikely.
|
||||
raise e.EDPError(_("Spark job execution failed. Exit status = "
|
||||
"%(status)s, stdout = %(stdout)s") %
|
||||
{'status': ret, 'stdout': stdout})
|
||||
|
||||
def _get_cluster_context(self, cluster):
|
||||
version = cluster.hadoop_version
|
||||
handler = vhf.VersionHandlerFactory.get().get_handler(version)
|
||||
return handler.get_context(cluster)
|
@ -1,40 +0,0 @@
|
||||
# Copyright (c) 2015, MapR Technologies
|
||||
#
|
||||
# Licensed 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.
|
||||
|
||||
from oslo_log import log as logging
|
||||
|
||||
from sahara.plugins.mapr.base import base_node_manager
|
||||
from sahara.plugins.mapr.services.spark import spark
|
||||
|
||||
LOG = logging.getLogger(__name__)
|
||||
|
||||
|
||||
class SparkNodeManager(base_node_manager.BaseNodeManager):
|
||||
def start(self, cluster_context, instances=None):
|
||||
super(SparkNodeManager, self).start(cluster_context, instances)
|
||||
|
||||
instances = instances or cluster_context.added_instances()
|
||||
slaves = cluster_context.filter_instances(instances, spark.SPARK_SLAVE)
|
||||
if slaves:
|
||||
spark.SPARK_SLAVE.start(cluster_context, slaves)
|
||||
LOG.debug("Spark workers successfully started.")
|
||||
|
||||
def stop(self, cluster_context, instances=None):
|
||||
super(SparkNodeManager, self).stop(cluster_context, instances)
|
||||
|
||||
instances = instances or cluster_context.removed_instances()
|
||||
slaves = cluster_context.filter_instances(instances, spark.SPARK_SLAVE)
|
||||
if slaves:
|
||||
spark.SPARK_SLAVE.stop(cluster_context, slaves)
|
||||
LOG.debug("Spark workers successfully stopped.")
|
@ -1,56 +0,0 @@
|
||||
# Copyright (c) 2015, MapR Technologies
|
||||
#
|
||||
# Licensed 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.
|
||||
|
||||
|
||||
from sahara.plugins.mapr.base import base_version_handler as bvh
|
||||
from sahara.plugins.mapr.services.management import management
|
||||
from sahara.plugins.mapr.services.maprfs import maprfs
|
||||
from sahara.plugins.mapr.services.spark import spark
|
||||
from sahara.plugins.mapr.versions.mapr_spark import context as c
|
||||
from sahara.plugins.mapr.versions.mapr_spark import spark_engine as edp_engine
|
||||
from sahara.plugins.mapr.versions.mapr_spark import spark_node_manager
|
||||
|
||||
|
||||
version = 'spark'
|
||||
|
||||
|
||||
class VersionHandler(bvh.BaseVersionHandler):
|
||||
def __init__(self):
|
||||
super(VersionHandler, self).__init__()
|
||||
self._node_manager = spark_node_manager.SparkNodeManager()
|
||||
self._version = version
|
||||
self._required_services = [
|
||||
management.Management(),
|
||||
maprfs.MapRFS(),
|
||||
spark.Spark(),
|
||||
]
|
||||
self._services = [
|
||||
management.Management(),
|
||||
maprfs.MapRFS(),
|
||||
spark.Spark(),
|
||||
]
|
||||
|
||||
def get_context(self, cluster, added=None, removed=None):
|
||||
return c.Context(cluster, self, added, removed)
|
||||
|
||||
def get_edp_engine(self, cluster, job_type):
|
||||
if job_type in edp_engine.MapRSparkEngine.get_supported_job_types():
|
||||
return edp_engine.MapRSparkEngine(cluster)
|
||||
return None
|
||||
|
||||
def get_edp_job_types(self):
|
||||
return edp_engine.MapRSparkEngine.get_supported_job_types()
|
||||
|
||||
def get_edp_config_hints(self, job_type):
|
||||
return edp_engine.MapRSparkEngine.get_possible_job_config(job_type)
|
@ -1,58 +0,0 @@
|
||||
# Copyright (c) 2015, MapR Technologies
|
||||
#
|
||||
# Licensed 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.
|
||||
|
||||
|
||||
import sahara.plugins.mapr.base.base_cluster_context as bc
|
||||
import sahara.plugins.mapr.services.mapreduce.mapreduce as mr
|
||||
import sahara.plugins.mapr.services.maprfs.maprfs as maprfs
|
||||
|
||||
|
||||
class Context(bc.BaseClusterContext):
|
||||
def __init__(self, cluster, version_handler, added=None, removed=None):
|
||||
super(Context, self).__init__(cluster, version_handler, added, removed)
|
||||
self._hadoop_version = mr.MapReduce().version
|
||||
self._hadoop_lib = None
|
||||
self._hadoop_conf = None
|
||||
self._resource_manager_uri = 'maprfs:///'
|
||||
self._cluster_mode = None
|
||||
self._node_aware = True
|
||||
self._mapr_version = '3.1.1'
|
||||
self._ubuntu_ecosystem_repo = (
|
||||
'http://package.mapr.com/releases/ecosystem/ubuntu binary/')
|
||||
self._centos_ecosystem_repo = (
|
||||
'http://package.mapr.com/releases/ecosystem/redhat')
|
||||
|
||||
@property
|
||||
def hadoop_lib(self):
|
||||
if not self._hadoop_lib:
|
||||
self._hadoop_lib = '%s/lib' % self.hadoop_home
|
||||
return self._hadoop_lib
|
||||
|
||||
@property
|
||||
def hadoop_conf(self):
|
||||
if not self._hadoop_conf:
|
||||
self._hadoop_conf = '%s/conf' % self.hadoop_home
|
||||
return self._hadoop_conf
|
||||
|
||||
@property
|
||||
def resource_manager_uri(self):
|
||||
return self._resource_manager_uri
|
||||
|
||||
@property
|
||||
def mapr_db(self):
|
||||
if self._mapr_db is None:
|
||||
mapr_db = maprfs.MapRFS.ENABLE_MAPR_DB_CONFIG
|
||||
mapr_db = self._get_cluster_config_value(mapr_db)
|
||||
self._mapr_db = '-M7' if mapr_db else ''
|
||||
return self._mapr_db
|
@ -1,22 +0,0 @@
|
||||
# Copyright (c) 2015, MapR Technologies
|
||||
#
|
||||
# Licensed 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.
|
||||
|
||||
|
||||
import sahara.plugins.mapr.base.base_edp_engine as edp
|
||||
import sahara.plugins.mapr.util.maprfs_helper as mfs
|
||||
|
||||
|
||||
class MapR3OozieJobEngine(edp.MapROozieJobEngine):
|
||||
def create_hdfs_dir(self, remote, dir_name):
|
||||
mfs.create_maprfs3_dir(remote, dir_name, self.get_hdfs_user())
|
@ -1,82 +0,0 @@
|
||||
# Copyright (c) 2015, MapR Technologies
|
||||
#
|
||||
# Licensed 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.
|
||||
|
||||
|
||||
from sahara.plugins.mapr.base import base_version_handler as bvh
|
||||
from sahara.plugins.mapr.services.drill import drill
|
||||
from sahara.plugins.mapr.services.flume import flume
|
||||
from sahara.plugins.mapr.services.hbase import hbase
|
||||
from sahara.plugins.mapr.services.hive import hive
|
||||
from sahara.plugins.mapr.services.httpfs import httpfs
|
||||
from sahara.plugins.mapr.services.hue import hue
|
||||
from sahara.plugins.mapr.services.mahout import mahout
|
||||
from sahara.plugins.mapr.services.management import management
|
||||
from sahara.plugins.mapr.services.mapreduce import mapreduce
|
||||
from sahara.plugins.mapr.services.maprfs import maprfs
|
||||
from sahara.plugins.mapr.services.oozie import oozie
|
||||
from sahara.plugins.mapr.services.pig import pig
|
||||
from sahara.plugins.mapr.services.sqoop import sqoop2
|
||||
from sahara.plugins.mapr.services.swift import swift
|
||||
import sahara.plugins.mapr.versions.v3_1_1.context as c
|
||||
import sahara.plugins.mapr.versions.v3_1_1.edp_engine as edp
|
||||
|
||||
|
||||
version = '3.1.1'
|
||||
|
||||
|
||||
class VersionHandler(bvh.BaseVersionHandler):
|
||||
def __init__(self):
|
||||
super(VersionHandler, self).__init__()
|
||||
self._version = version
|
||||
self._required_services = [
|
||||
mapreduce.MapReduce(),
|
||||
maprfs.MapRFS(),
|
||||
management.Management(),
|
||||
oozie.Oozie(),
|
||||
]
|
||||
self._services = [
|
||||
mapreduce.MapReduce(),
|
||||
maprfs.MapRFS(),
|
||||
management.Management(),
|
||||
oozie.OozieV401(),
|
||||
hive.HiveV013(),
|
||||
hbase.HBaseV094(),
|
||||
hbase.HBaseV0987(),
|
||||
httpfs.HttpFS(),
|
||||
mahout.MahoutV09(),
|
||||
pig.PigV013(),
|
||||
pig.PigV014(),
|
||||
swift.Swift(),
|
||||
flume.Flume(),
|
||||
drill.DrillV07(),
|
||||
drill.DrillV08(),
|
||||
drill.DrillV09(),
|
||||
drill.DrillV12(),
|
||||
sqoop2.Sqoop2(),
|
||||
hue.HueV360(),
|
||||
]
|
||||
|
||||
def get_context(self, cluster, added=None, removed=None):
|
||||
return c.Context(cluster, self, added, removed)
|
||||
|
||||
def get_edp_engine(self, cluster, job_type):
|
||||
if job_type in edp.MapR3OozieJobEngine.get_supported_job_types():
|
||||
return edp.MapR3OozieJobEngine(cluster)
|
||||
return None
|
||||
|
||||
def get_edp_job_types(self):
|
||||
return edp.MapR3OozieJobEngine.get_supported_job_types()
|
||||
|
||||
def get_edp_config_hints(self, job_type):
|
||||
return edp.MapR3OozieJobEngine.get_possible_job_config(job_type)
|
@ -1,49 +0,0 @@
|
||||
# Copyright (c) 2015, MapR Technologies
|
||||
#
|
||||
# Licensed 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.
|
||||
|
||||
|
||||
import sahara.plugins.mapr.base.base_cluster_context as bc
|
||||
import sahara.plugins.mapr.services.mapreduce.mapreduce as mr
|
||||
|
||||
|
||||
class Context(bc.BaseClusterContext):
|
||||
def __init__(self, cluster, version_handler, added=None, removed=None):
|
||||
super(Context, self).__init__(cluster, version_handler, added, removed)
|
||||
self._hadoop_version = mr.MapReduce().version
|
||||
self._hadoop_lib = None
|
||||
self._hadoop_conf = None
|
||||
self._resource_manager_uri = 'maprfs:///'
|
||||
self._cluster_mode = mr.MapReduce.cluster_mode
|
||||
self._node_aware = True
|
||||
self._mapr_version = '4.0.1'
|
||||
self._ubuntu_ecosystem_repo = (
|
||||
'http://package.mapr.com/releases/ecosystem-4.x/ubuntu binary/')
|
||||
self._centos_ecosystem_repo = (
|
||||
'http://package.mapr.com/releases/ecosystem-4.x/redhat')
|
||||
|
||||
@property
|
||||
def hadoop_lib(self):
|
||||
if not self._hadoop_lib:
|
||||
self._hadoop_lib = '%s/lib' % self.hadoop_home
|
||||
return self._hadoop_lib
|
||||
|
||||
@property
|
||||
def hadoop_conf(self):
|
||||
if not self._hadoop_conf:
|
||||
self._hadoop_conf = '%s/conf' % self.hadoop_home
|
||||
return self._hadoop_conf
|
||||
|
||||
@property
|
||||
def resource_manager_uri(self):
|
||||
return self._resource_manager_uri
|
@ -1,71 +0,0 @@
|
||||
# Copyright (c) 2015, MapR Technologies
|
||||
#
|
||||
# Licensed 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.
|
||||
|
||||
|
||||
from sahara.plugins.mapr.base import base_version_handler as bvh
|
||||
from sahara.plugins.mapr.services.drill import drill
|
||||
from sahara.plugins.mapr.services.flume import flume
|
||||
from sahara.plugins.mapr.services.hbase import hbase
|
||||
from sahara.plugins.mapr.services.hive import hive
|
||||
from sahara.plugins.mapr.services.httpfs import httpfs
|
||||
from sahara.plugins.mapr.services.hue import hue
|
||||
from sahara.plugins.mapr.services.mahout import mahout
|
||||
from sahara.plugins.mapr.services.management import management
|
||||
from sahara.plugins.mapr.services.mapreduce import mapreduce
|
||||
from sahara.plugins.mapr.services.maprfs import maprfs
|
||||
from sahara.plugins.mapr.services.oozie import oozie
|
||||
from sahara.plugins.mapr.services.pig import pig
|
||||
from sahara.plugins.mapr.services.sqoop import sqoop2
|
||||
from sahara.plugins.mapr.services.swift import swift
|
||||
import sahara.plugins.mapr.versions.v4_0_1_mrv1.context as c
|
||||
|
||||
version = '4.0.1.mrv1'
|
||||
|
||||
|
||||
class VersionHandler(bvh.BaseVersionHandler):
|
||||
def __init__(self):
|
||||
super(VersionHandler, self).__init__()
|
||||
self._version = version
|
||||
self._required_services = [
|
||||
mapreduce.MapReduce(),
|
||||
maprfs.MapRFS(),
|
||||
management.Management(),
|
||||
oozie.Oozie(),
|
||||
]
|
||||
self._services = [
|
||||
maprfs.MapRFS(),
|
||||
management.Management(),
|
||||
oozie.OozieV401(),
|
||||
hive.HiveV013(),
|
||||
hbase.HBaseV094(),
|
||||
hbase.HBaseV0987(),
|
||||
hbase.HBaseV0989(),
|
||||
httpfs.HttpFS(),
|
||||
mahout.MahoutV09(),
|
||||
pig.PigV013(),
|
||||
pig.PigV014(),
|
||||
swift.Swift(),
|
||||
mapreduce.MapReduce(),
|
||||
flume.Flume(),
|
||||
drill.DrillV07(),
|
||||
drill.DrillV08(),
|
||||
drill.DrillV09(),
|
||||
drill.DrillV12(),
|
||||
sqoop2.Sqoop2(),
|
||||
hue.HueV360(),
|
||||
hue.HueV370(),
|
||||
]
|
||||
|
||||
def get_context(self, cluster, added=None, removed=None):
|
||||
return c.Context(cluster, self, added, removed)
|
@ -1,64 +0,0 @@
|
||||
# Copyright (c) 2015, MapR Technologies
|
||||
#
|
||||
# Licensed 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.
|
||||
|
||||
|
||||
import sahara.plugins.mapr.base.base_cluster_context as bc
|
||||
import sahara.plugins.mapr.services.yarn.yarn as yarn
|
||||
|
||||
|
||||
class Context(bc.BaseClusterContext):
|
||||
def __init__(self, cluster, version_handler, added=None, removed=None):
|
||||
super(Context, self).__init__(cluster, version_handler, added, removed)
|
||||
self._hadoop_version = yarn.YARNv241().version
|
||||
self._hadoop_lib = None
|
||||
self._hadoop_conf = None
|
||||
self._resource_manager_uri = None
|
||||
self._cluster_mode = yarn.YARNv241.cluster_mode
|
||||
self._node_aware = True
|
||||
self._mapr_version = '4.0.1'
|
||||
self._ubuntu_ecosystem_repo = (
|
||||
'http://package.mapr.com/releases/ecosystem-4.x/ubuntu binary/')
|
||||
self._centos_ecosystem_repo = (
|
||||
'http://package.mapr.com/releases/ecosystem-4.x/redhat')
|
||||
|
||||
@property
|
||||
def hadoop_lib(self):
|
||||
if not self._hadoop_lib:
|
||||
self._hadoop_lib = '%s/share/hadoop/common' % self.hadoop_home
|
||||
return self._hadoop_lib
|
||||
|
||||
@property
|
||||
def hadoop_conf(self):
|
||||
if not self._hadoop_conf:
|
||||
self._hadoop_conf = '%s/etc/hadoop' % self.hadoop_home
|
||||
return self._hadoop_conf
|
||||
|
||||
@property
|
||||
def resource_manager_uri(self):
|
||||
if not self._resource_manager_uri:
|
||||
ip = self.get_instance(yarn.RESOURCE_MANAGER).internal_ip
|
||||
self._resource_manager_uri = '%s:8032' % ip
|
||||
return self._resource_manager_uri
|
||||
|
||||
@property
|
||||
def configure_sh(self):
|
||||
if not self._configure_sh:
|
||||
f = '%(base)s -RM %(resource_manager)s -HS %(history_server)s'
|
||||
args = {
|
||||
'base': super(Context, self).configure_sh,
|
||||
'resource_manager': self.get_resourcemanager_ip(),
|
||||
'history_server': self.get_historyserver_ip(),
|
||||
}
|
||||
self._configure_sh = f % args
|
||||
return self._configure_sh
|
@ -1,72 +0,0 @@
|
||||
# Copyright (c) 2015, MapR Technologies
|
||||
#
|
||||
# Licensed 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.
|
||||
|
||||
|
||||
from sahara.plugins.mapr.base import base_version_handler as bvh
|
||||
from sahara.plugins.mapr.services.drill import drill
|
||||
from sahara.plugins.mapr.services.flume import flume
|
||||
from sahara.plugins.mapr.services.hbase import hbase
|
||||
from sahara.plugins.mapr.services.hive import hive
|
||||
from sahara.plugins.mapr.services.httpfs import httpfs
|
||||
from sahara.plugins.mapr.services.hue import hue
|
||||
from sahara.plugins.mapr.services.mahout import mahout
|
||||
from sahara.plugins.mapr.services.management import management
|
||||
from sahara.plugins.mapr.services.maprfs import maprfs
|
||||
from sahara.plugins.mapr.services.oozie import oozie
|
||||
from sahara.plugins.mapr.services.pig import pig
|
||||
from sahara.plugins.mapr.services.sqoop import sqoop2
|
||||
from sahara.plugins.mapr.services.swift import swift
|
||||
from sahara.plugins.mapr.services.yarn import yarn
|
||||
import sahara.plugins.mapr.versions.v4_0_1_mrv2.context as c
|
||||
|
||||
|
||||
version = '4.0.1.mrv2'
|
||||
|
||||
|
||||
class VersionHandler(bvh.BaseVersionHandler):
|
||||
def __init__(self):
|
||||
super(VersionHandler, self).__init__()
|
||||
self._version = version
|
||||
self._required_services = [
|
||||
yarn.YARNv241(),
|
||||
maprfs.MapRFS(),
|
||||
management.Management(),
|
||||
oozie.Oozie(),
|
||||
]
|
||||
self._services = [
|
||||
maprfs.MapRFS(),
|
||||
management.Management(),
|
||||
oozie.OozieV401(),
|
||||
hive.HiveV013(),
|
||||
hbase.HBaseV094(),
|
||||
hbase.HBaseV0987(),
|
||||
hbase.HBaseV0989(),
|
||||
httpfs.HttpFS(),
|
||||
mahout.MahoutV09(),
|
||||
pig.PigV013(),
|
||||
pig.PigV014(),
|
||||
swift.Swift(),
|
||||
yarn.YARNv241(),
|
||||
flume.Flume(),
|
||||
drill.DrillV07(),
|
||||
drill.DrillV08(),
|
||||
drill.DrillV09(),
|
||||
drill.DrillV12(),
|
||||
sqoop2.Sqoop2(),
|
||||
hue.HueV360(),
|
||||
hue.HueV370(),
|
||||
]
|
||||
|
||||
def get_context(self, cluster, added=None, removed=None):
|
||||
return c.Context(cluster, self, added, removed)
|
@ -1,49 +0,0 @@
|
||||
# Copyright (c) 2015, MapR Technologies
|
||||
#
|
||||
# Licensed 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.
|
||||
|
||||
|
||||
import sahara.plugins.mapr.base.base_cluster_context as bc
|
||||
import sahara.plugins.mapr.services.mapreduce.mapreduce as mr
|
||||
|
||||
|
||||
class Context(bc.BaseClusterContext):
|
||||
def __init__(self, cluster, version_handler, added=None, removed=None):
|
||||
super(Context, self).__init__(cluster, version_handler, added, removed)
|
||||
self._hadoop_version = mr.MapReduce().version
|
||||
self._hadoop_lib = None
|
||||
self._hadoop_conf = None
|
||||
self._resource_manager_uri = 'maprfs:///'
|
||||
self._cluster_mode = mr.MapReduce.cluster_mode
|
||||
self._node_aware = True
|
||||
self._mapr_version = '4.0.2'
|
||||
self._ubuntu_ecosystem_repo = (
|
||||
'http://package.mapr.com/releases/ecosystem-4.x/ubuntu binary/')
|
||||
self._centos_ecosystem_repo = (
|
||||
'http://package.mapr.com/releases/ecosystem-4.x/redhat')
|
||||
|
||||
@property
|
||||
def hadoop_lib(self):
|
||||
if not self._hadoop_lib:
|
||||
self._hadoop_lib = '%s/lib' % self.hadoop_home
|
||||
return self._hadoop_lib
|
||||
|
||||
@property
|
||||
def hadoop_conf(self):
|
||||
if not self._hadoop_conf:
|
||||
self._hadoop_conf = '%s/conf' % self.hadoop_home
|
||||
return self._hadoop_conf
|
||||
|
||||
@property
|
||||
def resource_manager_uri(self):
|
||||
return self._resource_manager_uri
|
@ -1,74 +0,0 @@
|
||||
# Copyright (c) 2015, MapR Technologies
|
||||
#
|
||||
# Licensed 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.
|
||||
|
||||
|
||||
from sahara.plugins.mapr.base import base_version_handler as bvh
|
||||
from sahara.plugins.mapr.services.drill import drill
|
||||
from sahara.plugins.mapr.services.flume import flume
|
||||
from sahara.plugins.mapr.services.hbase import hbase
|
||||
from sahara.plugins.mapr.services.hive import hive
|
||||
from sahara.plugins.mapr.services.httpfs import httpfs
|
||||
from sahara.plugins.mapr.services.hue import hue
|
||||
from sahara.plugins.mapr.services.impala import impala
|
||||
from sahara.plugins.mapr.services.mahout import mahout
|
||||
from sahara.plugins.mapr.services.management import management as mng
|
||||
from sahara.plugins.mapr.services.mapreduce import mapreduce
|
||||
from sahara.plugins.mapr.services.maprfs import maprfs
|
||||
from sahara.plugins.mapr.services.oozie import oozie
|
||||
from sahara.plugins.mapr.services.pig import pig
|
||||
from sahara.plugins.mapr.services.sqoop import sqoop2
|
||||
from sahara.plugins.mapr.services.swift import swift
|
||||
import sahara.plugins.mapr.versions.v4_0_2_mrv1.context as c
|
||||
|
||||
|
||||
version = '4.0.2.mrv1'
|
||||
|
||||
|
||||
class VersionHandler(bvh.BaseVersionHandler):
|
||||
def __init__(self):
|
||||
super(VersionHandler, self).__init__()
|
||||
self._version = version
|
||||
self._required_services = [
|
||||
mapreduce.MapReduce(),
|
||||
maprfs.MapRFS(),
|
||||
mng.Management(),
|
||||
oozie.Oozie(),
|
||||
]
|
||||
self._services = [
|
||||
mapreduce.MapReduce(),
|
||||
maprfs.MapRFS(),
|
||||
mng.Management(),
|
||||
oozie.OozieV401(),
|
||||
oozie.OozieV410(),
|
||||
hive.HiveV013(),
|
||||
flume.Flume(),
|
||||
hbase.HBaseV0987(),
|
||||
hbase.HBaseV0989(),
|
||||
hue.HueV360(),
|
||||
hue.HueV370(),
|
||||
httpfs.HttpFS(),
|
||||
mahout.MahoutV09(),
|
||||
pig.PigV013(),
|
||||
pig.PigV014(),
|
||||
sqoop2.Sqoop2(),
|
||||
impala.ImpalaV141(),
|
||||
swift.Swift(),
|
||||
drill.DrillV07(),
|
||||
drill.DrillV08(),
|
||||
drill.DrillV09(),
|
||||
drill.DrillV12(),
|
||||
]
|
||||
|
||||
def get_context(self, cluster, added=None, removed=None):
|
||||
return c.Context(cluster, self, added, removed)
|
@ -1,65 +0,0 @@
|
||||
# Copyright (c) 2015, MapR Technologies
|
||||
#
|
||||
# Licensed 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.
|
||||
|
||||
|
||||
import sahara.plugins.mapr.base.base_cluster_context as bc
|
||||
import sahara.plugins.mapr.services.yarn.yarn as yarn
|
||||
|
||||
|
||||
class Context(bc.BaseClusterContext):
|
||||
def __init__(self, cluster, version_handler, added=None, removed=None):
|
||||
super(Context, self).__init__(cluster, version_handler, added, removed)
|
||||
self._hadoop_version = yarn.YARNv251().version
|
||||
self._hadoop_lib = None
|
||||
self._hadoop_conf = None
|
||||
self._cluster_mode = yarn.YARNv251.cluster_mode
|
||||
self._node_aware = True
|
||||
self._resource_manager_uri = None
|
||||
self._mapr_version = '4.0.2'
|
||||
self._ubuntu_ecosystem_repo = (
|
||||
'http://package.mapr.com/releases/ecosystem-4.x/ubuntu binary/')
|
||||
self._centos_ecosystem_repo = (
|
||||
'http://package.mapr.com/releases/ecosystem-4.x/redhat')
|
||||
|
||||
@property
|
||||
def hadoop_lib(self):
|
||||
if not self._hadoop_lib:
|
||||
self._hadoop_lib = '%s/share/hadoop/common' % self.hadoop_home
|
||||
return self._hadoop_lib
|
||||
|
||||
@property
|
||||
def hadoop_conf(self):
|
||||
if not self._hadoop_conf:
|
||||
self._hadoop_conf = '%s/etc/hadoop' % self.hadoop_home
|
||||
return self._hadoop_conf
|
||||
|
||||
@property
|
||||
def resource_manager_uri(self):
|
||||
# FIXME(aosadchyi): Wait for RM HA to work properly
|
||||
if not self._resource_manager_uri:
|
||||
ip = self.get_instance(yarn.RESOURCE_MANAGER).internal_ip
|
||||
self._resource_manager_uri = '%s:8032' % ip
|
||||
return self._resource_manager_uri
|
||||
|
||||
@property
|
||||
def configure_sh(self):
|
||||
if not self._configure_sh:
|
||||
f = '%(base)s -RM %(resource_manager)s -HS %(history_server)s'
|
||||
args = {
|
||||
'base': super(Context, self).configure_sh,
|
||||
'resource_manager': self.get_resourcemanager_ip(),
|
||||
'history_server': self.get_historyserver_ip(),
|
||||
}
|
||||
self._configure_sh = f % args
|
||||
return self._configure_sh
|
@ -1,74 +0,0 @@
|
||||
# Copyright (c) 2015, MapR Technologies
|
||||
#
|
||||
# Licensed 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.
|
||||
|
||||
|
||||
import sahara.plugins.mapr.base.base_version_handler as bvh
|
||||
from sahara.plugins.mapr.services.drill import drill
|
||||
from sahara.plugins.mapr.services.flume import flume
|
||||
from sahara.plugins.mapr.services.hbase import hbase
|
||||
from sahara.plugins.mapr.services.hive import hive
|
||||
from sahara.plugins.mapr.services.httpfs import httpfs
|
||||
from sahara.plugins.mapr.services.hue import hue
|
||||
from sahara.plugins.mapr.services.impala import impala
|
||||
from sahara.plugins.mapr.services.mahout import mahout
|
||||
from sahara.plugins.mapr.services.management import management as mng
|
||||
from sahara.plugins.mapr.services.maprfs import maprfs
|
||||
from sahara.plugins.mapr.services.oozie import oozie
|
||||
from sahara.plugins.mapr.services.pig import pig
|
||||
from sahara.plugins.mapr.services.sqoop import sqoop2
|
||||
from sahara.plugins.mapr.services.swift import swift
|
||||
from sahara.plugins.mapr.services.yarn import yarn
|
||||
import sahara.plugins.mapr.versions.v4_0_2_mrv2.context as c
|
||||
|
||||
|
||||
version = '4.0.2.mrv2'
|
||||
|
||||
|
||||
class VersionHandler(bvh.BaseVersionHandler):
|
||||
def __init__(self):
|
||||
super(VersionHandler, self).__init__()
|
||||
self._version = version
|
||||
self._required_services = [
|
||||
yarn.YARNv251(),
|
||||
maprfs.MapRFS(),
|
||||
mng.Management(),
|
||||
oozie.Oozie(),
|
||||
]
|
||||
self._services = [
|
||||
yarn.YARNv251(),
|
||||
maprfs.MapRFS(),
|
||||
mng.Management(),
|
||||
oozie.OozieV401(),
|
||||
oozie.OozieV410(),
|
||||
hive.HiveV013(),
|
||||
impala.ImpalaV141(),
|
||||
flume.Flume(),
|
||||
hbase.HBaseV0987(),
|
||||
hbase.HBaseV0989(),
|
||||
hue.HueV360(),
|
||||
hue.HueV370(),
|
||||
httpfs.HttpFS(),
|
||||
mahout.MahoutV09(),
|
||||
pig.PigV013(),
|
||||
pig.PigV014(),
|
||||
sqoop2.Sqoop2(),
|
||||
swift.Swift(),
|
||||
drill.DrillV07(),
|
||||
drill.DrillV08(),
|
||||
drill.DrillV09(),
|
||||
drill.DrillV12(),
|
||||
]
|
||||
|
||||
def get_context(self, cluster, added=None, removed=None):
|
||||
return c.Context(cluster, self, added, removed)
|
@ -1,49 +0,0 @@
|
||||
# Copyright (c) 2015, MapR Technologies
|
||||
#
|
||||
# Licensed 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.
|
||||
|
||||
|
||||
import sahara.plugins.mapr.base.base_cluster_context as bc
|
||||
import sahara.plugins.mapr.services.mapreduce.mapreduce as mr
|
||||
|
||||
|
||||
class Context(bc.BaseClusterContext):
|
||||
def __init__(self, cluster, version_handler, added=None, removed=None):
|
||||
super(Context, self).__init__(cluster, version_handler, added, removed)
|
||||
self._hadoop_version = mr.MapReduce().version
|
||||
self._hadoop_lib = None
|
||||
self._hadoop_conf = None
|
||||
self._resource_manager_uri = "maprfs:///"
|
||||
self._cluster_mode = mr.MapReduce.cluster_mode
|
||||
self._node_aware = True
|
||||
self._mapr_version = '5.0.0'
|
||||
self._ubuntu_ecosystem_repo = (
|
||||
"http://package.mapr.com/releases/ecosystem-5.x/ubuntu binary/")
|
||||
self._centos_ecosystem_repo = (
|
||||
"http://package.mapr.com/releases/ecosystem-5.x/redhat")
|
||||
|
||||
@property
|
||||
def hadoop_lib(self):
|
||||
if not self._hadoop_lib:
|
||||
self._hadoop_lib = "%s/lib" % self.hadoop_home
|
||||
return self._hadoop_lib
|
||||
|
||||
@property
|
||||
def hadoop_conf(self):
|
||||
if not self._hadoop_conf:
|
||||
self._hadoop_conf = "%s/conf" % self.hadoop_home
|
||||
return self._hadoop_conf
|
||||
|
||||
@property
|
||||
def resource_manager_uri(self):
|
||||
return self._resource_manager_uri
|
@ -1,71 +0,0 @@
|
||||
# Copyright (c) 2015, MapR Technologies
|
||||
#
|
||||
# Licensed 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.
|
||||
|
||||
|
||||
from sahara.plugins.mapr.base import base_version_handler as bvh
|
||||
from sahara.plugins.mapr.services.drill import drill
|
||||
from sahara.plugins.mapr.services.flume import flume
|
||||
from sahara.plugins.mapr.services.hbase import hbase
|
||||
from sahara.plugins.mapr.services.hive import hive
|
||||
from sahara.plugins.mapr.services.httpfs import httpfs
|
||||
from sahara.plugins.mapr.services.hue import hue
|
||||
from sahara.plugins.mapr.services.impala import impala
|
||||
from sahara.plugins.mapr.services.mahout import mahout
|
||||
from sahara.plugins.mapr.services.management import management as mng
|
||||
from sahara.plugins.mapr.services.mapreduce import mapreduce
|
||||
from sahara.plugins.mapr.services.maprfs import maprfs
|
||||
from sahara.plugins.mapr.services.oozie import oozie
|
||||
from sahara.plugins.mapr.services.pig import pig
|
||||
from sahara.plugins.mapr.services.sqoop import sqoop2
|
||||
from sahara.plugins.mapr.services.swift import swift
|
||||
import sahara.plugins.mapr.versions.v5_0_0_mrv1.context as c
|
||||
|
||||
|
||||
version = "5.0.0.mrv1"
|
||||
|
||||
|
||||
class VersionHandler(bvh.BaseVersionHandler):
|
||||
def __init__(self):
|
||||
super(VersionHandler, self).__init__()
|
||||
self._version = version
|
||||
self._required_services = [
|
||||
mapreduce.MapReduce(),
|
||||
maprfs.MapRFS(),
|
||||
mng.Management(),
|
||||
oozie.Oozie(),
|
||||
]
|
||||
self._services = [
|
||||
hive.HiveV013(),
|
||||
hive.HiveV10(),
|
||||
impala.ImpalaV141(),
|
||||
pig.PigV014(),
|
||||
flume.Flume(),
|
||||
sqoop2.Sqoop2(),
|
||||
mahout.MahoutV010(),
|
||||
oozie.OozieV410(),
|
||||
hue.HueV370(),
|
||||
hue.HueV381(),
|
||||
hbase.HBaseV0989(),
|
||||
hbase.HBaseV09812(),
|
||||
drill.DrillV11(),
|
||||
drill.DrillV12(),
|
||||
mapreduce.MapReduce(),
|
||||
maprfs.MapRFS(),
|
||||
mng.Management(),
|
||||
httpfs.HttpFS(),
|
||||
swift.Swift(),
|
||||
]
|
||||
|
||||
def get_context(self, cluster, added=None, removed=None):
|
||||
return c.Context(cluster, self, added, removed)
|
@ -20,7 +20,7 @@ from sahara.plugins.mapr.services.management import management
|
||||
from sahara.plugins.mapr.services.maprfs import maprfs
|
||||
from sahara.plugins.mapr.services.oozie import oozie
|
||||
from sahara.plugins.mapr.services.yarn import yarn
|
||||
import sahara.plugins.mapr.versions.v4_0_1_mrv2.version_handler as handler
|
||||
import sahara.plugins.mapr.versions.v5_0_0_mrv2.version_handler as handler
|
||||
from sahara.plugins import provisioning as p
|
||||
from sahara.tests.unit import base as b
|
||||
from sahara.tests.unit import testutils as tu
|
||||
@ -56,7 +56,7 @@ class TestHandler(b.SaharaTestCase):
|
||||
name='test_cluster',
|
||||
tenant='large',
|
||||
plugin='mapr',
|
||||
version='4.0.1.mrv1',
|
||||
version='5.0.0.mrv2',
|
||||
node_groups=[master_ng],
|
||||
cluster_configs=cluster_configs,
|
||||
)
|
||||
|
@ -21,8 +21,8 @@ from sahara.plugins.mapr.services.maprfs import maprfs
|
||||
from sahara.plugins.mapr.services.oozie import oozie
|
||||
from sahara.plugins.mapr.services.swift import swift
|
||||
from sahara.plugins.mapr.services.yarn import yarn
|
||||
import sahara.plugins.mapr.versions.v4_0_1_mrv2.context as cc
|
||||
import sahara.plugins.mapr.versions.v4_0_1_mrv2.version_handler as handler
|
||||
import sahara.plugins.mapr.versions.v5_0_0_mrv2.context as cc
|
||||
import sahara.plugins.mapr.versions.v5_0_0_mrv2.version_handler as handler
|
||||
from sahara.plugins import provisioning as p
|
||||
from sahara.tests.unit import base as b
|
||||
from sahara.tests.unit import testutils as tu
|
||||
@ -56,14 +56,14 @@ class TestClusterContext(b.SaharaTestCase):
|
||||
'Service Version': '1.1',
|
||||
},
|
||||
'Oozie': {
|
||||
'Oozie Version': '4.0.1',
|
||||
'Oozie Version': '4.1.0',
|
||||
}
|
||||
}
|
||||
cluster = tu.create_cluster(
|
||||
name='test_cluster',
|
||||
tenant='large',
|
||||
plugin='mapr',
|
||||
version='4.0.1.mrv1',
|
||||
version='5.0.0.mrv2',
|
||||
node_groups=[master_ng],
|
||||
cluster_configs=cluster_configs,
|
||||
)
|
||||
@ -90,12 +90,11 @@ class TestClusterContext(b.SaharaTestCase):
|
||||
ctx = self._get_context()
|
||||
conf_sh = ctx.configure_sh
|
||||
pattern = (r'^(\S+)\s+(-N (\S+))\s+(-C (\S+))\s+(-Z (\S+))\s+'
|
||||
r'(-no-autostart)\s+(-f)\s+(-RM (\S+))\s(-HS (\S+))')
|
||||
r'(-no-autostart)\s+(-f)\s+\s(-HS (\S+))')
|
||||
self.assertRegex(conf_sh, pattern)
|
||||
self.assertIn('/opt/mapr/server/configure.sh', conf_sh)
|
||||
self.assertIn('-C %s' % INTERNAL_IP, conf_sh)
|
||||
self.assertIn('-Z %s' % INTERNAL_IP, conf_sh)
|
||||
self.assertIn('-RM %s' % INTERNAL_IP, conf_sh)
|
||||
self.assertIn('-HS %s' % INTERNAL_IP, conf_sh)
|
||||
self.assertIn('-no-autostart', conf_sh)
|
||||
self.assertIn('-N ' + ctx.cluster.name, conf_sh)
|
||||
@ -282,5 +281,5 @@ class TestClusterContext(b.SaharaTestCase):
|
||||
cluster_context = self._get_context()
|
||||
|
||||
self.assertTrue(cluster_context.is_present(oozie.Oozie()))
|
||||
self.assertTrue(cluster_context.is_present(oozie.OozieV401()))
|
||||
self.assertFalse(cluster_context.is_present(oozie.OozieV410()))
|
||||
self.assertFalse(cluster_context.is_present(oozie.OozieV401()))
|
||||
self.assertTrue(cluster_context.is_present(oozie.OozieV410()))
|
||||
|
Loading…
x
Reference in New Issue
Block a user