Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 93452200AF9 for ; Thu, 5 May 2016 15:46:43 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 91FE1160A03; Thu, 5 May 2016 13:46:43 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 58123160A11 for ; Thu, 5 May 2016 15:46:41 +0200 (CEST) Received: (qmail 90247 invoked by uid 500); 5 May 2016 13:46:40 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 89551 invoked by uid 99); 5 May 2016 13:46:38 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 05 May 2016 13:46:38 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 84595E9114; Thu, 5 May 2016 13:46:38 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 8bit From: vozerov@apache.org To: commits@ignite.apache.org Date: Thu, 05 May 2016 13:46:50 -0000 Message-Id: In-Reply-To: <6bf70c6f35704c72b4d9c64edc021f12@git.apache.org> References: <6bf70c6f35704c72b4d9c64edc021f12@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [13/31] ignite git commit: IGNITE-1786: Implemented ODBC driver. archived-at: Thu, 05 May 2016 13:46:43 -0000 http://git-wip-us.apache.org/repos/asf/ignite/blob/764c97b9/modules/platforms/cpp/jni/include/ignite/jni/java.h ---------------------------------------------------------------------- diff --git a/modules/platforms/cpp/jni/include/ignite/jni/java.h b/modules/platforms/cpp/jni/include/ignite/jni/java.h new file mode 100644 index 0000000..5ffbb8a --- /dev/null +++ b/modules/platforms/cpp/jni/include/ignite/jni/java.h @@ -0,0 +1,745 @@ +/* + * 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. + */ + +#ifndef _IGNITE_JNI_JAVA +#define _IGNITE_JNI_JAVA + +#include + +#include "ignite/common/common.h" +#include "ignite/ignite_error.h" + +namespace ignite +{ + namespace jni + { + namespace java + { + /* Handlers for callbacks from Java. */ + typedef long long(JNICALL *CacheStoreCreateHandler)(void* target, long long memPtr); + typedef int(JNICALL *CacheStoreInvokeHandler)(void* target, long long objPtr, long long memPtr, void* cb); + typedef void(JNICALL *CacheStoreDestroyHandler)(void* target, long long objPtr); + typedef long long(JNICALL *CacheStoreSessionCreateHandler)(void* target, long long storePtr); + + typedef long long(JNICALL *CacheEntryFilterCreateHandler)(void* target, long long memPtr); + typedef int(JNICALL *CacheEntryFilterApplyHandler)(void* target, long long ptr, long long memPtr); + typedef void(JNICALL *CacheEntryFilterDestroyHandler)(void* target, long long ptr); + + typedef void(JNICALL *CacheInvokeHandler)(void* target, long long inMemPtr, long long outMemPtr); + + typedef void(JNICALL *ComputeTaskMapHandler)(void* target, long long taskPtr, long long inMemPtr, long long outMemPtr); + typedef int(JNICALL *ComputeTaskJobResultHandler)(void* target, long long taskPtr, long long jobPtr, long long memPtr); + typedef void(JNICALL *ComputeTaskReduceHandler)(void* target, long long taskPtr); + typedef void(JNICALL *ComputeTaskCompleteHandler)(void* target, long long taskPtr, long long memPtr); + typedef int(JNICALL *ComputeJobSerializeHandler)(void* target, long long jobPtr, long long memPtr); + typedef long long(JNICALL *ComputeJobCreateHandler)(void* target, long long memPtr); + typedef void(JNICALL *ComputeJobExecuteHandler)(void* target, long long jobPtr, int cancel, long long memPtr); + typedef void(JNICALL *ComputeJobCancelHandler)(void* target, long long jobPtr); + typedef void(JNICALL *ComputeJobDestroyHandler)(void* target, long long jobPtr); + + typedef void(JNICALL *ContinuousQueryListenerApplyHandler)(void* target, long long lsnrPtr, long long memPtr); + typedef long long(JNICALL *ContinuousQueryFilterCreateHandler)(void* target, long long memPtr); + typedef int(JNICALL *ContinuousQueryFilterApplyHandler)(void* target, long long filterPtr, long long memPtr); + typedef void(JNICALL *ContinuousQueryFilterReleaseHandler)(void* target, long long filterPtr); + + typedef void(JNICALL *DataStreamerTopologyUpdateHandler)(void* target, long long ldrPtr, long long topVer, int topSize); + typedef void(JNICALL *DataStreamerStreamReceiverInvokeHandler)(void* target, long long ptr, void* cache, long long memPtr, unsigned char keepPortable); + + typedef void(JNICALL *FutureByteResultHandler)(void* target, long long futAddr, int res); + typedef void(JNICALL *FutureBoolResultHandler)(void* target, long long futAddr, int res); + typedef void(JNICALL *FutureShortResultHandler)(void* target, long long futAddr, int res); + typedef void(JNICALL *FutureCharResultHandler)(void* target, long long futAddr, int res); + typedef void(JNICALL *FutureIntResultHandler)(void* target, long long futAddr, int res); + typedef void(JNICALL *FutureFloatResultHandler)(void* target, long long futAddr, float res); + typedef void(JNICALL *FutureLongResultHandler)(void* target, long long futAddr, long long res); + typedef void(JNICALL *FutureDoubleResultHandler)(void* target, long long futAddr, double res); + typedef void(JNICALL *FutureObjectResultHandler)(void* target, long long futAddr, long long memPtr); + typedef void(JNICALL *FutureNullResultHandler)(void* target, long long futAddr); + typedef void(JNICALL *FutureErrorHandler)(void* target, long long futAddr, long long memPtr); + + typedef void(JNICALL *LifecycleEventHandler)(void* target, long long ptr, int evt); + + typedef void(JNICALL *MemoryReallocateHandler)(void* target, long long memPtr, int cap); + + typedef long long(JNICALL *MessagingFilterCreateHandler)(void* target, long long memPtr); + typedef int(JNICALL *MessagingFilterApplyHandler)(void* target, long long ptr, long long memPtr); + typedef void(JNICALL *MessagingFilterDestroyHandler)(void* target, long long ptr); + + typedef long long(JNICALL *EventFilterCreateHandler)(void* target, long long memPtr); + typedef int(JNICALL *EventFilterApplyHandler)(void* target, long long ptr, long long memPtr); + typedef void(JNICALL *EventFilterDestroyHandler)(void* target, long long ptr); + + typedef long long(JNICALL *ServiceInitHandler)(void* target, long long memPtr); + typedef void(JNICALL *ServiceExecuteHandler)(void* target, long long svcPtr, long long memPtr); + typedef void(JNICALL *ServiceCancelHandler)(void* target, long long svcPtr, long long memPtr); + typedef void(JNICALL *ServiceInvokeMethodHandler)(void* target, long long svcPtr, long long inMemPtr, long long outMemPtr); + typedef int(JNICALL *ClusterNodeFilterApplyHandler)(void* target, long long memPtr); + + typedef long long(JNICALL *NodeInfoHandler)(void* target, long long memPtr); + + typedef void(JNICALL *OnStartHandler)(void* target, void* proc, long long memPtr); + typedef void(JNICALL *OnStopHandler)(void* target); + typedef void(JNICALL *ErrorHandler)(void* target, int errCode, const char* errClsChars, int errClsCharsLen, const char* errMsgChars, int errMsgCharsLen, void* errData, int errDataLen); + + typedef long long(JNICALL *ExtensionCallbackInLongOutLongHandler)(void* target, int typ, long long arg1); + typedef long long(JNICALL *ExtensionCallbackInLongLongOutLongHandler)(void* target, int typ, long long arg1, long long arg2); + + typedef void(JNICALL *OnClientDisconnectedHandler)(void* target); + typedef void(JNICALL *OnClientReconnectedHandler)(void* target, unsigned char clusterRestarted); + + /** + * JNI handlers holder. + */ + struct JniHandlers { + void* target; + + CacheStoreCreateHandler cacheStoreCreate; + CacheStoreInvokeHandler cacheStoreInvoke; + CacheStoreDestroyHandler cacheStoreDestroy; + CacheStoreSessionCreateHandler cacheStoreSessionCreate; + + CacheEntryFilterCreateHandler cacheEntryFilterCreate; + CacheEntryFilterApplyHandler cacheEntryFilterApply; + CacheEntryFilterDestroyHandler cacheEntryFilterDestroy; + + CacheInvokeHandler cacheInvoke; + + ComputeTaskMapHandler computeTaskMap; + ComputeTaskJobResultHandler computeTaskJobRes; + ComputeTaskReduceHandler computeTaskReduce; + ComputeTaskCompleteHandler computeTaskComplete; + ComputeJobSerializeHandler computeJobSerialize; + ComputeJobCreateHandler computeJobCreate; + ComputeJobExecuteHandler computeJobExec; + ComputeJobCancelHandler computeJobCancel; + ComputeJobDestroyHandler computeJobDestroy; + + ContinuousQueryListenerApplyHandler contQryLsnrApply; + ContinuousQueryFilterCreateHandler contQryFilterCreate; + ContinuousQueryFilterApplyHandler contQryFilterApply; + ContinuousQueryFilterReleaseHandler contQryFilterRelease; + + DataStreamerTopologyUpdateHandler dataStreamerTopologyUpdate; + DataStreamerStreamReceiverInvokeHandler streamReceiverInvoke; + + FutureByteResultHandler futByteRes; + FutureBoolResultHandler futBoolRes; + FutureShortResultHandler futShortRes; + FutureCharResultHandler futCharRes; + FutureIntResultHandler futIntRes; + FutureFloatResultHandler futFloatRes; + FutureLongResultHandler futLongRes; + FutureDoubleResultHandler futDoubleRes; + FutureObjectResultHandler futObjRes; + FutureNullResultHandler futNullRes; + FutureErrorHandler futErr; + + LifecycleEventHandler lifecycleEvt; + + MemoryReallocateHandler memRealloc; + + MessagingFilterCreateHandler messagingFilterCreate; + MessagingFilterApplyHandler messagingFilterApply; + MessagingFilterDestroyHandler messagingFilterDestroy; + + EventFilterCreateHandler eventFilterCreate; + EventFilterApplyHandler eventFilterApply; + EventFilterDestroyHandler eventFilterDestroy; + + ServiceInitHandler serviceInit; + ServiceExecuteHandler serviceExecute; + ServiceCancelHandler serviceCancel; + ServiceInvokeMethodHandler serviceInvokeMethod; + + ClusterNodeFilterApplyHandler clusterNodeFilterApply; + + NodeInfoHandler nodeInfo; + + OnStartHandler onStart; + OnStopHandler onStop; + ErrorHandler error; + + ExtensionCallbackInLongOutLongHandler extensionCallbackInLongOutLong; + ExtensionCallbackInLongLongOutLongHandler extensionCallbackInLongLongOutLong; + + OnClientDisconnectedHandler onClientDisconnected; + OnClientReconnectedHandler onClientReconnected; + }; + + /** + * JNI Java members. + */ + struct JniJavaMembers { + jclass c_Class; + jmethodID m_Class_getName; + + jclass c_Throwable; + jmethodID m_Throwable_getMessage; + jmethodID m_Throwable_printStackTrace; + + /** + * Constructor. + */ + void Initialize(JNIEnv* env); + + /** + * Destroy members releasing all allocated classes. + */ + void Destroy(JNIEnv* env); + + /** + * Write error information. + */ + bool WriteErrorInfo(JNIEnv* env, char** errClsName, int* errClsNameLen, char** errMsg, int* errMsgLen); + }; + + /** + * JNI members. + */ + struct JniMembers { + jclass c_PlatformAbstractQryCursor; + jmethodID m_PlatformAbstractQryCursor_iter; + jmethodID m_PlatformAbstractQryCursor_iterHasNext; + jmethodID m_PlatformAbstractQryCursor_close; + + jclass c_PlatformAffinity; + jmethodID m_PlatformAffinity_partitions; + + jclass c_PlatformCache; + jmethodID m_PlatformCache_withSkipStore; + jmethodID m_PlatformCache_withNoRetries; + jmethodID m_PlatformCache_withExpiryPolicy; + jmethodID m_PlatformCache_withAsync; + jmethodID m_PlatformCache_withKeepPortable; + jmethodID m_PlatformCache_clear; + jmethodID m_PlatformCache_removeAll; + jmethodID m_PlatformCache_iterator; + jmethodID m_PlatformCache_localIterator; + jmethodID m_PlatformCache_enterLock; + jmethodID m_PlatformCache_exitLock; + jmethodID m_PlatformCache_tryEnterLock; + jmethodID m_PlatformCache_closeLock; + jmethodID m_PlatformCache_rebalance; + jmethodID m_PlatformCache_size; + + jclass c_PlatformCacheStoreCallback; + jmethodID m_PlatformCacheStoreCallback_invoke; + + jclass c_IgniteException; + + jclass c_PlatformClusterGroup; + jmethodID m_PlatformClusterGroup_forOthers; + jmethodID m_PlatformClusterGroup_forRemotes; + jmethodID m_PlatformClusterGroup_forDaemons; + jmethodID m_PlatformClusterGroup_forRandom; + jmethodID m_PlatformClusterGroup_forOldest; + jmethodID m_PlatformClusterGroup_forYoungest; + jmethodID m_PlatformClusterGroup_resetMetrics; + + jclass c_PlatformCompute; + jmethodID m_PlatformCompute_withNoFailover; + jmethodID m_PlatformCompute_withTimeout; + jmethodID m_PlatformCompute_executeNative; + + jclass c_PlatformContinuousQuery; + jmethodID m_PlatformContinuousQuery_close; + jmethodID m_PlatformContinuousQuery_getInitialQueryCursor; + + jclass c_PlatformDataStreamer; + jmethodID m_PlatformDataStreamer_listenTopology; + jmethodID m_PlatformDataStreamer_getAllowOverwrite; + jmethodID m_PlatformDataStreamer_setAllowOverwrite; + jmethodID m_PlatformDataStreamer_getSkipStore; + jmethodID m_PlatformDataStreamer_setSkipStore; + jmethodID m_PlatformDataStreamer_getPerNodeBufSize; + jmethodID m_PlatformDataStreamer_setPerNodeBufSize; + jmethodID m_PlatformDataStreamer_getPerNodeParallelOps; + jmethodID m_PlatformDataStreamer_setPerNodeParallelOps; + + jclass c_PlatformEvents; + jmethodID m_PlatformEvents_withAsync; + jmethodID m_PlatformEvents_stopLocalListen; + jmethodID m_PlatformEvents_localListen; + jmethodID m_PlatformEvents_isEnabled; + + jclass c_PlatformServices; + jmethodID m_PlatformServices_withAsync; + jmethodID m_PlatformServices_withServerKeepPortable; + jmethodID m_PlatformServices_cancel; + jmethodID m_PlatformServices_cancelAll; + jmethodID m_PlatformServices_serviceProxy; + + jclass c_PlatformIgnition; + jmethodID m_PlatformIgnition_start; + jmethodID m_PlatformIgnition_instance; + jmethodID m_PlatformIgnition_environmentPointer; + jmethodID m_PlatformIgnition_stop; + jmethodID m_PlatformIgnition_stopAll; + + jclass c_PlatformMessaging; + jmethodID m_PlatformMessaging_withAsync; + + jclass c_PlatformProcessor; + jmethodID m_PlatformProcessor_releaseStart; + jmethodID m_PlatformProcessor_cache; + jmethodID m_PlatformProcessor_createCache; + jmethodID m_PlatformProcessor_getOrCreateCache; + jmethodID m_PlatformProcessor_createCacheFromConfig; + jmethodID m_PlatformProcessor_getOrCreateCacheFromConfig; + jmethodID m_PlatformProcessor_createNearCache; + jmethodID m_PlatformProcessor_getOrCreateNearCache; + jmethodID m_PlatformProcessor_destroyCache; + jmethodID m_PlatformProcessor_affinity; + jmethodID m_PlatformProcessor_dataStreamer; + jmethodID m_PlatformProcessor_transactions; + jmethodID m_PlatformProcessor_projection; + jmethodID m_PlatformProcessor_compute; + jmethodID m_PlatformProcessor_message; + jmethodID m_PlatformProcessor_events; + jmethodID m_PlatformProcessor_services; + jmethodID m_PlatformProcessor_extensions; + jmethodID m_PlatformProcessor_atomicLong; + jmethodID m_PlatformProcessor_getIgniteConfiguration; + jmethodID m_PlatformProcessor_getCacheNames; + jmethodID m_PlatformProcessor_atomicSequence; + jmethodID m_PlatformProcessor_atomicReference; + + jclass c_PlatformTarget; + jmethodID m_PlatformTarget_inStreamOutLong; + jmethodID m_PlatformTarget_inStreamOutObject; + jmethodID m_PlatformTarget_outLong; + jmethodID m_PlatformTarget_outStream; + jmethodID m_PlatformTarget_outObject; + jmethodID m_PlatformTarget_inStreamOutStream; + jmethodID m_PlatformTarget_inObjectStreamOutStream; + jmethodID m_PlatformTarget_listenFuture; + jmethodID m_PlatformTarget_listenFutureForOperation; + jmethodID m_PlatformTarget_listenFutureAndGet; + jmethodID m_PlatformTarget_listenFutureForOperationAndGet; + + jclass c_PlatformTransactions; + jmethodID m_PlatformTransactions_txStart; + jmethodID m_PlatformTransactions_txCommit; + jmethodID m_PlatformTransactions_txCommitAsync; + jmethodID m_PlatformTransactions_txRollback; + jmethodID m_PlatformTransactions_txRollbackAsync; + jmethodID m_PlatformTransactions_txState; + jmethodID m_PlatformTransactions_txSetRollbackOnly; + jmethodID m_PlatformTransactions_txClose; + jmethodID m_PlatformTransactions_resetMetrics; + + jclass c_PlatformUtils; + jmethodID m_PlatformUtils_reallocate; + jmethodID m_PlatformUtils_errData; + + jclass c_PlatformAtomicLong; + jmethodID m_PlatformAtomicLong_get; + jmethodID m_PlatformAtomicLong_incrementAndGet; + jmethodID m_PlatformAtomicLong_getAndIncrement; + jmethodID m_PlatformAtomicLong_addAndGet; + jmethodID m_PlatformAtomicLong_getAndAdd; + jmethodID m_PlatformAtomicLong_decrementAndGet; + jmethodID m_PlatformAtomicLong_getAndDecrement; + jmethodID m_PlatformAtomicLong_getAndSet; + jmethodID m_PlatformAtomicLong_compareAndSetAndGet; + jmethodID m_PlatformAtomicLong_isClosed; + jmethodID m_PlatformAtomicLong_close; + + jclass c_PlatformAtomicSequence; + jmethodID m_PlatformAtomicSequence_get; + jmethodID m_PlatformAtomicSequence_incrementAndGet; + jmethodID m_PlatformAtomicSequence_getAndIncrement; + jmethodID m_PlatformAtomicSequence_addAndGet; + jmethodID m_PlatformAtomicSequence_getAndAdd; + jmethodID m_PlatformAtomicSequence_getBatchSize; + jmethodID m_PlatformAtomicSequence_setBatchSize; + jmethodID m_PlatformAtomicSequence_isClosed; + jmethodID m_PlatformAtomicSequence_close; + + jclass c_PlatformAtomicReference; + jmethodID m_PlatformAtomicReference_isClosed; + jmethodID m_PlatformAtomicReference_close; + + jclass c_PlatformListenable; + jmethodID m_PlatformListenable_cancel; + jmethodID m_PlatformListenable_isCancelled; + + /** + * Constructor. + */ + void Initialize(JNIEnv* env); + + /** + * Destroy members releasing all allocated classes. + */ + void Destroy(JNIEnv* env); + }; + + /** + * JNI JVM wrapper. + */ + class IGNITE_IMPORT_EXPORT JniJvm { + public: + /** + * Default constructor for uninitialized JVM. + */ + JniJvm(); + + /** + * Constructor. + * + * @param jvm JVM. + * @param javaMembers Java members. + * @param members Members. + */ + JniJvm(JavaVM* jvm, JniJavaMembers javaMembers, JniMembers members); + + /** + * Get JVM. + * + * @param JVM. + */ + JavaVM* GetJvm(); + + /** + * Get Java members. + * + * @param Java members. + */ + JniJavaMembers& GetJavaMembers(); + + /** + * Get members. + * + * @param Members. + */ + JniMembers& GetMembers(); + private: + /** JVM. */ + JavaVM* jvm; + + /** Java members. */ + JniJavaMembers javaMembers; + + /** Members. */ + JniMembers members; + }; + + /** + * JNI error information. + */ + struct IGNITE_IMPORT_EXPORT JniErrorInfo + { + int code; + char* errCls; + char* errMsg; + + /** + * Default constructor. Creates empty error info. + */ + JniErrorInfo(); + + /** + * Constructor. + * + * @param code Code. + * @param errCls Error class. + * @param errMsg Error message. + */ + JniErrorInfo(int code, const char* errCls, const char* errMsg); + + /** + * Copy constructor. + * + * @param other Other instance. + */ + JniErrorInfo(const JniErrorInfo& other); + + /** + * Assignment operator overload. + * + * @param other Other instance. + * @return This instance. + */ + JniErrorInfo& operator=(const JniErrorInfo& other); + + /** + * Destructor. + */ + ~JniErrorInfo(); + }; + + /** + * Unmanaged context. + */ + class IGNITE_IMPORT_EXPORT JniContext { + public: + static JniContext* Create(char** opts, int optsLen, JniHandlers hnds); + static JniContext* Create(char** opts, int optsLen, JniHandlers hnds, JniErrorInfo* errInfo); + static int Reallocate(long long memPtr, int cap); + static void Detach(); + static void Release(jobject obj); + + jobject IgnitionStart(char* cfgPath, char* name, int factoryId, long long dataPtr); + jobject IgnitionStart(char* cfgPath, char* name, int factoryId, long long dataPtr, JniErrorInfo* errInfo); + jobject IgnitionInstance(char* name); + jobject IgnitionInstance(char* name, JniErrorInfo* errInfo); + long long IgnitionEnvironmentPointer(char* name); + long long IgnitionEnvironmentPointer(char* name, JniErrorInfo* errInfo); + bool IgnitionStop(char* name, bool cancel); + bool IgnitionStop(char* name, bool cancel, JniErrorInfo* errInfo); + void IgnitionStopAll(bool cancel); + void IgnitionStopAll(bool cancel, JniErrorInfo* errInfo); + + void ProcessorReleaseStart(jobject obj); + jobject ProcessorProjection(jobject obj); + jobject ProcessorCache(jobject obj, const char* name); + jobject ProcessorCache(jobject obj, const char* name, JniErrorInfo* errInfo); + jobject ProcessorCreateCache(jobject obj, const char* name); + jobject ProcessorCreateCache(jobject obj, const char* name, JniErrorInfo* errInfo); + jobject ProcessorGetOrCreateCache(jobject obj, const char* name); + jobject ProcessorGetOrCreateCache(jobject obj, const char* name, JniErrorInfo* errInfo); + jobject ProcessorCreateCacheFromConfig(jobject obj, long long memPtr); + jobject ProcessorCreateCacheFromConfig(jobject obj, long long memPtr, JniErrorInfo* errInfo); + jobject ProcessorGetOrCreateCacheFromConfig(jobject obj, long long memPtr); + jobject ProcessorGetOrCreateCacheFromConfig(jobject obj, long long memPtr, JniErrorInfo* errInfo); + jobject ProcessorCreateNearCache(jobject obj, const char* name, long long memPtr); + jobject ProcessorGetOrCreateNearCache(jobject obj, const char* name, long long memPtr); + void ProcessorDestroyCache(jobject obj, const char* name); + void ProcessorDestroyCache(jobject obj, const char* name, JniErrorInfo* errInfo); + jobject ProcessorAffinity(jobject obj, const char* name); + jobject ProcessorDataStreamer(jobject obj, const char* name, bool keepPortable); + jobject ProcessorTransactions(jobject obj); + jobject ProcessorCompute(jobject obj, jobject prj); + jobject ProcessorMessage(jobject obj, jobject prj); + jobject ProcessorEvents(jobject obj, jobject prj); + jobject ProcessorServices(jobject obj, jobject prj); + jobject ProcessorExtensions(jobject obj); + jobject ProcessorAtomicLong(jobject obj, char* name, long long initVal, bool create); + jobject ProcessorAtomicSequence(jobject obj, char* name, long long initVal, bool create); + jobject ProcessorAtomicReference(jobject obj, char* name, long long memPtr, bool create); + void ProcessorGetIgniteConfiguration(jobject obj, long long memPtr); + void ProcessorGetCacheNames(jobject obj, long long memPtr); + + long long TargetInStreamOutLong(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL); + void TargetInStreamOutStream(jobject obj, int opType, long long inMemPtr, long long outMemPtr, JniErrorInfo* errInfo = NULL); + jobject TargetInStreamOutObject(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL); + void TargetInObjectStreamOutStream(jobject obj, int opType, void* arg, long long inMemPtr, long long outMemPtr, JniErrorInfo* errInfo = NULL); + long long TargetOutLong(jobject obj, int opType, JniErrorInfo* errInfo = NULL); + void TargetOutStream(jobject obj, int opType, long long memPtr, JniErrorInfo* errInfo = NULL); + jobject TargetOutObject(jobject obj, int opType, JniErrorInfo* errInfo = NULL); + void TargetListenFuture(jobject obj, long long futId, int typ); + void TargetListenFutureForOperation(jobject obj, long long futId, int typ, int opId); + void* TargetListenFutureAndGet(jobject obj, long long futId, int typ); + void* TargetListenFutureForOperationAndGet(jobject obj, long long futId, int typ, int opId); + + int AffinityPartitions(jobject obj); + + jobject CacheWithSkipStore(jobject obj); + jobject CacheWithNoRetries(jobject obj); + jobject CacheWithExpiryPolicy(jobject obj, long long create, long long update, long long access); + jobject CacheWithAsync(jobject obj); + jobject CacheWithKeepPortable(jobject obj); + void CacheClear(jobject obj, JniErrorInfo* errInfo = NULL); + void CacheRemoveAll(jobject obj, JniErrorInfo* errInfo = NULL); + jobject CacheOutOpQueryCursor(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL); + jobject CacheOutOpContinuousQuery(jobject obj, int type, long long memPtr); + jobject CacheIterator(jobject obj); + jobject CacheLocalIterator(jobject obj, int peekModes); + void CacheEnterLock(jobject obj, long long id); + void CacheExitLock(jobject obj, long long id); + bool CacheTryEnterLock(jobject obj, long long id, long long timeout); + void CacheCloseLock(jobject obj, long long id); + void CacheRebalance(jobject obj, long long futId); + int CacheSize(jobject obj, int peekModes, bool loc, JniErrorInfo* errInfo = NULL); + + void CacheStoreCallbackInvoke(jobject obj, long long memPtr); + + void ComputeWithNoFailover(jobject obj); + void ComputeWithTimeout(jobject obj, long long timeout); + void* ComputeExecuteNative(jobject obj, long long taskPtr, long long topVer); + + void ContinuousQueryClose(jobject obj); + jobject ContinuousQueryGetInitialQueryCursor(jobject obj); + + void DataStreamerListenTopology(jobject obj, long long ptr); + bool DataStreamerAllowOverwriteGet(jobject obj); + void DataStreamerAllowOverwriteSet(jobject obj, bool val); + bool DataStreamerSkipStoreGet(jobject obj); + void DataStreamerSkipStoreSet(jobject obj, bool val); + int DataStreamerPerNodeBufferSizeGet(jobject obj); + void DataStreamerPerNodeBufferSizeSet(jobject obj, int val); + int DataStreamerPerNodeParallelOperationsGet(jobject obj); + void DataStreamerPerNodeParallelOperationsSet(jobject obj, int val); + + jobject MessagingWithAsync(jobject obj); + + jobject ProjectionForOthers(jobject obj, jobject prj); + jobject ProjectionForRemotes(jobject obj); + jobject ProjectionForDaemons(jobject obj); + jobject ProjectionForRandom(jobject obj); + jobject ProjectionForOldest(jobject obj); + jobject ProjectionForYoungest(jobject obj); + void ProjectionResetMetrics(jobject obj); + jobject ProjectionOutOpRet(jobject obj, int type, long long memPtr); + + void QueryCursorIterator(jobject obj, JniErrorInfo* errInfo = NULL); + bool QueryCursorIteratorHasNext(jobject obj, JniErrorInfo* errInfo = NULL); + void QueryCursorClose(jobject obj, JniErrorInfo* errInfo = NULL); + + long long TransactionsStart(jobject obj, int concurrency, int isolation, long long timeout, int txSize); + int TransactionsCommit(jobject obj, long long id); + void TransactionsCommitAsync(jobject obj, long long id, long long futId); + int TransactionsRollback(jobject obj, long long id); + void TransactionsRollbackAsync(jobject obj, long long id, long long futId); + int TransactionsClose(jobject obj, long long id); + int TransactionsState(jobject obj, long long id); + bool TransactionsSetRollbackOnly(jobject obj, long long id); + void TransactionsResetMetrics(jobject obj); + + jobject EventsWithAsync(jobject obj); + bool EventsStopLocalListen(jobject obj, long long hnd); + void EventsLocalListen(jobject obj, long long hnd, int type); + bool EventsIsEnabled(jobject obj, int type); + + jobject ServicesWithAsync(jobject obj); + jobject ServicesWithServerKeepPortable(jobject obj); + void ServicesCancel(jobject obj, char* name); + void ServicesCancelAll(jobject obj); + void* ServicesGetServiceProxy(jobject obj, char* name, bool sticky); + + long long AtomicLongGet(jobject obj); + long long AtomicLongIncrementAndGet(jobject obj); + long long AtomicLongGetAndIncrement(jobject obj); + long long AtomicLongAddAndGet(jobject obj, long long value); + long long AtomicLongGetAndAdd(jobject obj, long long value); + long long AtomicLongDecrementAndGet(jobject obj); + long long AtomicLongGetAndDecrement(jobject obj); + long long AtomicLongGetAndSet(jobject obj, long long value); + long long AtomicLongCompareAndSetAndGet(jobject obj, long long expVal, long long newVal); + bool AtomicLongIsClosed(jobject obj); + void AtomicLongClose(jobject obj); + + long long AtomicSequenceGet(jobject obj); + long long AtomicSequenceIncrementAndGet(jobject obj); + long long AtomicSequenceGetAndIncrement(jobject obj); + long long AtomicSequenceAddAndGet(jobject obj, long long l); + long long AtomicSequenceGetAndAdd(jobject obj, long long l); + int AtomicSequenceGetBatchSize(jobject obj); + void AtomicSequenceSetBatchSize(jobject obj, int size); + bool AtomicSequenceIsClosed(jobject obj); + void AtomicSequenceClose(jobject obj); + + bool AtomicReferenceIsClosed(jobject obj); + void AtomicReferenceClose(jobject obj); + + bool ListenableCancel(jobject obj); + bool ListenableIsCancelled(jobject obj); + + jobject Acquire(jobject obj); + + void DestroyJvm(); + void ThrowToJava(char* errMsg); + private: + JniJvm* jvm; + JniHandlers hnds; + + JniContext(JniJvm* jvm, JniHandlers hnds); + + JNIEnv* Attach(); + void ExceptionCheck(JNIEnv* env); + void ExceptionCheck(JNIEnv* env, JniErrorInfo* errInfo); + jobject LocalToGlobal(JNIEnv* env, jobject obj); + jobject ProcessorCache0(jobject proc, const char* name, jmethodID mthd, JniErrorInfo* errInfo); + jobject ProcessorCacheFromConfig0(jobject proc, long long memPtr, jmethodID mthd, JniErrorInfo* errInfo); + jobject ProcessorGetOrCreateNearCache0(jobject obj, const char* name, long long memPtr, jmethodID methodID); + }; + + JNIEXPORT jlong JNICALL JniCacheStoreCreate(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr); + JNIEXPORT jint JNICALL JniCacheStoreInvoke(JNIEnv *env, jclass cls, jlong envPtr, jlong objPtr, jlong memPtr, jobject cb); + JNIEXPORT void JNICALL JniCacheStoreDestroy(JNIEnv *env, jclass cls, jlong envPtr, jlong objPtr); + JNIEXPORT jlong JNICALL JniCacheStoreSessionCreate(JNIEnv *env, jclass cls, jlong envPtr, jlong storePtr); + + JNIEXPORT jlong JNICALL JniCacheEntryFilterCreate(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr); + JNIEXPORT jint JNICALL JniCacheEntryFilterApply(JNIEnv *env, jclass cls, jlong envPtr, jlong objPtr, jlong memPtr); + JNIEXPORT void JNICALL JniCacheEntryFilterDestroy(JNIEnv *env, jclass cls, jlong envPtr, jlong objPtr); + + JNIEXPORT void JNICALL JniCacheInvoke(JNIEnv *env, jclass cls, jlong envPtr, jlong inMemPtr, jlong outMemPtr); + + JNIEXPORT void JNICALL JniComputeTaskMap(JNIEnv *env, jclass cls, jlong envPtr, jlong taskPtr, jlong inMemPtr, jlong outMemPtr); + JNIEXPORT jint JNICALL JniComputeTaskJobResult(JNIEnv *env, jclass cls, jlong envPtr, jlong taskPtr, jlong jobPtr, jlong memPtr); + JNIEXPORT void JNICALL JniComputeTaskReduce(JNIEnv *env, jclass cls, jlong envPtr, jlong taskPtr); + JNIEXPORT void JNICALL JniComputeTaskComplete(JNIEnv *env, jclass cls, jlong envPtr, jlong taskPtr, jlong memPtr); + JNIEXPORT jint JNICALL JniComputeJobSerialize(JNIEnv *env, jclass cls, jlong envPtr, jlong jobPtr, jlong memPtr); + JNIEXPORT jlong JNICALL JniComputeJobCreate(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr); + JNIEXPORT void JNICALL JniComputeJobExecute(JNIEnv *env, jclass cls, jlong envPtr, jlong jobPtr, jint cancel, jlong memPtr); + JNIEXPORT void JNICALL JniComputeJobCancel(JNIEnv *env, jclass cls, jlong envPtr, jlong jobPtr); + JNIEXPORT void JNICALL JniComputeJobDestroy(JNIEnv *env, jclass cls, jlong envPtr, jlong jobPtr); + + JNIEXPORT void JNICALL JniContinuousQueryListenerApply(JNIEnv *env, jclass cls, jlong envPtr, jlong cbPtr, jlong memPtr); + JNIEXPORT jlong JNICALL JniContinuousQueryFilterCreate(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr); + JNIEXPORT jint JNICALL JniContinuousQueryFilterApply(JNIEnv *env, jclass cls, jlong envPtr, jlong filterPtr, jlong memPtr); + JNIEXPORT void JNICALL JniContinuousQueryFilterRelease(JNIEnv *env, jclass cls, jlong envPtr, jlong filterPtr); + + JNIEXPORT void JNICALL JniDataStreamerTopologyUpdate(JNIEnv *env, jclass cls, jlong envPtr, jlong ldrPtr, jlong topVer, jint topSize); + JNIEXPORT void JNICALL JniDataStreamerStreamReceiverInvoke(JNIEnv *env, jclass cls, jlong envPtr, jlong ptr, jobject cache, jlong memPtr, jboolean keepPortable); + + JNIEXPORT void JNICALL JniFutureByteResult(JNIEnv *env, jclass cls, jlong envPtr, jlong futPtr, jint res); + JNIEXPORT void JNICALL JniFutureBoolResult(JNIEnv *env, jclass cls, jlong envPtr, jlong futPtr, jint res); + JNIEXPORT void JNICALL JniFutureShortResult(JNIEnv *env, jclass cls, jlong envPtr, jlong futPtr, jint res); + JNIEXPORT void JNICALL JniFutureCharResult(JNIEnv *env, jclass cls, jlong envPtr, jlong futPtr, jint res); + JNIEXPORT void JNICALL JniFutureIntResult(JNIEnv *env, jclass cls, jlong envPtr, jlong futPtr, jint res); + JNIEXPORT void JNICALL JniFutureFloatResult(JNIEnv *env, jclass cls, jlong envPtr, jlong futPtr, jfloat res); + JNIEXPORT void JNICALL JniFutureLongResult(JNIEnv *env, jclass cls, jlong envPtr, jlong futPtr, jlong res); + JNIEXPORT void JNICALL JniFutureDoubleResult(JNIEnv *env, jclass cls, jlong envPtr, jlong futPtr, jdouble res); + JNIEXPORT void JNICALL JniFutureObjectResult(JNIEnv *env, jclass cls, jlong envPtr, jlong futPtr, jlong memPtr); + JNIEXPORT void JNICALL JniFutureNullResult(JNIEnv *env, jclass cls, jlong envPtr, jlong futPtr); + JNIEXPORT void JNICALL JniFutureError(JNIEnv *env, jclass cls, jlong envPtr, jlong futPtr, jlong memPtr); + + JNIEXPORT void JNICALL JniLifecycleEvent(JNIEnv *env, jclass cls, jlong envPtr, jlong ptr, jint evt); + + JNIEXPORT void JNICALL JniMemoryReallocate(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr, jint cap); + + JNIEXPORT jlong JNICALL JniMessagingFilterCreate(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr); + JNIEXPORT jint JNICALL JniMessagingFilterApply(JNIEnv *env, jclass cls, jlong envPtr, jlong ptr, jlong memPtr); + JNIEXPORT void JNICALL JniMessagingFilterDestroy(JNIEnv *env, jclass cls, jlong envPtr, jlong ptr); + + JNIEXPORT jlong JNICALL JniEventFilterCreate(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr); + JNIEXPORT jint JNICALL JniEventFilterApply(JNIEnv *env, jclass cls, jlong envPtr, jlong ptr, jlong memPtr); + JNIEXPORT void JNICALL JniEventFilterDestroy(JNIEnv *env, jclass cls, jlong envPtr, jlong ptr); + + JNIEXPORT jlong JNICALL JniServiceInit(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr); + JNIEXPORT void JNICALL JniServiceExecute(JNIEnv *env, jclass cls, jlong envPtr, jlong svcPtr, jlong memPtr); + JNIEXPORT void JNICALL JniServiceCancel(JNIEnv *env, jclass cls, jlong envPtr, jlong svcPtr, jlong memPtr); + JNIEXPORT void JNICALL JniServiceInvokeMethod(JNIEnv *env, jclass cls, jlong envPtr, jlong svcPtr, jlong inMemPtr, jlong outMemPtr); + JNIEXPORT jint JNICALL JniClusterNodeFilterApply(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr); + + JNIEXPORT jlong JNICALL JniNodeInfo(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr); + + JNIEXPORT void JNICALL JniOnStart(JNIEnv *env, jclass cls, jlong envPtr, jobject proc, jlong memPtr); + JNIEXPORT void JNICALL JniOnStop(JNIEnv *env, jclass cls, jlong envPtr); + + JNIEXPORT jlong JNICALL JniExtensionCallbackInLongOutLong(JNIEnv *env, jclass cls, jlong envPtr, jint typ, jlong arg1); + JNIEXPORT jlong JNICALL JniExtensionCallbackInLongLongOutLong(JNIEnv *env, jclass cls, jlong envPtr, jint typ, jlong arg1, jlong arg2); + + JNIEXPORT void JNICALL JniOnClientDisconnected(JNIEnv *env, jclass cls, jlong envPtr); + JNIEXPORT void JNICALL JniOnClientReconnected(JNIEnv *env, jclass cls, jlong envPtr, jboolean clusterRestarted); + } + } +} + +#endif //_IGNITE_JNI_JAVA \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/764c97b9/modules/platforms/cpp/jni/include/ignite/jni/utils.h ---------------------------------------------------------------------- diff --git a/modules/platforms/cpp/jni/include/ignite/jni/utils.h b/modules/platforms/cpp/jni/include/ignite/jni/utils.h new file mode 100644 index 0000000..ee40248 --- /dev/null +++ b/modules/platforms/cpp/jni/include/ignite/jni/utils.h @@ -0,0 +1,101 @@ +/* + * 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. + */ +#ifndef _IGNITE_JNI_UTILS +#define _IGNITE_JNI_UTILS + +#include + +#include + +namespace ignite +{ + namespace jni + { + /** + * Helper class to manage attached threads. + */ + class AttachHelper + { + public: + /** + * Destructor. + */ + ~AttachHelper(); + + /** + * Callback invoked on successful thread attach ot JVM. + */ + static void OnThreadAttach(); + }; + + /** + * Attempts to find JVM library to load it into the process later. + * First search is performed using the passed path argument (is not NULL). + * Then JRE_HOME is evaluated. Last, JAVA_HOME is evaluated. + * + * @param Explicitly defined path (optional). + * @param found Whether library was found. + * @return Path to the file. + */ + IGNITE_IMPORT_EXPORT std::string FindJvmLibrary(const std::string* path, bool* found); + + /** + * Load JVM library into the process. + * + * @param path Optional path to the library. + * @return Whether load was successful. + */ + IGNITE_IMPORT_EXPORT bool LoadJvmLibrary(const std::string& path); + + /** + * Create Ignite classpath based on user input and home directory. + * + * @param usrCp User's classpath. + * @param home Ignite home directory. + * @return Classpath. + */ + IGNITE_IMPORT_EXPORT std::string CreateIgniteClasspath(const std::string* usrCp, const std::string* home); + + /** + * Create Ignite classpath based on user input and home directory. + * + * @param usrCp User's classpath. + * @param home Ignite home directory. + * @param test Whether test classpath must be used. + * @return Classpath. + */ + IGNITE_IMPORT_EXPORT std::string CreateIgniteClasspath(const std::string* usrCp, const std::string* home, bool test); + + /** + * Resolve IGNITE_HOME directory. Resolution is performed in several + * steps: + * 1) Check for path provided as argument. + * 2) Check for environment variable. + * 3) Check for current working directory. + * Result of these 3 checks are evaluated based on existence of certain + * predefined folders inside possible GG home. If they are found, + * IGNITE_HOME is considered resolved. + * + * @param path Optional path to evaluate. + * @param found Whether IGNITE_HOME home was found. + * @return Resolved GG home. + */ + IGNITE_IMPORT_EXPORT std::string ResolveIgniteHome(const std::string* path, bool* found); + } +} + +#endif //_IGNITE_JNI_UTILS \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/764c97b9/modules/platforms/cpp/jni/os/linux/src/utils.cpp ---------------------------------------------------------------------- diff --git a/modules/platforms/cpp/jni/os/linux/src/utils.cpp b/modules/platforms/cpp/jni/os/linux/src/utils.cpp new file mode 100644 index 0000000..a0f74ef --- /dev/null +++ b/modules/platforms/cpp/jni/os/linux/src/utils.cpp @@ -0,0 +1,417 @@ +/* + * 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. + */ +#include + +#include + +#include +#include +#include + +#include "ignite/common/utils.h" +#include "ignite/jni/utils.h" +#include "ignite/jni/java.h" + +using namespace ignite::common; +using namespace ignite::jni::java; + +namespace ignite +{ + namespace jni + { + const char* JAVA_HOME = "JAVA_HOME"; + const char* JAVA_DLL = "/jre/lib/amd64/server/libjvm.so"; + + const char* IGNITE_HOME = "IGNITE_HOME"; + + const char* PROBE_BIN = "/bin"; + const char* PROBE_EXAMPLES = "/examples"; + + const char* IGNITE_NATIVE_TEST_CLASSPATH = "IGNITE_NATIVE_TEST_CLASSPATH"; + + /** Key indicating that the thread is attached. */ + static pthread_key_t attachKey; + + /** Helper to ensure that attach key is allocated only once. */ + static pthread_once_t attachKeyInit = PTHREAD_ONCE_INIT; + + void DestroyAttachKey(void* key) + { + delete reinterpret_cast(key); + } + + void AllocateAttachKey() + { + pthread_key_create(&attachKey, DestroyAttachKey); + } + + AttachHelper::~AttachHelper() + { + JniContext::Detach(); + } + + void AttachHelper::OnThreadAttach() + { + pthread_once(&attachKeyInit, AllocateAttachKey); + + void* val = pthread_getspecific(attachKey); + + if (!val) + pthread_setspecific(attachKey, new AttachHelper()); + } + + /** + * Helper method to set boolean result to reference with proper NULL-check. + * + * @param res Result. + * @param outRes Where to set the result. + */ + inline void SetBoolResult(bool res, bool* outRes) + { + if (outRes) + *outRes = res; + } + + /** + * Helper function for GG home resolution. Checks whether certain folders + * exist in the path. Optionally goes upwards in directory hierarchy. + * + * @param path Path to evaluate. + * @param up Whether to go upwards. + * @res Resolution result. + * @return Resolved directory. + */ + std::string ResolveIgniteHome0(const std::string& path, bool up, bool* res) + { + struct stat pathStat; + + if (stat(path.c_str(), &pathStat) != -1 && S_ISDIR(pathStat.st_mode)) + { + // Remove trailing slashes, otherwise we will have an infinite loop. + std::string path0 = path; + + while (true) { + char lastChar = *path0.rbegin(); + + if (lastChar == '/' || lastChar == ' ') { + size_t off = path0.find_last_of(lastChar); + + path0.erase(off, 1); + } + else + break; + } + + std::string binStr = path0 + PROBE_BIN; + struct stat binStat; + + std::string examplesStr = path0 + PROBE_EXAMPLES; + struct stat examplesStat; + + if (stat(binStr.c_str(), &binStat) != -1 && S_ISDIR(binStat.st_mode) && + stat(examplesStr.c_str(), &examplesStat) != -1 && S_ISDIR(examplesStat.st_mode)) + { + SetBoolResult(true, res); + + return std::string(path0); + } + + if (up) + { + // Evaluate parent directory. + size_t slashPos = path0.find_last_of("/"); + + if (slashPos != std::string::npos) + { + std::string parent = path0.substr(0, slashPos); + + return ResolveIgniteHome0(parent, true, res); + } + } + + } + + SetBoolResult(false, res); + + return std::string(); + } + + /** + * Create classpath picking JARs from the given path. + * + * @path Path. + * @return Classpath; + */ + std::string ClasspathJars(const std::string& path) + { + std::string res = std::string(); + + DIR* dir = opendir(path.c_str()); + + if (dir) + { + struct dirent* entry; + + while ((entry = readdir(dir)) != NULL) + { + if (strstr(entry->d_name, ".jar")) + { + res.append(path); + res.append("/"); + res.append(entry->d_name); + res.append(":"); + } + } + + closedir(dir); + } + + return res; + } + + /** + * Create classpath picking compiled classes from the given path. + * + * @path Path. + * @return Classpath; + */ + std::string ClasspathExploded(const std::string& path, bool down) + { + std::string res; + + if (FileExists(path)) + { + // 1. Append "target\classes". + std::string classesPath = path + "/target/classes"; + + if (FileExists(classesPath)) { + res += classesPath; + res += ":"; + } + + // 2. Append "target\test-classes" + std::string testClassesPath = path + "/target/test-classes"; + + if (FileExists(testClassesPath)) { + res += testClassesPath; + res += ":"; + } + + // 3. Append "target\libs" + std::string libsPath = path + "/target/libs"; + + if (FileExists(libsPath)) { + std::string libsCp = ClasspathJars(libsPath); + res += libsCp; + } + + // 4. Do the same for child if needed. + if (down) + { + DIR* dir = opendir(path.c_str()); + + if (dir) + { + struct dirent* entry; + + while ((entry = readdir(dir)) != NULL) + { + std::string entryPath = entry->d_name; + + if (entryPath.compare(".") != 0 && entryPath.compare("..") != 0) + { + std::string entryFullPath = path + "/" + entryPath; + + struct stat entryFullStat; + + if (stat(entryFullPath.c_str(), &entryFullStat) != -1 && S_ISDIR(entryFullStat.st_mode)) + { + std::string childCp = ClasspathExploded(entryFullPath, false); + + res += childCp; + } + } + } + + closedir(dir); + } + } + } + + return res; + } + + /** + * Helper function to create classpath based on Ignite home directory. + * + * @param home Home directory; expected to be valid. + * @param forceTest Force test classpath. + */ + std::string CreateIgniteHomeClasspath(const std::string& home, bool forceTest) + { + std::string res = std::string(); + + // 1. Add exploded test directories. + if (forceTest) + { + std::string examplesPath = home + "/examples"; + std::string examplesCp = ClasspathExploded(examplesPath, true); + res.append(examplesCp); + + std::string modulesPath = home + "/modules"; + std::string modulesCp = ClasspathExploded(modulesPath, true); + res.append(modulesCp); + } + + // 2. Add regular jars from "libs" folder excluding "optional". + std::string libsPath = home + "/libs"; + + if (FileExists(libsPath)) + { + res.append(ClasspathJars(libsPath)); + + // Append inner directories. + DIR* dir = opendir(libsPath.c_str()); + + if (dir) + { + struct dirent* entry; + + while ((entry = readdir(dir)) != NULL) + { + std::string entryPath = entry->d_name; + + if (entryPath.compare(".") != 0 && entryPath.compare("..") != 0 && + entryPath.compare("optional") != 0) + { + std::string entryFullPath = libsPath; + + entryFullPath.append("/"); + entryFullPath.append(entryPath); + + struct stat entryFullStat; + + if (stat(entryFullPath.c_str(), &entryFullStat) != -1 && + S_ISDIR(entryFullStat.st_mode)) + res.append(ClasspathJars(entryFullPath)); + } + } + + closedir(dir); + } + } + + // 3. Return. + return res; + } + + std::string FindJvmLibrary(const std::string* path, bool* found) + { + SetBoolResult(true, found); // Optimistically assume that we will find it. + + if (path) { + // If path is provided explicitly, then check only it. + if (FileExists(*path)) + return std::string(path->data()); + } + else + { + bool javaEnvFound; + std::string javaEnv = GetEnv(JAVA_HOME, javaEnvFound); + + if (javaEnvFound) + { + std::string javaDll = javaEnv + JAVA_DLL; + + if (FileExists(javaDll)) + return std::string(javaDll); + } + } + + SetBoolResult(false, found); + + return std::string(); + } + + bool LoadJvmLibrary(const std::string& path) + { + void* hnd = dlopen(path.c_str(), RTLD_LAZY); + + return hnd != NULL; + } + + std::string ResolveIgniteHome(const std::string* path, bool* found) + { + if (path) + // 1. Check passed argument. + return ResolveIgniteHome0(*path, false, found); + else + { + // 2. Check environment variable. + bool envFound; + std::string env = GetEnv(IGNITE_HOME, envFound); + + if (envFound) + return ResolveIgniteHome0(env, false, found); + } + + SetBoolResult(false, found); + + return std::string(); + } + + std::string CreateIgniteClasspath(const std::string* usrCp, const std::string* home) + { + bool forceTest = false; + + if (home) + { + bool envFound; + std::string env = GetEnv(IGNITE_NATIVE_TEST_CLASSPATH, envFound); + + forceTest = envFound && env.compare("true") == 0; + } + + return CreateIgniteClasspath(usrCp, home, forceTest); + } + + std::string CreateIgniteClasspath(const std::string* usrCp, const std::string* home, bool forceTest) + { + // 1. Append user classpath if it exists. + std::string cp = std::string(); + + if (usrCp) + { + cp.append(*usrCp); + + if (*cp.rbegin() != ':') + cp.append(":"); + } + + // 2. Append home classpath if home is defined. + if (home) + { + std::string homeCp = CreateIgniteHomeClasspath(*home, forceTest); + + cp.append(homeCp); + } + + // 3. Return. + return cp; + } + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/764c97b9/modules/platforms/cpp/jni/os/win/src/utils.cpp ---------------------------------------------------------------------- diff --git a/modules/platforms/cpp/jni/os/win/src/utils.cpp b/modules/platforms/cpp/jni/os/win/src/utils.cpp new file mode 100644 index 0000000..0647aa3 --- /dev/null +++ b/modules/platforms/cpp/jni/os/win/src/utils.cpp @@ -0,0 +1,428 @@ +/* + * 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. + */ + +#include + +#include "ignite/common/concurrent.h" +#include "ignite/common/utils.h" + +#include "ignite/jni/utils.h" +#include "ignite/jni/java.h" + +using namespace ignite::common; +using namespace ignite::common::concurrent; + +using namespace ignite::jni::java; + +namespace ignite +{ + namespace jni + { + AttachHelper::~AttachHelper() + { + // No-op. + } + + void AttachHelper::OnThreadAttach() + { + // No-op. + } + + const char* JAVA_HOME = "JAVA_HOME"; + const char* JAVA_DLL = "\\jre\\bin\\server\\jvm.dll"; + + const char* IGNITE_HOME = "IGNITE_HOME"; + + const char* PROBE_BIN = "\\bin"; + const char* PROBE_EXAMPLES = "\\examples"; + + const char* IGNITE_NATIVE_TEST_CLASSPATH = "IGNITE_NATIVE_TEST_CLASSPATH"; + + /** + * Helper method to set boolean result to pointer with proper NULL-check. + * + * @param res Result. + * @param outRes Where to set the result. + */ + inline void SetBoolResult(bool res, bool* outRes) + { + if (outRes) + *outRes = res; + } + + /** + * Helper function for GG home resolution. Checks whether certain folders + * exist in the path. Optionally goes upwards in directory hierarchy. + * + * @param path Path to evaluate. + * @param up Whether to go upwards. + * @res Resolution result. + * @return Resolved directory. + */ + std::string ResolveIgniteHome0(const std::string& path, bool up, bool* res) + { + DWORD attrs = GetFileAttributesA(path.c_str()); + + if (attrs != INVALID_FILE_ATTRIBUTES && (attrs & FILE_ATTRIBUTE_DIRECTORY)) + { + // Remove trailing slashes, otherwise we will have an infinite loop. + std::string path0 = path; + + while (true) { + char lastChar = *path0.rbegin(); + + if (lastChar == '/' || lastChar == '\\' || lastChar == ' ') { + size_t off = path0.find_last_of(lastChar); + + path0.erase(off, 1); + } + else + break; + } + + std::string binStr = path0 + PROBE_BIN; + DWORD binAttrs = GetFileAttributesA(binStr.c_str()); + + std::string examplesStr = path0 + PROBE_EXAMPLES; + DWORD examplesAttrs = GetFileAttributesA(examplesStr.c_str()); + + if (binAttrs != INVALID_FILE_ATTRIBUTES && (binAttrs & FILE_ATTRIBUTE_DIRECTORY) && + examplesAttrs != INVALID_FILE_ATTRIBUTES && (examplesAttrs & FILE_ATTRIBUTE_DIRECTORY)) + { + SetBoolResult(true, res); + return std::string(path0); + } + + if (up) + { + // Evaluate parent directory. + size_t slashPos = path0.find_last_of("/\\"); + + if (slashPos != std::string::npos) + { + std::string parent = path0.substr(0, slashPos); + + return ResolveIgniteHome0(parent, true, res); + } + } + } + + SetBoolResult(false, res); + + return std::string(); + } + + /** + * Create classpath picking JARs from the given path. + * + * @path Path. + * @return Classpath; + */ + std::string ClasspathJars(const std::string& path) + { + std::string searchPath = path + "\\*.jar"; + + std::string res = std::string(); + + WIN32_FIND_DATAA findData; + + HANDLE hnd = FindFirstFileA(searchPath.c_str(), &findData); + + if (hnd != INVALID_HANDLE_VALUE) + { + do + { + res.append(path); + res.append("\\"); + res.append(findData.cFileName); + res.append(";"); + } while (FindNextFileA(hnd, &findData) != 0); + + FindClose(hnd); + } + + return res; + } + + /** + * Create classpath picking compiled classes from the given path. + * + * @path Path. + * @return Classpath; + */ + std::string ClasspathExploded(const std::string& path, bool down) + { + std::string res; + + if (FileExists(path)) + { + // 1. Append "target\classes". + std::string classesPath = path + "\\target\\classes"; + + if (FileExists(classesPath)) { + res.append(classesPath); + res.append(";"); + } + + // 2. Append "target\test-classes" + std::string testClassesPath = path + "\\target\\test-classes"; + + if (FileExists(testClassesPath)) { + res.append(testClassesPath); + res.append(";"); + } + + // 3. Append "target\libs" + std::string libsPath = path + "\\target\\libs"; + + if (FileExists(libsPath)) { + std::string libsCp = ClasspathJars(libsPath); + res.append(libsCp); + } + + // 4. Do the same for child if needed. + if (down) + { + std::string searchPath = path + "\\*"; + + WIN32_FIND_DATAA findData; + + HANDLE hnd = FindFirstFileA(searchPath.c_str(), &findData); + + if (hnd != INVALID_HANDLE_VALUE) + { + do + { + if (findData.dwFileAttributes & FILE_ATTRIBUTE_DIRECTORY) + { + std::string childPath = findData.cFileName; + + if (childPath.compare(".") != 0 && + childPath.compare("..") != 0) + { + std::string childCp = + ClasspathExploded(path + "\\" + childPath, false); + + res.append(childCp); + } + } + } while (FindNextFileA(hnd, &findData) != 0); + + FindClose(hnd); + } + } + } + + return res; + } + + /** + * Helper function to create classpath based on Ignite home directory. + * + * @param home Home directory; expected to be valid. + * @param forceTest Force test classpath. + */ + std::string CreateIgniteHomeClasspath(const std::string& home, bool forceTest) + { + std::string res = std::string(); + + // 1. Add exploded test directories. + if (forceTest) + { + std::string examplesPath = home + "\\examples"; + std::string examplesCp = ClasspathExploded(examplesPath, true); + res.append(examplesCp); + + std::string modulesPath = home + "\\modules"; + std::string modulesCp = ClasspathExploded(modulesPath, true); + res.append(modulesCp); + } + + // 2. Add regular jars from "libs" folder excluding "optional". + std::string libsPath = home + "\\libs"; + + if (FileExists(libsPath)) + { + res.append(ClasspathJars(libsPath)); + + // Append inner directories. + std::string libsSearchPath = libsPath + "\\*"; + + WIN32_FIND_DATAA libsFindData; + + HANDLE libsHnd = FindFirstFileA(libsSearchPath.c_str(), &libsFindData); + + if (libsHnd != INVALID_HANDLE_VALUE) + { + do + { + if (libsFindData.dwFileAttributes & FILE_ATTRIBUTE_DIRECTORY) + { + std::string libsChildPath = libsFindData.cFileName; + + if (libsChildPath.compare(".") != 0 && + libsChildPath.compare("..") != 0 && + libsChildPath.compare("optional") != 0) { + std::string libsFolder = libsPath + "\\" + libsChildPath; + + res.append(ClasspathJars(libsFolder)); + } + } + } while (FindNextFileA(libsHnd, &libsFindData) != 0); + + FindClose(libsHnd); + } + } + + // 3. Return. + return res; + } + + std::string FindJvmLibrary(const std::string* path, bool* found) + { + SetBoolResult(true, found); // Optimistically assume that we will find it. + + if (path) { + // If path is provided explicitly, then check only it. + if (FileExists(*path)) + return std::string(path->data()); + } + else + { + bool javaEnvFound; + std::string javaEnv = GetEnv(JAVA_HOME, javaEnvFound); + + if (javaEnvFound) + { + std::string javaDll = javaEnv + JAVA_DLL; + + if (FileExists(javaDll)) + return std::string(javaDll); + } + } + + *found = false; + + return std::string(); + } + + bool LoadJvmLibrary(const std::string& path) + { + HMODULE mod = LoadLibraryA(path.c_str()); + + return mod != NULL; + } + + std::string CreateIgniteClasspath(const std::string* usrCp, const std::string* home) + { + bool forceTest = false; + + if (home) + { + bool envFound; + std::string env = GetEnv(IGNITE_NATIVE_TEST_CLASSPATH, envFound); + + forceTest = envFound && env.compare("true") == 0; + } + + return CreateIgniteClasspath(usrCp, home, forceTest); + } + + std::string CreateIgniteClasspath(const std::string* usrCp, const std::string* home, bool forceTest) + { + // 1. Append user classpath if it exists. + std::string cp = std::string(); + + if (usrCp) + { + cp.append(*usrCp); + + if (*cp.rbegin() != ';') + cp.append(";"); + } + + // 2. Append home classpath if home is defined. + if (home) + { + std::string homeCp = CreateIgniteHomeClasspath(*home, forceTest); + + cp.append(homeCp); + } + + // 3. Return. + return cp; + } + + std::string ResolveIgniteHome(const std::string* path, bool* found) + { + if (path) + // 1. Check passed argument. + return ResolveIgniteHome0(*path, false, found); + else + { + // 2. Check environment variable. + bool envFound; + std::string env = GetEnv(IGNITE_HOME, envFound); + + if (envFound) + return ResolveIgniteHome0(env, false, found); + + // 3. Check current work dir. + const DWORD curDirLen = GetCurrentDirectory(0, NULL); + + char* curDir = new char[curDirLen]; + + GetCurrentDirectoryA(curDirLen, curDir); + + std::string curDirStr = curDir; + + delete[] curDir; + + return ResolveIgniteHome0(curDirStr, true, found); + } + } + } +} + +BOOL WINAPI DllMain(_In_ HINSTANCE hinstDLL, _In_ DWORD fdwReason, _In_ LPVOID lpvReserved) +{ + switch (fdwReason) + { + case DLL_PROCESS_ATTACH: + if (!ThreadLocal::OnProcessAttach()) + return FALSE; + + break; + + case DLL_THREAD_DETACH: + ThreadLocal::OnThreadDetach(); + + JniContext::Detach(); + + break; + + case DLL_PROCESS_DETACH: + ThreadLocal::OnProcessDetach(); + + break; + + default: + break; + } + + return TRUE; +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/764c97b9/modules/platforms/cpp/jni/project/README.TXT ---------------------------------------------------------------------- diff --git a/modules/platforms/cpp/jni/project/README.TXT b/modules/platforms/cpp/jni/project/README.TXT new file mode 100644 index 0000000..97f4c64 --- /dev/null +++ b/modules/platforms/cpp/jni/project/README.TXT @@ -0,0 +1 @@ +Contains IDE projects artifacts. http://git-wip-us.apache.org/repos/asf/ignite/blob/764c97b9/modules/platforms/cpp/jni/project/vs/README.TXT ---------------------------------------------------------------------- diff --git a/modules/platforms/cpp/jni/project/vs/README.TXT b/modules/platforms/cpp/jni/project/vs/README.TXT new file mode 100644 index 0000000..f4fb456 --- /dev/null +++ b/modules/platforms/cpp/jni/project/vs/README.TXT @@ -0,0 +1 @@ +Contains Visual Studio project artifacts. \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/764c97b9/modules/platforms/cpp/jni/project/vs/jni.vcxproj ---------------------------------------------------------------------- diff --git a/modules/platforms/cpp/jni/project/vs/jni.vcxproj b/modules/platforms/cpp/jni/project/vs/jni.vcxproj new file mode 100644 index 0000000..ac17ad3 --- /dev/null +++ b/modules/platforms/cpp/jni/project/vs/jni.vcxproj @@ -0,0 +1,205 @@ + + + + + Debug + Win32 + + + Debug + x64 + + + Release + Win32 + + + Release + x64 + + + + {4F7E4917-4612-4B96-9838-025711ADE391} + Win32Proj + jni + + + + DynamicLibrary + true + v100 + Unicode + + + DynamicLibrary + true + v100 + Unicode + + + DynamicLibrary + false + v100 + true + Unicode + + + DynamicLibrary + false + v100 + true + Unicode + + + + + + + + + + + + + + + + + + + ignite.jni + $(SolutionDir)$(Platform)\$(Configuration)\ + $(Platform)\$(Configuration)\ + false + + + ignite.jni + $(SolutionDir)$(Platform)\$(Configuration)\ + $(Platform)\$(Configuration)\ + false + + + ignite.jni + $(SolutionDir)$(Platform)\$(Configuration)\ + $(Platform)\$(Configuration)\ + false + + + ignite.jni + $(SolutionDir)$(Platform)\$(Configuration)\ + $(Platform)\$(Configuration)\ + false + + + + NotUsing + Level3 + Disabled + WIN32;_DEBUG;_WINDOWS;_USRDLL;IGNITEJVM_EXPORTS;_CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;%(PreprocessorDefinitions) + $(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;%(AdditionalIncludeDirectories) + + + Windows + true + $(JAVA_HOME)\lib\jvm.lib;%(AdditionalDependencies) + $(JAVA_HOME)\lib\jvm.lib;%(AdditionalLibraryDirectories) + jvm.dll + module.def + false + false + Default + + + + + Level3 + false + $(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;%(AdditionalIncludeDirectories) + WIN32;_DEBUG;_WINDOWS;_USRDLL;IGNITEJVM_EXPORTS;_CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;%(PreprocessorDefinitions) + ProgramDatabase + + + true + $(JAVA_HOME)\lib\jvm.lib;%(AdditionalLibraryDirectories) + $(JAVA_HOME)\lib\jvm.lib;%(AdditionalDependencies) + true + true + jvm.dll + Default + module.def + + + + + Level3 + NotUsing + MaxSpeed + true + true + WIN32;NDEBUG;_WINDOWS;_USRDLL;IGNITEJVM_EXPORTS;_CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;%(PreprocessorDefinitions) + $(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;%(AdditionalIncludeDirectories) + AnySuitable + Speed + true + false + true + + + Windows + true + true + true + $(JAVA_HOME)\lib\jvm.lib;%(AdditionalDependencies) + $(JAVA_HOME)\lib\jvm.lib;%(AdditionalLibraryDirectories) + jvm.dll + module.def + + + + + Level3 + Full + true + true + false + $(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;%(AdditionalIncludeDirectories) + AnySuitable + Speed + true + true + false + _CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;%(PreprocessorDefinitions) + + + true + true + true + $(JAVA_HOME)\lib\jvm.lib;%(AdditionalLibraryDirectories) + $(JAVA_HOME)\lib\jvm.lib;%(AdditionalDependencies) + jvm.dll + module.def + + + + + + + + + + + + + + + + + + + + + + {b63f2e01-5157-4719-8491-0e1c7cd3b701} + + + \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/764c97b9/modules/platforms/cpp/jni/project/vs/jni.vcxproj.filters ---------------------------------------------------------------------- diff --git a/modules/platforms/cpp/jni/project/vs/jni.vcxproj.filters b/modules/platforms/cpp/jni/project/vs/jni.vcxproj.filters new file mode 100644 index 0000000..7bc8dfd --- /dev/null +++ b/modules/platforms/cpp/jni/project/vs/jni.vcxproj.filters @@ -0,0 +1,42 @@ + + + + + {1dbec2be-5cb4-4f70-aef6-b4627d39b99b} + + + {93995380-89BD-4b04-88EB-625FBE52EBFB} + h;hh;hpp;hxx;hm;inl;inc;xsd + + + + + Code + + + Code + + + Misc + + + Code + + + + + Code + + + Code + + + Code + + + + + Misc + + + \ No newline at end of file