#include <IOKit/IOCatalogue.h>
#include <IOKit/IOMemoryDescriptor.h>
#include <IOKit/IOBufferMemoryDescriptor.h>
+#include <IOKit/IOMapper.h>
#include <IOKit/IOLib.h>
#include <IOKit/IOBSD.h>
#include <IOKit/system.h>
#include <DriverKit/IOBufferMemoryDescriptor.h>
#include <DriverKit/IOMemoryMap.h>
#include <DriverKit/IODataQueueDispatchSource.h>
+#include <DriverKit/IOServiceNotificationDispatchSource.h>
#include <DriverKit/IOUserServer.h>
/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * */
virtual bool
start(IOService * provider) APPLE_KEXT_OVERRIDE;
- virtual IOReturn
- setProperties(OSObject * props) APPLE_KEXT_OVERRIDE;
};
OSDefineMetaClassAndStructors(IOUserService, IOService)
return ok;
}
-IOReturn
-IOUserService::setProperties(OSObject * properties)
-{
- setProperty("USER", properties);
- return kIOReturnSuccess;
-}
-
/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * */
#undef super
kern_return_t
IMPL(IOService, SetProperties)
{
- IOReturn ret = kIOReturnUnsupported;
+ IOUserServer * us;
+ OSDictionary * dict;
+ IOReturn ret;
ret = setProperties(properties);
+ if (kIOReturnUnsupported == ret) {
+ dict = OSDynamicCast(OSDictionary, properties);
+ us = (typeof(us))thread_iokit_tls_get(0);
+ if (dict && reserved->uvars && (reserved->uvars->userServer == us)) {
+ ret = runPropertyActionBlock(^IOReturn (void) {
+ OSDictionary * userProps;
+ IOReturn ret;
+
+ userProps = OSDynamicCast(OSDictionary, getProperty(gIOUserServicePropertiesKey));
+ if (userProps) {
+ userProps = (typeof(userProps))userProps->copyCollection();
+ } else {
+ userProps = OSDictionary::withCapacity(4);
+ }
+ if (!userProps) {
+ ret = kIOReturnNoMemory;
+ } else {
+ bool ok = userProps->merge(dict);
+ if (ok) {
+ ok = setProperty(gIOUserServicePropertiesKey, userProps);
+ }
+ OSSafeReleaseNULL(userProps);
+ ret = ok ? kIOReturnSuccess : kIOReturnNotWritable;
+ }
+ return ret;
+ });
+ }
+ }
+
return ret;
}
return kIOReturnSuccess;
}
+
+/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * */
+
+kern_return_t
+IMPL(IODMACommand, Create)
+{
+ IOReturn ret;
+ IODMACommand * dma;
+ IODMACommand::SegmentOptions segmentOptions;
+ IOMapper * mapper;
+
+ if (options & ~((uint64_t) kIODMACommandCreateNoOptions)) {
+ // no other options currently defined
+ return kIOReturnBadArgument;
+ }
+
+ if (os_convert_overflow(specification->maxAddressBits, &segmentOptions.fNumAddressBits)) {
+ return kIOReturnBadArgument;
+ }
+ segmentOptions.fMaxSegmentSize = 0;
+ segmentOptions.fMaxTransferSize = 0;
+ segmentOptions.fAlignment = 1;
+ segmentOptions.fAlignmentLength = 1;
+ segmentOptions.fAlignmentInternalSegments = 1;
+ segmentOptions.fStructSize = sizeof(segmentOptions);
+
+ mapper = IOMapper::copyMapperForDevice(device);
+
+ dma = IODMACommand::withSpecification(
+ kIODMACommandOutputHost64,
+ &segmentOptions,
+ kIODMAMapOptionMapped,
+ mapper,
+ NULL);
+
+ OSSafeReleaseNULL(mapper);
+ *command = dma;
+
+ if (!dma) {
+ return kIOReturnNoMemory;
+ }
+ ret = kIOReturnSuccess;
+
+ return ret;
+}
+
+kern_return_t
+IMPL(IODMACommand, PrepareForDMA)
+{
+ IOReturn ret;
+ uint64_t lflags, mdFlags;
+ UInt32 numSegments;
+ UInt64 genOffset;
+
+ if (options & ~((uint64_t) kIODMACommandPrepareForDMANoOptions)) {
+ // no other options currently defined
+ return kIOReturnBadArgument;
+ }
+
+ ret = setMemoryDescriptor(memory, false);
+ if (kIOReturnSuccess != ret) {
+ return ret;
+ }
+
+ ret = prepare(offset, length);
+ if (kIOReturnSuccess != ret) {
+ clearMemoryDescriptor(false);
+ return ret;
+ }
+
+ static_assert(sizeof(IODMACommand::Segment64) == sizeof(IOAddressSegment));
+
+ numSegments = *segmentsCount;
+ genOffset = offset;
+ ret = genIOVMSegments(&genOffset, segments, &numSegments);
+
+ if (kIOReturnSuccess == ret) {
+ IOMemoryDescriptor * mem;
+ mem = __IODEQUALIFY(IOMemoryDescriptor *, fMemory);
+ mdFlags = mem->getFlags();
+ lflags = 0;
+ if (kIODirectionOut & mdFlags) {
+ lflags |= kIOMemoryDirectionOut;
+ }
+ if (kIODirectionIn & mdFlags) {
+ lflags |= kIOMemoryDirectionIn;
+ }
+ *flags = lflags;
+ *segmentsCount = numSegments;
+ }
+
+ return ret;
+}
+
+kern_return_t
+IMPL(IODMACommand, CompleteDMA)
+{
+ IOReturn ret;
+
+ if (options & ~((uint64_t) kIODMACommandCompleteDMANoOptions)) {
+ // no other options currently defined
+ return kIOReturnBadArgument;
+ }
+
+ ret = clearMemoryDescriptor(true);
+
+ return ret;
+}
+
+kern_return_t
+IMPL(IODMACommand, GetPreparation)
+{
+ IOReturn ret;
+ IOMemoryDescriptor * md;
+
+ if (!fActive) {
+ return kIOReturnNotReady;
+ }
+
+ ret = getPreparedOffsetAndLength(offset, length);
+ if (kIOReturnSuccess != ret) {
+ return ret;
+ }
+
+ if (memory) {
+ md = __DECONST(IOMemoryDescriptor *, fMemory);
+ *memory = md;
+ if (!md) {
+ ret = kIOReturnNotReady;
+ } else {
+ md->retain();
+ }
+ }
+ return ret;
+}
+
+kern_return_t
+IMPL(IODMACommand, PerformOperation)
+{
+ IOReturn ret;
+ void * buffer;
+ UInt64 copiedDMA;
+ IOByteCount mdOffset, mdLength, copied;
+
+ if (options & ~((uint64_t)
+ (kIODMACommandPerformOperationOptionRead
+ | kIODMACommandPerformOperationOptionWrite
+ | kIODMACommandPerformOperationOptionZero))) {
+ // no other options currently defined
+ return kIOReturnBadArgument;
+ }
+
+ if (!fActive) {
+ return kIOReturnNotReady;
+ }
+ if (os_convert_overflow(dataOffset, &mdOffset)) {
+ return kIOReturnBadArgument;
+ }
+ if (os_convert_overflow(length, &mdLength)) {
+ return kIOReturnBadArgument;
+ }
+ if (length > fMemory->getLength()) {
+ return kIOReturnBadArgument;
+ }
+ buffer = IONew(uint8_t, length);
+ if (NULL == buffer) {
+ return kIOReturnNoMemory;
+ }
+
+ switch (options) {
+ case kIODMACommandPerformOperationOptionZero:
+ bzero(buffer, length);
+ copiedDMA = writeBytes(dmaOffset, buffer, length);
+ if (copiedDMA != length) {
+ ret = kIOReturnUnderrun;
+ break;
+ }
+ ret = kIOReturnSuccess;
+ break;
+
+ case kIODMACommandPerformOperationOptionRead:
+ case kIODMACommandPerformOperationOptionWrite:
+
+ if (!data) {
+ ret = kIOReturnBadArgument;
+ break;
+ }
+ if (length > data->getLength()) {
+ ret = kIOReturnBadArgument;
+ break;
+ }
+ if (kIODMACommandPerformOperationOptionWrite == options) {
+ copied = data->readBytes(mdOffset, buffer, mdLength);
+ if (copied != mdLength) {
+ ret = kIOReturnUnderrun;
+ break;
+ }
+ copiedDMA = writeBytes(dmaOffset, buffer, length);
+ if (copiedDMA != length) {
+ ret = kIOReturnUnderrun;
+ break;
+ }
+ } else { /* kIODMACommandPerformOperationOptionRead */
+ copiedDMA = readBytes(dmaOffset, buffer, length);
+ if (copiedDMA != length) {
+ ret = kIOReturnUnderrun;
+ break;
+ }
+ copied = data->writeBytes(mdOffset, buffer, mdLength);
+ if (copied != mdLength) {
+ ret = kIOReturnUnderrun;
+ break;
+ }
+ }
+ ret = kIOReturnSuccess;
+ break;
+ default:
+ ret = kIOReturnBadArgument;
+ break;
+ }
+
+ IODelete(buffer, uint8_t, length);
+
+ return ret;
+}
+
+
/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * */
kern_return_t
super::free();
}
+kern_return_t
+IMPL(IODispatchSource, SetEnable)
+{
+ return SetEnableWithCompletion(enable, NULL);
+}
+
/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * */
struct IOInterruptDispatchSource_IVars {
return ret;
}
+kern_return_t
+IMPL(IOInterruptDispatchSource, GetInterruptType)
+{
+ IOReturn ret;
+ int type;
+
+ *interruptType = 0;
+ ret = provider->getInterruptType(index, &type);
+ if (kIOReturnSuccess == ret) {
+ *interruptType = type;
+ }
+
+ return ret;
+}
+
bool
IOInterruptDispatchSource::init()
{
assert(kIOReturnSuccess == ret);
}
+ if (ivars && ivars->lock) {
+ IOSimpleLockFree(ivars->lock);
+ }
+
IOSafeDeleteNULL(ivars, IOInterruptDispatchSource_IVars, 1);
super::free();
}
kern_return_t
-IMPL(IODispatchSource, SetEnable)
+IMPL(IOInterruptDispatchSource, Cancel)
{
- return SetEnableWithCompletion(enable, NULL);
+ return kIOReturnUnsupported;
}
kern_return_t
/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * */
+enum {
+ kIOServiceNotificationTypeCount = kIOServiceNotificationTypeLast + 1,
+};
+
+struct IOServiceNotificationDispatchSource_IVars {
+ OSObject * serverName;
+ OSAction * action;
+ IOLock * lock;
+ IONotifier * notifier;
+ OSDictionary * interestNotifiers;
+ OSArray * pending[kIOServiceNotificationTypeCount];
+ bool enable;
+};
+
+kern_return_t
+IMPL(IOServiceNotificationDispatchSource, Create)
+{
+ IOUserServer * us;
+ IOReturn ret;
+ IOServiceNotificationDispatchSource * inst;
+
+ inst = OSTypeAlloc(IOServiceNotificationDispatchSource);
+ if (!inst->init()) {
+ OSSafeReleaseNULL(inst);
+ return kIOReturnNoMemory;
+ }
+
+ us = (typeof(us))thread_iokit_tls_get(0);
+ assert(OSDynamicCast(IOUserServer, us));
+ if (!us) {
+ OSSafeReleaseNULL(inst);
+ return kIOReturnError;
+ }
+ inst->ivars->serverName = us->copyProperty(gIOUserServerNameKey);
+ if (!inst->ivars->serverName) {
+ OSSafeReleaseNULL(inst);
+ return kIOReturnNoMemory;
+ }
+
+ inst->ivars->lock = IOLockAlloc();
+ if (!inst->ivars->lock) {
+ OSSafeReleaseNULL(inst);
+ return kIOReturnNoMemory;
+ }
+ for (uint32_t idx = 0; idx < kIOServiceNotificationTypeCount; idx++) {
+ inst->ivars->pending[idx] = OSArray::withCapacity(4);
+ if (!inst->ivars->pending[idx]) {
+ OSSafeReleaseNULL(inst);
+ return kIOReturnNoMemory;
+ }
+ }
+ inst->ivars->interestNotifiers = OSDictionary::withCapacity(4);
+ if (!inst->ivars->interestNotifiers) {
+ OSSafeReleaseNULL(inst);
+ return kIOReturnNoMemory;
+ }
+
+ inst->ivars->notifier = IOService::addMatchingNotification(gIOMatchedNotification, matching, 0 /*priority*/,
+ ^bool (IOService * newService, IONotifier * notifier) {
+ bool notifyReady = false;
+ IONotifier * interest;
+ OSObject * serverName;
+ bool okToUse;
+
+ serverName = newService->copyProperty(gIOUserServerNameKey);
+ okToUse = (serverName && inst->ivars->serverName->isEqualTo(serverName));
+ OSSafeReleaseNULL(serverName);
+ if (!okToUse) {
+ return false;
+ }
+
+ IOLockLock(inst->ivars->lock);
+ notifyReady = (0 == inst->ivars->pending[kIOServiceNotificationTypeMatched]->getCount());
+ inst->ivars->pending[kIOServiceNotificationTypeMatched]->setObject(newService);
+ IOLockUnlock(inst->ivars->lock);
+
+ interest = newService->registerInterest(gIOGeneralInterest,
+ ^IOReturn (uint32_t messageType, IOService * provider,
+ void * messageArgument, size_t argSize) {
+ IONotifier * interest;
+ bool notifyReady = false;
+
+ switch (messageType) {
+ case kIOMessageServiceIsTerminated:
+ IOLockLock(inst->ivars->lock);
+ notifyReady = (0 == inst->ivars->pending[kIOServiceNotificationTypeTerminated]->getCount());
+ inst->ivars->pending[kIOServiceNotificationTypeTerminated]->setObject(provider);
+ interest = (typeof(interest))inst->ivars->interestNotifiers->getObject((const OSSymbol *) newService);
+ assert(interest);
+ interest->remove();
+ inst->ivars->interestNotifiers->removeObject((const OSSymbol *) newService);
+ IOLockUnlock(inst->ivars->lock);
+ break;
+ default:
+ break;
+ }
+ if (notifyReady && inst->ivars->action) {
+ inst->ServiceNotificationReady(inst->ivars->action);
+ }
+ return kIOReturnSuccess;
+ });
+ if (interest) {
+ IOLockLock(inst->ivars->lock);
+ inst->ivars->interestNotifiers->setObject((const OSSymbol *) newService, interest);
+ IOLockUnlock(inst->ivars->lock);
+ }
+ if (notifyReady) {
+ if (inst->ivars->action) {
+ inst->ServiceNotificationReady(inst->ivars->action);
+ }
+ }
+ return false;
+ });
+
+ if (!inst->ivars->notifier) {
+ OSSafeReleaseNULL(inst);
+ ret = kIOReturnError;
+ }
+
+ *notification = inst;
+ ret = kIOReturnSuccess;
+
+ return ret;
+}
+
+kern_return_t
+IMPL(IOServiceNotificationDispatchSource, CopyNextNotification)
+{
+ IOService * next;
+ uint32_t idx;
+
+ IOLockLock(ivars->lock);
+ for (idx = 0; idx < kIOServiceNotificationTypeCount; idx++) {
+ next = (IOService *) ivars->pending[idx]->getObject(0);
+ if (next) {
+ next->retain();
+ ivars->pending[idx]->removeObject(0);
+ break;
+ }
+ }
+ IOLockUnlock(ivars->lock);
+
+ if (idx == kIOServiceNotificationTypeCount) {
+ idx = kIOServiceNotificationTypeNone;
+ }
+ *type = idx;
+ *service = next;
+ *options = 0;
+
+ return kIOReturnSuccess;
+}
+
+bool
+IOServiceNotificationDispatchSource::init()
+{
+ if (!super::init()) {
+ return false;
+ }
+ ivars = IONewZero(IOServiceNotificationDispatchSource_IVars, 1);
+ if (!ivars) {
+ return false;
+ }
+
+ return true;
+}
+
+void
+IOServiceNotificationDispatchSource::free()
+{
+ if (ivars) {
+ OSSafeReleaseNULL(ivars->serverName);
+ if (ivars->interestNotifiers) {
+ ivars->interestNotifiers->iterateObjects(^bool (const OSSymbol * key, OSObject * object) {
+ IONotifier * interest = (typeof(interest))object;
+ interest->remove();
+ return false;
+ });
+ OSSafeReleaseNULL(ivars->interestNotifiers);
+ }
+ for (uint32_t idx = 0; idx < kIOServiceNotificationTypeCount; idx++) {
+ OSSafeReleaseNULL(ivars->pending[idx]);
+ }
+ if (ivars->lock) {
+ IOLockFree(ivars->lock);
+ ivars->lock = NULL;
+ }
+ if (ivars->notifier) {
+ ivars->notifier->remove();
+ ivars->notifier = NULL;
+ }
+ IOSafeDeleteNULL(ivars, IOServiceNotificationDispatchSource_IVars, 1);
+ }
+
+ super::free();
+}
+
+kern_return_t
+IMPL(IOServiceNotificationDispatchSource, SetHandler)
+{
+ IOReturn ret;
+ bool notifyReady;
+
+ notifyReady = false;
+
+ IOLockLock(ivars->lock);
+ OSSafeReleaseNULL(ivars->action);
+ action->retain();
+ ivars->action = action;
+ if (action) {
+ for (uint32_t idx = 0; idx < kIOServiceNotificationTypeCount; idx++) {
+ notifyReady = (ivars->pending[idx]->getCount());
+ if (notifyReady) {
+ break;
+ }
+ }
+ }
+ IOLockUnlock(ivars->lock);
+
+ if (notifyReady) {
+ ServiceNotificationReady(action);
+ }
+ ret = kIOReturnSuccess;
+
+ return ret;
+}
+
+kern_return_t
+IMPL(IOServiceNotificationDispatchSource, SetEnableWithCompletion)
+{
+ if (enable == ivars->enable) {
+ return kIOReturnSuccess;
+ }
+
+ IOLockLock(ivars->lock);
+ ivars->enable = enable;
+ IOLockUnlock(ivars->lock);
+
+ return kIOReturnSuccess;
+}
+
+kern_return_t
+IMPL(IOServiceNotificationDispatchSource, Cancel)
+{
+ return kIOReturnUnsupported;
+}
+
+kern_return_t
+IMPL(IOServiceNotificationDispatchSource, CheckForWork)
+{
+ return kIOReturnNotReady;
+}
+
+kern_return_t
+IOServiceNotificationDispatchSource::DeliverNotifications(IOServiceNotificationBlock block)
+{
+ return kIOReturnUnsupported;
+}
+
+/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * */
+
kern_return_t
IOUserServer::waitInterruptTrap(void * p1, void * p2, void * p3, void * p4, void * p5, void * p6)
{
kern_return_t
IMPL(IODispatchQueue, SetPort)
{
+ if (MACH_PORT_NULL != ivars->serverPort) {
+ return kIOReturnNotReady;
+ }
+
ivars->serverPort = port;
return kIOReturnSuccess;
}
void
IODispatchQueue::free()
{
+ if (ivars && ivars->serverPort) {
+ ipc_port_release_send(ivars->serverPort);
+ ivars->serverPort = MACH_PORT_NULL;
+ }
IOSafeDeleteNULL(ivars, IODispatchQueue_IVars, 1);
super::free();
}
kern_return_t
OSUserMetaClass::Dispatch(IORPC rpc)
{
- return const_cast<OSMetaClass *>(meta)->Dispatch(rpc);
+ if (meta) {
+ return const_cast<OSMetaClass *>(meta)->Dispatch(rpc);
+ } else {
+ return kIOReturnUnsupported;
+ }
}
void
array->count = 0;
cstr = &array->strings[0];
end = &array->strings[array->dataSize];
- while ((len = cstr[0])) {
+ while ((len = (unsigned char)cstr[0])) {
cstr++;
if ((cstr + len) >= end) {
break;
cstr = &array->strings[0];
end = &array->strings[array->dataSize];
llen = strlen(look);
- while ((len = cstr[0])) {
+ while ((len = (unsigned char)cstr[0])) {
cstr++;
if ((cstr + len) >= end) {
break;
resultFlags |= kOSObjectRPCRemote;
}
if (service->reserved->uvars && service->reserved->uvars->userServer) {
+ IOLockLock(service->reserved->uvars->userServer->fLock);
userMeta = (typeof(userMeta))service->reserved->uvars->userServer->fClasses->getObject(str);
+ IOLockUnlock(service->reserved->uvars->userServer->fLock);
}
}
if (!str && !userMeta) {
const OSMetaClass * meta;
meta = obj->getMetaClass();
+ IOLockLock(fLock);
while (meta && !userMeta) {
str = (OSString *) meta->getClassNameSymbol();
userMeta = (typeof(userMeta))fClasses->getObject(str);
meta = meta->getSuperClass();
}
}
+ IOLockUnlock(fLock);
}
if (str) {
if (!userMeta) {
+ IOLockLock(fLock);
userMeta = (typeof(userMeta))fClasses->getObject(str);
+ IOLockUnlock(fLock);
}
if (kIODKLogSetup & gIODKDebug) {
DKLOG("userMeta %s %p\n", str->getCStringNoCopy(), userMeta);
idx = 0;
sendPort = NULL;
if (queue && (kIODispatchQueueStopped != queue)) {
- sendPort = ipc_port_make_send(queue->ivars->serverPort);
+ sendPort = ipc_port_copy_send(queue->ivars->serverPort);
}
replySize = sizeof(OSObject_Instantiate_Rpl)
+ queueCount * sizeof(machReply->objects[0])
queue = uvars->queueArray[idx];
sendPort = NULL;
if (queue) {
- sendPort = ipc_port_make_send(queue->ivars->serverPort);
+ sendPort = ipc_port_copy_send(queue->ivars->serverPort);
}
machReply->objects[idx].type = MACH_MSG_PORT_DESCRIPTOR;
machReply->objects[idx].disposition = MACH_MSG_TYPE_MOVE_SEND;
if (!message) {
return kIOReturnIPCError;
}
+ if (message->objectRefs == 0) {
+ return kIOReturnIPCError;
+ }
ret = copyInObjects(msgin, message, msgin->msgh.msgh_size, true, false);
if (kIOReturnSuccess != ret) {
if (kIODKLogIPC & gIODKDebug) {
bzero((void *)msgout, replyAlloc);
}
- IORPC rpc = { .message = msgin, .sendSize = msgin->msgh.msgh_size, .reply = msgout, .replySize = replyAlloc };
+ IORPC rpc = { .message = msgin, .reply = msgout, .sendSize = msgin->msgh.msgh_size, .replySize = replyAlloc };
if (object) {
thread_iokit_tls_set(0, this);
} else {
objectArg1 = NULL;
if (refs > 1) {
- objectArg1 = iokit_lookup_uext_ref_current_task(objectName1);
- if (!objectArg1) {
- return kIOReturnIPCError;
+ if (objectName1) {
+ objectArg1 = iokit_lookup_uext_ref_current_task(objectName1);
+ if (!objectArg1) {
+ return kIOReturnIPCError;
+ }
}
message->objects[1] = (OSObjectRef) objectArg1;
}
port = queue->ivars->serverPort;
}
if (port) {
- sendPort = ipc_port_make_send(port);
+ sendPort = ipc_port_copy_send(port);
}
IOLockUnlock(gIOUserServerLock);
if (!sendPort) {
ret = copyOutObjects(mach, message, sendSize, false);
mach->msgh.msgh_bits = MACH_MSGH_BITS_COMPLEX |
- MACH_MSGH_BITS(MACH_MSG_TYPE_MOVE_SEND, (oneway ? 0 : MACH_MSG_TYPE_MAKE_SEND_ONCE));
+ MACH_MSGH_BITS(MACH_MSG_TYPE_COPY_SEND, (oneway ? 0 : MACH_MSG_TYPE_MAKE_SEND_ONCE));
mach->msgh.msgh_remote_port = sendPort;
mach->msgh.msgh_local_port = (oneway ? MACH_PORT_NULL : mig_get_reply_port());
mach->msgh.msgh_id = kIORPCVersionCurrent;
mach->msgh.msgh_reserved = 0;
+ boolean_t message_moved;
+
if (oneway) {
- ret = mach_msg_send_from_kernel(&mach->msgh, sendSize);
+ ret = kernel_mach_msg_send(&mach->msgh, sendSize,
+ MACH_SEND_MSG | MACH_SEND_ALWAYS | MACH_SEND_NOIMPORTANCE,
+ 0, &message_moved);
} else {
assert(replySize >= (sizeof(IORPCMessageMach) + sizeof(IORPCMessage)));
- ret = mach_msg_rpc_from_kernel(&mach->msgh, sendSize, replySize);
- if (KERN_SUCCESS == ret) {
- if (kIORPCVersionCurrentReply != mach->msgh.msgh_id) {
- ret = (MACH_NOTIFY_SEND_ONCE == mach->msgh.msgh_id) ? MIG_SERVER_DIED : MIG_REPLY_MISMATCH;
- } else if ((replySize = mach->msgh.msgh_size) < (sizeof(IORPCMessageMach) + sizeof(IORPCMessage))) {
+ ret = kernel_mach_msg_rpc(&mach->msgh, sendSize, replySize, FALSE, &message_moved);
+ }
+
+ ipc_port_release_send(sendPort);
+
+ if (MACH_MSG_SUCCESS != ret) {
+ if (kIODKLogIPC & gIODKDebug) {
+ DKLOG("mach_msg() failed 0x%x\n", ret);
+ }
+ if (!message_moved) {
+ // release ports
+ copyInObjects(mach, message, sendSize, false, true);
+ }
+ }
+
+ if ((KERN_SUCCESS == ret) && !oneway) {
+ if (kIORPCVersionCurrentReply != mach->msgh.msgh_id) {
+ ret = (MACH_NOTIFY_SEND_ONCE == mach->msgh.msgh_id) ? MIG_SERVER_DIED : MIG_REPLY_MISMATCH;
+ } else if ((replySize = mach->msgh.msgh_size) < (sizeof(IORPCMessageMach) + sizeof(IORPCMessage))) {
// printf("BAD REPLY SIZE\n");
+ ret = MIG_BAD_ARGUMENTS;
+ } else {
+ if (!(MACH_MSGH_BITS_COMPLEX & mach->msgh.msgh_bits)) {
+ mach->msgh_body.msgh_descriptor_count = 0;
+ }
+ message = IORPCMessageFromMach(mach, true);
+ if (!message) {
+ ret = kIOReturnIPCError;
+ } else if (message->msgid != msgid) {
+// printf("BAD REPLY ID\n");
ret = MIG_BAD_ARGUMENTS;
} else {
- if (!(MACH_MSGH_BITS_COMPLEX & mach->msgh.msgh_bits)) {
- mach->msgh_body.msgh_descriptor_count = 0;
- }
- message = IORPCMessageFromMach(mach, true);
- if (!message) {
- ret = kIOReturnIPCError;
- } else if (message->msgid != msgid) {
-// printf("BAD REPLY ID\n");
- ret = MIG_BAD_ARGUMENTS;
- } else {
- bool isError = (0 != (kIORPCMessageError & message->flags));
- ret = copyInObjects(mach, message, replySize, !isError, true);
- if (kIOReturnSuccess != ret) {
- if (kIODKLogIPC & gIODKDebug) {
- DKLOG("rpc copyin(0x%x) %x\n", ret, mach->msgh.msgh_id);
- }
- return KERN_NOT_SUPPORTED;
- }
- if (isError) {
- IORPCMessageErrorReturnContent * errorMsg = (typeof(errorMsg))message;
- ret = errorMsg->result;
+ bool isError = (0 != (kIORPCMessageError & message->flags));
+ ret = copyInObjects(mach, message, replySize, !isError, true);
+ if (kIOReturnSuccess != ret) {
+ if (kIODKLogIPC & gIODKDebug) {
+ DKLOG("rpc copyin(0x%x) %x\n", ret, mach->msgh.msgh_id);
}
+ return KERN_NOT_SUPPORTED;
+ }
+ if (isError) {
+ IORPCMessageErrorReturnContent * errorMsg = (typeof(errorMsg))message;
+ ret = errorMsg->result;
}
}
}
}
}
+ /* Mark the current task's space as eligible for uext object ports */
+ iokit_label_dext_task(inst->fOwningTask);
+
inst->fLock = IOLockAlloc();
inst->fServices = OSArray::withCapacity(4);
inst->fClasses = OSDictionary::withCapacity(16);
cls->name = sym;
cls->meta = OSMetaClass::copyMetaClassWithName(sym);
+ IOLockLock(fLock);
cls->superMeta = OSDynamicCast(OSUserMetaClass, fClasses->getObject(desc->superName));
- fClasses->setObject(sym, cls);
+ if (fClasses->getObject(sym) != NULL) {
+ /* class with this name exists */
+ ret = kIOReturnBadArgument;
+ } else {
+ if (fClasses->setObject(sym, cls)) {
+ *pCls = cls;
+ } else {
+ /* could not add class to fClasses */
+ ret = kIOReturnNoMemory;
+ }
+ }
+ IOLockUnlock(fLock);
cls->release();
-
- *pCls = cls;
-
return ret;
}
userUC->setTask(owningTask);
if (!(kIODKDisableEntitlementChecking & gIODKDebug)) {
- entitlements = IOUserClient::copyClientEntitlements(owningTask);
- bundleID = service->copyProperty(gIOModuleIdentifierKey);
- ok = (entitlements
- && bundleID
- && (prop = entitlements->getObject(gIODriverKitUserClientEntitlementsKey)));
- if (ok) {
- bool found __block = false;
- ok = prop->iterateObjects(^bool (OSObject * object) {
- found = object->isEqualTo(bundleID);
- return found;
- });
- ok = found;
+ bundleID = NULL;
+ entitlements = NULL;
+ if (fEntitlements && fEntitlements->getObject(gIODriverKitUserClientEntitlementAllowAnyKey)) {
+ ok = true;
+ } else {
+ entitlements = IOUserClient::copyClientEntitlements(owningTask);
+ bundleID = service->copyProperty(gIOModuleIdentifierKey);
+ ok = (entitlements
+ && bundleID
+ && (prop = entitlements->getObject(gIODriverKitUserClientEntitlementsKey)));
+ if (ok) {
+ bool found __block = false;
+ ok = prop->iterateObjects(^bool (OSObject * object) {
+ found = object->isEqualTo(bundleID);
+ return found;
+ });
+ ok = found;
+ }
}
if (ok) {
prop = userUC->copyProperty(gIOServiceDEXTEntitlementsKey);
}
}
- ret = userUC->Start(service);
- if (kIOReturnSuccess != ret) {
- userUC->detach(this);
- userUC->release();
- return ret;
- }
-
*handler = userUC;
return ret;
service = OSDynamicCast(IOService, inst);
if (service && service->init(properties) && service->attach(this)) {
reserved->uvars->userServer->serviceAttach(service, this);
+ service->reserved->uvars->started = true;
ret = kIOReturnSuccess;
*result = service;
}
return ret;
}
+kern_return_t
+IMPL(IOService, Terminate)
+{
+ IOUserServer * us;
+
+ if (options) {
+ return kIOReturnUnsupported;
+ }
+
+ us = (typeof(us))thread_iokit_tls_get(0);
+ if (!reserved->uvars
+ || (reserved->uvars->userServer != us)) {
+ return kIOReturnNotPermitted;
+ }
+ terminate(kIOServiceTerminateNeedWillTerminate);
+
+ return kIOReturnSuccess;
+}
+
kern_return_t
IMPL(IOService, NewUserClient)
{
return object ? kIOReturnSuccess : kIOReturnNotFound;
}
+kern_return_t
+IMPL(IOService, CopyProviderProperties)
+{
+ IOReturn ret;
+ OSArray * result;
+ IOService * provider;
+
+ result = OSArray::withCapacity(8);
+ if (!result) {
+ return kIOReturnNoMemory;
+ }
+
+ ret = kIOReturnSuccess;
+ for (provider = this; provider; provider = provider->getProvider()) {
+ OSObject * obj;
+ OSDictionary * props;
+
+ obj = provider->copyProperty(gIOSupportedPropertiesKey);
+ props = OSDynamicCast(OSDictionary, obj);
+ if (!props) {
+ OSSafeReleaseNULL(obj);
+ props = provider->dictionaryWithProperties();
+ }
+ if (!props) {
+ ret = kIOReturnNoMemory;
+ break;
+ }
+ bool __block addClass = true;
+ if (propertyKeys) {
+ OSDictionary * retProps;
+ retProps = OSDictionary::withCapacity(4);
+ addClass = false;
+ if (!retProps) {
+ ret = kIOReturnNoMemory;
+ break;
+ }
+ propertyKeys->iterateObjects(^bool (OSObject * _key) {
+ OSString * key = OSDynamicCast(OSString, _key);
+ if (gIOClassKey->isEqualTo(key)) {
+ addClass = true;
+ return false;
+ }
+ retProps->setObject(key, props->getObject(key));
+ return false;
+ });
+ OSSafeReleaseNULL(props);
+ props = retProps;
+ }
+ if (addClass) {
+ OSArray * classes = OSArray::withCapacity(8);
+ if (!classes) {
+ ret = kIOReturnNoMemory;
+ break;
+ }
+ for (const OSMetaClass * meta = provider->getMetaClass(); meta; meta = meta->getSuperClass()) {
+ classes->setObject(meta->getClassNameSymbol());
+ }
+ props->setObject(gIOClassKey, classes);
+ OSSafeReleaseNULL(classes);
+ }
+ bool ok = result->setObject(props);
+ props->release();
+ if (!ok) {
+ ret = kIOReturnNoMemory;
+ break;
+ }
+ }
+ if (kIOReturnSuccess != ret) {
+ OSSafeReleaseNULL(result);
+ }
+ *properties = result;
+ return ret;
+}
+
void
IOUserServer::systemPower(bool powerOff)
{
return kIOReturnSuccess;
}
- IOMachPortDestroyUserReferences(service, IKOT_UEXT_OBJECT);
-
if (uvars->queueArray && uvars->userMeta) {
queueAlloc = 1;
if (uvars->userMeta->queueNames) {
return kIOReturnSuccess;
}
-kern_return_t
-IMPL(IOInterruptDispatchSource, Cancel)
-{
- return kIOReturnUnsupported;
-}
-
/* * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * * */
#undef super
IOReturn
IOUserUserClient::clientClose(void)
{
- terminate();
+ terminate(kIOServiceTerminateNeedWillTerminate);
return kIOReturnSuccess;
}